Copying existing implementation from old Beam branch
diff --git a/Sources/ApacheBeam/Client/ApiServiceDescriptor.swift b/Sources/ApacheBeam/Client/ApiServiceDescriptor.swift
new file mode 100644
index 0000000..66aeaea
--- /dev/null
+++ b/Sources/ApacheBeam/Client/ApiServiceDescriptor.swift
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import GRPC
+import NIOCore
+
+/// Representation of the API Service Descriptors used to communicate with runners (and vice versa)
+public struct ApiServiceDescriptor {
+ public enum EncodedAs {
+ case json, textproto
+ }
+
+ let url: String
+
+ public init(host: String, port: Int) {
+ url = "\(host):\(port)"
+ }
+
+ public init(unixAddress: String) {
+ url = "unix://\(unixAddress)"
+ }
+}
+
+extension ApiServiceDescriptor {
+ init(proto: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor) {
+ url = proto.url
+ }
+}
+
+extension ApiServiceDescriptor: ProtoConversion {
+ func populate(_ proto: inout Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor) throws {
+ proto.url = url
+ }
+}
+
+extension ApiServiceDescriptor: Hashable {}
+
+public extension ApiServiceDescriptor {
+ static func from(env: String, format: EncodedAs = .textproto) throws -> ApiServiceDescriptor {
+ switch format {
+ case .textproto:
+ try ApiServiceDescriptor(proto: .init(textFormatString: env))
+ case .json:
+ try ApiServiceDescriptor(proto: .init(jsonString: env))
+ }
+ }
+}
+
+public extension GRPCChannelPool {
+ static func with(endpoint: ApiServiceDescriptor, eventLoopGroup: EventLoopGroup) throws -> GRPCChannel {
+ let url = endpoint.url
+ // TODO: Transport Security configuration
+ if url.starts(with: "unix://") {
+ return try GRPCChannelPool.with(target: .unixDomainSocket(url.replacing("unix://", with: "")),
+ transportSecurity: .plaintext,
+ eventLoopGroup: eventLoopGroup)
+ } else {
+ if let lastNdx = url.lastIndex(of: ":") {
+ guard lastNdx.utf16Offset(in: url) > 0 else {
+ throw ApacheBeamError.runtimeError("Service URL must be of the form host:port")
+ }
+ let host = String(url.prefix(upTo: lastNdx))
+ let port = Int(url.suffix(from: url.index(lastNdx, offsetBy: 1)))!
+ return try GRPCChannelPool.with(target: .host(host, port: port),
+ transportSecurity: .plaintext,
+ eventLoopGroup: eventLoopGroup)
+ {
+ $0.maximumReceiveMessageLength = .max
+ }
+ } else {
+ throw ApacheBeamError.runtimeError("Service URL must be of the form host:port")
+ }
+ }
+ }
+}
diff --git a/Sources/ApacheBeam/Client/ExpansionClient.swift b/Sources/ApacheBeam/Client/ExpansionClient.swift
new file mode 100644
index 0000000..60b8c9f
--- /dev/null
+++ b/Sources/ApacheBeam/Client/ExpansionClient.swift
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+import GRPC
+
+public struct ExpansionClient {
+ let client: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceAsyncClient
+
+ public init(endpoint: ApiServiceDescriptor) throws {
+ client = try Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceAsyncClient(channel: GRPCChannelPool.with(endpoint: endpoint, eventLoopGroup: PlatformSupport.makeEventLoopGroup(loopCount: 1)))
+ }
+
+ public func transforms() async throws -> [String: ([String], [String], Schema)] {
+ let call = try await client.makeDiscoverSchemaTransformCall(.with { _ in }).response
+ var transforms: [String: ([String], [String], Schema)] = [:]
+ for (k, v) in call.schemaTransformConfigs {
+ transforms[k] = (v.inputPcollectionNames, v.outputPcollectionNames, .from(v.configSchema))
+ }
+ return transforms
+ }
+}
diff --git a/Sources/ApacheBeam/Coders/BeamValue.swift b/Sources/ApacheBeam/Coders/BeamValue.swift
new file mode 100644
index 0000000..97f1ccd
--- /dev/null
+++ b/Sources/ApacheBeam/Coders/BeamValue.swift
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+/// An enum representing values coming over the FnApi Data Plane.
+public indirect enum BeamValue {
+ /// A value not representable in the Swift SDK
+ case invalid(String)
+
+ // Scalar values
+
+ /// Bytes coded
+ case bytes(Data?)
+ /// UTF8 Strings
+ case string(String?)
+ /// Integers (Signed 64-bit)
+ case integer(Int?)
+ /// Doubles
+ case double(Double?)
+ /// Booleans
+ case boolean(Bool?)
+ /// A window
+ case window(Window)
+ /// Schema-valued thing. Doesn't technically need to be a row, but that's the only coder support.
+ case row(FieldValue)
+
+ // Composite Values
+
+ /// An iterable
+ case array([BeamValue])
+ /// A key-value pair
+ case kv(BeamValue, BeamValue)
+ /// A windowed value
+ case windowed(BeamValue, Date, UInt8, BeamValue)
+
+ /// Convenience method for extacting the base value from one
+ /// of the scalar representations.
+ var baseValue: Any? {
+ switch self {
+ case let .bytes(d):
+ d
+ case let .string(s):
+ s
+ case let .integer(i):
+ i
+ case let .double(d):
+ d
+ case let .boolean(b):
+ b
+ case let .window(w):
+ w
+ case let .row(r):
+ r.baseValue
+ default:
+ nil as Any?
+ }
+ }
+}
diff --git a/Sources/ApacheBeam/Coders/Beamable.swift b/Sources/ApacheBeam/Coders/Beamable.swift
new file mode 100644
index 0000000..dd85f64
--- /dev/null
+++ b/Sources/ApacheBeam/Coders/Beamable.swift
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+/// Codable is already taken and besides Beamable is too good to pass up
+public protocol Beamable {
+ static var coder: Coder { get }
+}
+
+extension Data: Beamable {
+ public static let coder: Coder = .bytes
+}
+
+extension String: Beamable {
+ public static let coder: Coder = .string
+}
+
+extension Int: Beamable {
+ public static let coder: Coder = .varint
+}
+
+extension Bool: Beamable {
+ public static let coder: Coder = .boolean
+}
diff --git a/Sources/ApacheBeam/Coders/Coder+Decoding.swift b/Sources/ApacheBeam/Coders/Coder+Decoding.swift
new file mode 100644
index 0000000..0ee82f0
--- /dev/null
+++ b/Sources/ApacheBeam/Coders/Coder+Decoding.swift
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+/// This extension contains all of the decoding implementation. File separation is for clarity.
+public extension Coder {
+ /// Decodes a raw data block into a BeamValue for further processing
+ func decode(_ data: inout Data) throws -> BeamValue {
+ switch self {
+ // Scalar values check for size 0 input data and return null if that's a problem
+
+ case .bytes:
+ return try .bytes(data.count == 0 ? Data() : data.subdata())
+ case .string:
+ return try .string(data.count == 0 ? "" : String(data: data.subdata(), encoding: .utf8))
+ case .varint:
+ return try .integer(data.count == 0 ? nil : data.varint())
+ case .fixedint:
+ return try .integer(data.count == 0 ? nil : data.next(Int.self))
+ case .byte:
+ return try .integer(data.count == 0 ? nil : Int(data.next(UInt8.self)))
+ case .boolean:
+ return try .boolean(data.count == 0 ? nil : data.next(UInt8.self) != 0)
+ case .double:
+ return try .double(data.count == 0 ? nil : data.next(Double.self))
+ case .globalwindow:
+ return .window(.global)
+ case let .lengthprefix(coder): // Length prefix basically serves to make values nullable
+ var subdata = try data.subdata()
+ return try coder.decode(&subdata)
+ case let .keyvalue(keyCoder, valueCoder):
+ return try .kv(keyCoder.decode(&data), valueCoder.decode(&data))
+ case let .iterable(coder):
+ let length = try data.next(Int32.self)
+ return try .array((0 ..< length).map { _ in try coder.decode(&data) })
+ case let .windowedvalue(valueCoder, windowCoder):
+ // This will be big endian to match java
+ let timestamp = try data.instant()
+
+ let windowCount = try data.next(Int32.self)
+ if windowCount > 1 {
+ throw ApacheBeamError.runtimeError("Windowed values with > 1 window not yet supported")
+ }
+ let window = try windowCoder.decode(&data)
+
+ // TODO: Actually handle pane info
+ let pane = try data.next(UInt8.self)
+ switch (pane >> 4) & 0x0F {
+ case 0x0:
+ break
+ case 0x1:
+ _ = try data.varint()
+ case 0x2:
+ _ = try data.varint()
+ _ = try data.varint()
+ default:
+ throw ApacheBeamError.runtimeError("Invalid pane encoding \(String(pane, radix: 2))")
+ }
+ return try .windowed(valueCoder.decode(&data), timestamp, pane, window)
+ case let .row(schema):
+ return try .row(.from(data: &data, as: .row(schema))!)
+ default:
+ throw ApacheBeamError.runtimeError("Decoding of \(urn) coders not supported.")
+ }
+ }
+}
diff --git a/Sources/ApacheBeam/Coders/Coder+Encoding.swift b/Sources/ApacheBeam/Coders/Coder+Encoding.swift
new file mode 100644
index 0000000..9f0218f
--- /dev/null
+++ b/Sources/ApacheBeam/Coders/Coder+Encoding.swift
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+public extension Coder {
+ func encode(_ value: Any?) throws -> Data {
+ var data = Data()
+ try encode(value, data: &data)
+ return data
+ }
+
+ func encode(_ value: Any?, data: inout Data) throws {
+ switch self {
+ // Scalar values check for size 0 input data and return null if that's a problem
+
+ // TODO: Endian and other encoding checks
+
+ case .bytes:
+ if let v = value as? Data {
+ data.varint(v.count)
+ data.append(v)
+ }
+ case .string:
+ if let v = value as? String {
+ let d = Data(v.utf8)
+ data.varint(d.count)
+ data.append(d)
+ }
+ case .varint:
+ if let v = value as? Int {
+ data.varint(v)
+ }
+ case .fixedint:
+ if let v = value as? Int {
+ data.next(v)
+ }
+ case .byte:
+ if let v = value as? UInt8 {
+ data.next(v)
+ }
+ case .boolean:
+ if let v = value as? Bool {
+ let byte: UInt8 = v ? 1 : 0
+ data.next(byte)
+ }
+ case .double:
+ if let v = value as? Double {
+ data.next(v)
+ }
+ case .globalwindow:
+ break
+ case let .row(coderSchema):
+ if let fieldValue = value as? FieldValue {
+ if case let .row(schema, _) = fieldValue {
+ guard schema == coderSchema else {
+ // FUTURE: Should we have a less strict schema conformance here?
+ throw ApacheBeamError.runtimeError("\(coderSchema) does not match \(schema)")
+ }
+ try data.next(fieldValue)
+ } else {
+ throw ApacheBeamError.runtimeError("Row coder can only encode rows not \(fieldValue)")
+ }
+ }
+ case let .lengthprefix(coder):
+ let subData = try coder.encode(value)
+ data.varint(subData.count)
+ data.append(subData)
+ case let .keyvalue(keyCoder, valueCoder):
+ if let v = value as? AnyKeyValue {
+ try keyCoder.encode(v.anyKey, data: &data)
+ // We do a special case check here to account for the fact that
+ // keyvalue is used both for group by as well as a pair type
+ switch valueCoder {
+ case .iterable:
+ try valueCoder.encode(v.anyValues, data: &data)
+ default:
+ try valueCoder.encode(v.anyValue, data: &data)
+ }
+ }
+ case let .iterable(coder):
+ if let v = value as? [Any] {
+ data.next(Int32(truncatingIfNeeded: v.count))
+ for item in v {
+ try coder.encode(item, data: &data)
+ }
+ }
+ case let .windowedvalue(valueCoder, windowCoder):
+ if let (v, ts, w) = value as? (Any, Date, Window) {
+ // Timestamp
+ data.instant(ts)
+ switch w {
+ case .global:
+ data.next(Int32(1))
+ default:
+ data.next(Int32(1))
+ try windowCoder.encode(w, data: &data)
+ }
+ // TODO: Real Panes
+ data.append(UInt8(1 >> 5 | 1 >> 6 | 1 >> 7))
+ try valueCoder.encode(v, data: &data)
+ }
+ default:
+ throw ApacheBeamError.runtimeError("Encoding of \(urn) coders not supported.")
+ }
+ }
+}
diff --git a/Sources/ApacheBeam/Coders/Coder.swift b/Sources/ApacheBeam/Coders/Coder.swift
new file mode 100644
index 0000000..20c58c7
--- /dev/null
+++ b/Sources/ApacheBeam/Coders/Coder.swift
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+public indirect enum Coder {
+ /// Catch-all for coders we don't understand. Mostly used for error reporting
+ case unknown(String)
+ // TODO: Actually implement this
+ case custom(Data)
+
+ /// Standard scalar coders. Does not necessarily correspond 1:1 with BeamValue. For example, varint and fixedint both map to integer
+ case double, varint, fixedint, byte, bytes, string, boolean, globalwindow
+
+ /// Composite coders.
+ case keyvalue(Coder, Coder)
+ case iterable(Coder)
+ case lengthprefix(Coder)
+ case windowedvalue(Coder, Coder)
+
+ /// Schema-valued things
+ case row(Schema)
+}
+
+public extension Coder {
+ var urn: String {
+ switch self {
+ case let .unknown(name):
+ .coderUrn(name)
+ case .custom:
+ .coderUrn("custom")
+ case .double:
+ .coderUrn("double")
+ case .varint:
+ .coderUrn("varint")
+ case .fixedint:
+ .coderUrn("integer")
+ case .bytes:
+ .coderUrn("bytes")
+ case .byte:
+ .coderUrn("byte")
+ case .string:
+ .coderUrn("string_utf8")
+ case .boolean:
+ .coderUrn("bool")
+ case .globalwindow:
+ .coderUrn("global_window")
+ case .keyvalue:
+ .coderUrn("kv")
+ case .iterable:
+ .coderUrn("iterable")
+ case .lengthprefix:
+ .coderUrn("length_prefix")
+ case .windowedvalue:
+ .coderUrn("windowed_value")
+ case .row:
+ .coderUrn("row")
+ }
+ }
+
+ /// Static list of coders for use in capabilities arrays in environments.
+ static let capabilities: [String] = ["byte", "bytes", "bool", "varint", "double", "integer", "string_utf8", "length_prefix", "kv", "iterable", "windowed_value", "global_window"]
+ .map { .coderUrn($0) }
+}
+
+extension Coder: Hashable {
+ public func hash(into hasher: inout Hasher) {
+ hasher.combine(urn)
+ switch self {
+ case let .keyvalue(k, v):
+ k.hash(into: &hasher)
+ v.hash(into: &hasher)
+ case let .iterable(c):
+ c.hash(into: &hasher)
+ case let .lengthprefix(c):
+ c.hash(into: &hasher)
+ case let .windowedvalue(v, w):
+ v.hash(into: &hasher)
+ w.hash(into: &hasher)
+ default:
+ break
+ }
+ }
+}
+
+extension Coder: Equatable {
+ public static func == (_ lhs: Coder, _ rhs: Coder) -> Bool {
+ switch (lhs, rhs) {
+ case let (.unknown(a), .unknown(b)):
+ a == b
+ case let (.custom(a), .custom(b)):
+ a == b
+ case (.double, .double):
+ true
+ case (.varint, .varint):
+ true
+ case (.fixedint, .fixedint):
+ true
+ case (.bytes, .bytes):
+ true
+ case (.byte, .byte):
+ true
+ case (.string, .string):
+ true
+ case (.boolean, .boolean):
+ true
+ case (.globalwindow, .globalwindow):
+ true
+ case let (.row(ls), .row(rs)):
+ ls == rs
+ case let (.keyvalue(lk, lv), .keyvalue(rk, rv)):
+ lk == rk && lv == rv
+ case let (.iterable(a), .iterable(b)):
+ a == b
+ case let (.lengthprefix(a), .lengthprefix(b)):
+ a == b
+ case let (.windowedvalue(lv, lw), .windowedvalue(rv, rw)):
+ lv == rv && lw == rw
+ default:
+ false
+ }
+ }
+}
+
+protocol CoderContainer {
+ subscript(_: String) -> CoderProto? { get }
+}
+
+struct PipelineCoderContainer: CoderContainer {
+ let pipeline: PipelineProto
+ subscript(name: String) -> CoderProto? {
+ pipeline.components.coders[name]
+ }
+}
+
+struct BundleCoderContainer: CoderContainer {
+ let bundle: ProcessBundleDescriptorProto
+ subscript(name: String) -> CoderProto? {
+ bundle.coders[name]
+ }
+}
+
+extension Coder {
+ static func of(name: String, in container: CoderContainer) throws -> Coder {
+ if let baseCoder = container[name] {
+ switch baseCoder.spec.urn {
+ case "beam:coder:bytes:v1":
+ return .bytes
+ case "beam:coder:varint:v1":
+ return .varint
+ case "beam:coder:string_utf8:v1":
+ return .string
+ case "beam:coder:double:v1":
+ return .double
+ case "beam:coder:iterable:v1":
+ return try .iterable(.of(name: baseCoder.componentCoderIds[0], in: container))
+ case "beam:coder:kv:v1":
+ return try .keyvalue(
+ .of(name: baseCoder.componentCoderIds[0], in: container),
+ .of(name: baseCoder.componentCoderIds[1], in: container)
+ )
+ case "beam:coder:global_window:v1":
+ return .globalwindow
+ case "beam:coder:windowed_value:v1":
+ return try .windowedvalue(
+ .of(name: baseCoder.componentCoderIds[0], in: container),
+ .of(name: baseCoder.componentCoderIds[1], in: container)
+ )
+ case "beam:coder:length_prefix:v1":
+ return try .lengthprefix(.of(name: baseCoder.componentCoderIds[0], in: container))
+ case "beam:coder:row:v1":
+ let proto: SchemaProto = try SchemaProto(serializedData: baseCoder.spec.payload)
+ return .row(.from(proto))
+ default:
+ return .unknown(baseCoder.spec.urn)
+ }
+ } else {
+ throw ApacheBeamError.runtimeError("Unable to location coder \(name) in container.")
+ }
+ }
+}
+
+public extension Coder {
+ static func of<Of>(type _: Of?.Type) -> Coder? {
+ .lengthprefix(.of(type: Of.self)!)
+ }
+
+ static func of<Of>(type _: [Of].Type) -> Coder? {
+ .iterable(.of(type: Of.self)!)
+ }
+
+ static func of<Of>(type _: Of.Type) -> Coder? {
+ // Beamables provider their own default coder implementation
+ if let beamable = Of.self as? Beamable.Type {
+ return beamable.coder
+ }
+ return nil
+ }
+}
diff --git a/Sources/ApacheBeam/Core/ArtifactInfo.swift b/Sources/ApacheBeam/Core/ArtifactInfo.swift
new file mode 100644
index 0000000..b7df6e3
--- /dev/null
+++ b/Sources/ApacheBeam/Core/ArtifactInfo.swift
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public struct ArtifactInfo {
+ let role: String
+ let type: String
+}
+
+extension ArtifactInfo: ProtoConversion {
+ func populate(_ proto: inout Org_Apache_Beam_Model_Pipeline_V1_ArtifactInformation) throws {
+ proto.roleUrn = .beamUrn(role, type: "artifact:type")
+ proto.typeUrn = .beamUrn(type, type: "artifact:role")
+ }
+}
diff --git a/Sources/ApacheBeam/Core/DynamicProperties.swift b/Sources/ApacheBeam/Core/DynamicProperties.swift
new file mode 100644
index 0000000..f6525b0
--- /dev/null
+++ b/Sources/ApacheBeam/Core/DynamicProperties.swift
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Logging
+
+public protocol DynamicProperty {}
+
+@propertyWrapper
+public struct RemoteLog: DynamicProperty {
+ public var wrappedValue: Logging.Logger
+ public init(wrappedValue: Logging.Logger = Logging.Logger(label: "TEST")) {
+ self.wrappedValue = wrappedValue
+ }
+}
+
+@propertyWrapper
+public struct Serialized<Value: Codable>: DynamicProperty {
+ public var wrappedValue: Value?
+ public init(wrappedValue: Value? = nil) {
+ self.wrappedValue = wrappedValue
+ }
+}
diff --git a/Sources/ApacheBeam/Core/Environment.swift b/Sources/ApacheBeam/Core/Environment.swift
new file mode 100644
index 0000000..15e67ab
--- /dev/null
+++ b/Sources/ApacheBeam/Core/Environment.swift
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/// A struct that encodes the different types of available in Beam
+public struct Environment {
+ public enum Category {
+ /// Default environment type. "default" is a reserved word so we use "system" here
+ case system
+ /// Process. command, arch, os, environment
+ case process(String, String, String, [String: String])
+ /// Docker container image
+ case docker(String)
+ /// External service using an api descriptor
+ case external(ApiServiceDescriptor)
+ }
+
+ let category: Category
+ let capabilities: [String]
+ let dependencies: [ArtifactInfo]
+
+ public init(_ category: Category = .system, capabilities: [String] = [], dependencies: [ArtifactInfo]) {
+ self.category = category
+ self.capabilities = capabilities
+ self.dependencies = dependencies
+ }
+}
+
+extension Environment: ProtoConversion {
+ func populate(_ proto: inout EnvironmentProto) throws {
+ proto.urn = switch category {
+ case .docker: .beamUrn("docker", type: "env")
+ case .system: .beamUrn("default", type: "env")
+ case .process: .beamUrn("process", type: "env")
+ case .external: .beamUrn("external", type: "env")
+ }
+ proto.capabilities = capabilities
+
+ proto.dependencies = try dependencies.map { artifact in
+ try .with {
+ try artifact.populate(&$0)
+ }
+ }
+
+ if case let .docker(containerImage) = category {
+ proto.payload = try Org_Apache_Beam_Model_Pipeline_V1_DockerPayload.with {
+ $0.containerImage = containerImage
+ }.serializedData()
+ }
+
+ if case let .external(endpoint) = category {
+ proto.payload = try Org_Apache_Beam_Model_Pipeline_V1_ExternalPayload.with {
+ $0.endpoint = try .with {
+ try endpoint.populate(&$0)
+ }
+ }.serializedData()
+ }
+
+ if case let .process(command, arch, os, env) = category {
+ proto.payload = try Org_Apache_Beam_Model_Pipeline_V1_ProcessPayload.with {
+ $0.arch = arch
+ $0.command = command
+ $0.os = os
+ $0.env = env
+ }.serializedData()
+ }
+ }
+
+ public static func docker(_ imageName: String, capabilities: [String] = [], dependencies: [ArtifactInfo] = []) -> Environment {
+ Environment(.docker(imageName), capabilities: capabilities, dependencies: dependencies)
+ }
+
+ public static func external(_ endpoint: ApiServiceDescriptor, capabilities: [String] = [], dependencies: [ArtifactInfo] = []) -> Environment {
+ Environment(.external(endpoint), capabilities: capabilities, dependencies: dependencies)
+ }
+}
diff --git a/Sources/ApacheBeam/Core/Fn/ClosureFn.swift b/Sources/ApacheBeam/Core/Fn/ClosureFn.swift
new file mode 100644
index 0000000..e44eb6f
--- /dev/null
+++ b/Sources/ApacheBeam/Core/Fn/ClosureFn.swift
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+/// A SerializableFn that holds a reference to a function that takes a single input and produces a variable number of outputs
+public final class ClosureFn: SerializableFn {
+ let processClosure: (SerializableFnBundleContext, [AnyPCollectionStream], [AnyPCollectionStream]) async throws -> Void
+
+ public init<Of>(_ fn: @Sendable @escaping (PCollection<Of>.Stream) async throws -> Void) {
+ processClosure = { _, inputs, _ in
+ try await fn(inputs[0].stream())
+ }
+ }
+
+ public init<Of, O0>(_ fn: @Sendable @escaping (PCollection<Of>.Stream, PCollection<O0>.Stream) async throws -> Void) {
+ processClosure = { _, inputs, outputs in
+ try await fn(inputs[0].stream(), outputs[0].stream())
+ }
+ }
+
+ public init<Of, O0, O1>(_ fn: @Sendable @escaping (PCollection<Of>.Stream, PCollection<O0>.Stream, PCollection<O1>.Stream) async throws -> Void) {
+ processClosure = { _, inputs, outputs in
+ try await fn(inputs[0].stream(), outputs[0].stream(), outputs[1].stream())
+ }
+ }
+
+ public init<Of, O0, O1, O2>(_ fn: @Sendable @escaping (PCollection<Of>.Stream, PCollection<O0>.Stream, PCollection<O1>.Stream, PCollection<O2>.Stream) async throws -> Void) {
+ processClosure = { _, inputs, outputs in
+ try await fn(inputs[0].stream(), outputs[0].stream(), outputs[1].stream(), outputs[2].stream())
+ }
+ }
+
+ public init<Of, O0, O1, O2, O3>(_ fn: @Sendable @escaping (PCollection<Of>.Stream, PCollection<O0>.Stream, PCollection<O1>.Stream, PCollection<O2>.Stream, PCollection<O3>.Stream) async throws -> Void) {
+ processClosure = { _, inputs, outputs in
+ try await fn(inputs[0].stream(), outputs[0].stream(), outputs[1].stream(), outputs[2].stream(), outputs[3].stream())
+ }
+ }
+
+ public func process(context: SerializableFnBundleContext, inputs: [AnyPCollectionStream], outputs: [AnyPCollectionStream]) async throws -> (String, String) {
+ try await processClosure(context, inputs, outputs)
+ outputs.finish()
+ return (context.instruction, context.transform)
+ }
+}
diff --git a/Sources/ApacheBeam/Core/Fn/DoFn.swift b/Sources/ApacheBeam/Core/Fn/DoFn.swift
new file mode 100644
index 0000000..cf5c7a9
--- /dev/null
+++ b/Sources/ApacheBeam/Core/Fn/DoFn.swift
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+/// A higher level interface to SerializableFn using dependency injected dynamic properties in the same
+/// way as we define Composite PTransforms
+public protocol DoFn {
+ func process() async throws
+ func finishBundle() async throws
+}
+
+public extension DoFn {
+ func finishBundle() async throws {}
+}
+
+public struct PInput<Of> {
+ public let value: Of
+ public let timestamp: Date
+ public let window: Window
+
+ public init(_ value: Of, _ timestamp: Date, _ window: Window) {
+ self.value = value
+ self.timestamp = timestamp
+ self.window = window
+ }
+
+ public init(_ element: (Of, Date, Window)) {
+ value = element.0
+ timestamp = element.1
+ window = element.2
+ }
+}
+
+public struct POutput<Of> {
+ let stream: PCollectionStream<Of>
+ let timestamp: Date
+ let window: Window
+
+ func emit(_ value: Of, timestamp: Date? = nil, window: Window? = nil) {
+ stream.emit(value, timestamp: timestamp ?? self.timestamp, window: window ?? self.window)
+ }
+}
diff --git a/Sources/ApacheBeam/Core/Fn/ObjectFn.swift b/Sources/ApacheBeam/Core/Fn/ObjectFn.swift
new file mode 100644
index 0000000..0dcfbbd
--- /dev/null
+++ b/Sources/ApacheBeam/Core/Fn/ObjectFn.swift
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/// Allows the user to define a DoFn struct. Input and output streams along with parameterized values are dynamically
+/// injected at bundle processing time. This allows for more complicated DoFns to be written.
+public final class ObjectFn: SerializableFn {
+ var doFn: DoFn
+
+ public init(_ doFn: DoFn) {
+ self.doFn = doFn
+ }
+
+ public func process(context: SerializableFnBundleContext, inputs _: [AnyPCollectionStream], outputs _: [AnyPCollectionStream]) async throws -> (String, String) {
+ // TODO: Attach context to the doFn
+ try await doFn.process()
+ return (context.instruction, context.transform)
+ }
+}
diff --git a/Sources/ApacheBeam/Core/Fn/ParameterizedClosureFn.swift b/Sources/ApacheBeam/Core/Fn/ParameterizedClosureFn.swift
new file mode 100644
index 0000000..075d2db
--- /dev/null
+++ b/Sources/ApacheBeam/Core/Fn/ParameterizedClosureFn.swift
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+/// A SerializableFn that holds a reference to a function that takes a single input and produces a variable number of outputs
+public final class ParameterizedClosureFn<Param: Codable>: SerializableFn {
+ let param: Param
+ let processClosure: (SerializableFnBundleContext, [AnyPCollectionStream], [AnyPCollectionStream]) async throws -> Void
+
+ // TODO: Replace this with a parameter pack version once I figure out how to do that
+
+ public init<Of>(_ param: Param, _ fn: @Sendable @escaping (Param, PCollection<Of>.Stream) async throws -> Void) {
+ self.param = param
+ processClosure = { context, inputs, _ in
+ try await fn(JSONDecoder().decode(Param.self, from: context.payload), inputs[0].stream())
+ }
+ }
+
+ public init<Of, O0>(_ param: Param, _ fn: @Sendable @escaping (Param, PCollection<Of>.Stream, PCollection<O0>.Stream) async throws -> Void) {
+ self.param = param
+ processClosure = { context, inputs, outputs in
+ try await fn(JSONDecoder().decode(Param.self, from: context.payload), inputs[0].stream(), outputs[0].stream())
+ }
+ }
+
+ public init<Of, O0, O1>(_ param: Param, _ fn: @Sendable @escaping (Param, PCollection<Of>.Stream, PCollection<O0>.Stream, PCollection<O1>.Stream) async throws -> Void) {
+ self.param = param
+ processClosure = { context, inputs, outputs in
+ try await fn(JSONDecoder().decode(Param.self, from: context.payload), inputs[0].stream(), outputs[0].stream(), outputs[1].stream())
+ }
+ }
+
+ public init<Of, O0, O1, O2>(_ param: Param, _ fn: @Sendable @escaping (Param, PCollection<Of>.Stream, PCollection<O0>.Stream, PCollection<O1>.Stream, PCollection<O2>.Stream) async throws -> Void) {
+ self.param = param
+ processClosure = { context, inputs, outputs in
+ try await fn(JSONDecoder().decode(Param.self, from: context.payload), inputs[0].stream(), outputs[0].stream(), outputs[1].stream(), outputs[2].stream())
+ }
+ }
+
+ public init<Of, O0, O1, O2, O3>(_ param: Param, _ fn: @Sendable @escaping (Param, PCollection<Of>.Stream, PCollection<O0>.Stream, PCollection<O1>.Stream, PCollection<O2>.Stream, PCollection<O3>.Stream) async throws -> Void) {
+ self.param = param
+ processClosure = { context, inputs, outputs in
+ try await fn(JSONDecoder().decode(Param.self, from: context.payload), inputs[0].stream(), outputs[0].stream(), outputs[1].stream(), outputs[2].stream(), outputs[3].stream())
+ }
+ }
+
+ public func process(context: SerializableFnBundleContext, inputs: [AnyPCollectionStream], outputs: [AnyPCollectionStream]) async throws -> (String, String) {
+ try await processClosure(context, inputs, outputs)
+ outputs.finish()
+ return (context.instruction, context.transform)
+ }
+
+ public var payload: Data {
+ get throws {
+ try JSONEncoder().encode(param)
+ }
+ }
+}
diff --git a/Sources/ApacheBeam/Core/Fn/SerializableFn.swift b/Sources/ApacheBeam/Core/Fn/SerializableFn.swift
new file mode 100644
index 0000000..9b8cbc2
--- /dev/null
+++ b/Sources/ApacheBeam/Core/Fn/SerializableFn.swift
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+import Logging
+
+public struct SerializableFnBundleContext {
+ let instruction: String
+ let transform: String
+ let payload: Data
+ let log: Logging.Logger
+}
+
+/// SerialiableFn is a protocol for functions that should be parameterized for the pipeline. This is intended as a fairly low level class and users
+/// should interact with the apply() functions defined in the transform section or implement the DoFn protocol which is then wrapped
+public protocol SerializableFn {
+ var urn: String { get }
+ var payload: Data { get throws }
+ func process(context: SerializableFnBundleContext, inputs: [AnyPCollectionStream], outputs: [AnyPCollectionStream]) async throws -> (String, String)
+}
+
+/// Provide some defaults where our function doesn't have any payload
+public extension SerializableFn {
+ var urn: String { .beamUrn("dofn", type: "swift:transform") }
+ var payload: Data { Data() }
+}
diff --git a/Sources/ApacheBeam/Core/Fn/SerializableFnRegistry.swift b/Sources/ApacheBeam/Core/Fn/SerializableFnRegistry.swift
new file mode 100644
index 0000000..aa7b8ff
--- /dev/null
+++ b/Sources/ApacheBeam/Core/Fn/SerializableFnRegistry.swift
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+#if canImport(ApacheBeamPlugin)
+
+ @freestanding(expression)
+ public macro fnregister<T>(_ value: T) -> (T, String) = #externalMacro(module: "ApacheBeamPlugin", type: "FnRegistrationMacro")
+
+#endif
diff --git a/Sources/ApacheBeam/Core/KeyValue.swift b/Sources/ApacheBeam/Core/KeyValue.swift
new file mode 100644
index 0000000..ad34a3c
--- /dev/null
+++ b/Sources/ApacheBeam/Core/KeyValue.swift
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public protocol AnyKeyValue {
+ var anyKey: Any { get }
+ var anyValues: [Any] { get }
+ var anyValue: Any? { get }
+}
+
+/// A structure representing an array of values grouped by key
+public struct KV<Key, Value>: AnyKeyValue {
+ public let key: Key
+ public let values: [Value]
+ public var value: Value? { values.first }
+
+ public init(_ key: Key, _ value: Value) {
+ self.key = key
+ values = [value]
+ }
+
+ public init(_ key: Key, values: [Value]) {
+ self.key = key
+ self.values = values
+ }
+
+ public init(beam value: BeamValue) throws {
+ switch value {
+ case let .windowed(v, _, _, _):
+ self = try KV<Key, Value>(beam: v)
+ case let .kv(k, v):
+ key = k.baseValue! as! Key
+ values = v.baseValue as! [Value]
+ default:
+ throw ApacheBeamError.runtimeError("KV can only accept kv or windowed kv types")
+ }
+ }
+
+ public var anyKey: Any { key }
+ public var anyValues: [Any] { values.map { $0 as Any } }
+ public var anyValue: Any? { value }
+}
+
+extension KV: Beamable {
+ public static var coder: Coder {
+ .keyvalue(.of(type: Key.self)!, .of(type: Value.self)!)
+ }
+}
diff --git a/Sources/ApacheBeam/Core/PCollection/AnyPCollection.swift b/Sources/ApacheBeam/Core/PCollection/AnyPCollection.swift
new file mode 100644
index 0000000..4bf88c5
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PCollection/AnyPCollection.swift
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public struct AnyPCollection: PCollectionProtocol, PipelineMember {
+ let type: Any.Type
+ let ofType: Any.Type
+ let collection: Any
+
+ let parentClosure: (Any) -> PipelineTransform?
+ let applyClosure: (Any, PipelineTransform) -> PipelineTransform
+ let consumersClosure: (Any) -> [PipelineTransform]
+ let coderClosure: (Any) -> Coder
+ let streamClosure: (Any) -> AnyPCollectionStream
+ let rootsClosure: (Any) -> [PCollection<Never>]
+ let streamTypeClosure: (Any) -> StreamType
+
+ public init<C>(_ collection: C) where C: PCollectionProtocol {
+ if let anyCollection = collection as? AnyPCollection {
+ self = anyCollection
+ } else {
+ type = C.self
+ ofType = C.Of.self
+ self.collection = collection
+
+ applyClosure = { ($0 as! C).apply($1) }
+ consumersClosure = { ($0 as! C).consumers }
+ coderClosure = { ($0 as! C).coder }
+ streamClosure = { AnyPCollectionStream(($0 as! C).stream) }
+ parentClosure = { ($0 as! C).parent }
+ rootsClosure = { ($0 as! PipelineMember).roots }
+ streamTypeClosure = { ($0 as! C).streamType }
+ }
+ }
+
+ public var consumers: [PipelineTransform] {
+ consumersClosure(collection)
+ }
+
+ @discardableResult
+ public func apply(_ transform: PipelineTransform) -> PipelineTransform {
+ applyClosure(collection, transform)
+ }
+
+ public var parent: PipelineTransform? {
+ parentClosure(collection)
+ }
+
+ public var coder: Coder {
+ coderClosure(collection)
+ }
+
+ public var stream: PCollectionStream<Never> {
+ fatalError("Do not use `stream` on AnyPCollection. Use `anyStream` instead.")
+ }
+
+ public var streamType: StreamType {
+ streamTypeClosure(collection)
+ }
+
+ public var anyStream: AnyPCollectionStream {
+ streamClosure(collection)
+ }
+
+ var roots: [PCollection<Never>] {
+ rootsClosure(collection)
+ }
+
+ func of<Of>(_: Of.Type) -> PCollection<Of>? {
+ if let ret = collection as? PCollection<Of> {
+ return ret
+ } else {
+ return nil
+ }
+ }
+}
+
+extension AnyPCollection: Hashable {
+ public func hash(into hasher: inout Hasher) {
+ hasher.combine(ObjectIdentifier(collection as AnyObject))
+ }
+
+ public static func == (lhs: AnyPCollection, rhs: AnyPCollection) -> Bool {
+ ObjectIdentifier(lhs.collection as AnyObject) == ObjectIdentifier(rhs.collection as AnyObject)
+ }
+}
diff --git a/Sources/ApacheBeam/Core/PCollection/AnyPCollectionStream.swift b/Sources/ApacheBeam/Core/PCollection/AnyPCollectionStream.swift
new file mode 100644
index 0000000..893e977
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PCollection/AnyPCollectionStream.swift
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+public struct AnyPCollectionStream: AsyncSequence {
+ public typealias Element = Iterator.Element
+ public typealias AsyncIterator = Iterator
+
+ public struct Iterator: AsyncIteratorProtocol {
+ public typealias Element = (Any, Date, Window)
+
+ let nextClosure: () async throws -> Element?
+ public mutating func next() async throws -> Element? {
+ try await nextClosure()
+ }
+ }
+
+ let value: Any
+ let nextGenerator: (Any) -> (() async throws -> Iterator.Element?)
+ let emitClosure: (Any, Any) throws -> Void
+ let finishClosure: (Any) -> Void
+
+ public func makeAsyncIterator() -> Iterator {
+ Iterator(nextClosure: nextGenerator(value))
+ }
+
+ public init(_ value: AnyPCollectionStream) {
+ self = value
+ }
+
+ public init<Of>(_ value: PCollectionStream<Of>) {
+ self.value = value
+
+ emitClosure = {
+ let stream = ($0 as! PCollectionStream<Of>)
+ if let beamValue = $1 as? BeamValue {
+ try stream.emit(beamValue)
+ } else if let element = $1 as? Element {
+ stream.emit((element.0 as! Of, element.1, element.2))
+ } else {
+ throw ApacheBeamError.runtimeError("Unable to send \($1) to \(stream)")
+ }
+ }
+
+ finishClosure = {
+ ($0 as! PCollectionStream<Of>).finish()
+ }
+
+ nextGenerator = {
+ var iterator = ($0 as! PCollectionStream<Of>).makeAsyncIterator()
+ return {
+ if let element = await iterator.next() {
+ return (element.0 as Any, element.1, element.2)
+ } else {
+ return nil
+ }
+ }
+ }
+ }
+
+ public func stream<Out>() -> PCollectionStream<Out> {
+ value as! PCollectionStream<Out>
+ }
+
+ public func emit(value element: Any) throws {
+ try emitClosure(value, element)
+ }
+
+ public func finish() {
+ finishClosure(value)
+ }
+}
+
+/// Convenience function of an array of AnyPCollectionStream elements to finish processing.
+public extension Array where Array.Element == AnyPCollectionStream {
+ func finish() {
+ for stream in self {
+ stream.finish()
+ }
+ }
+}
diff --git a/Sources/ApacheBeam/Core/PCollection/PCollection.swift b/Sources/ApacheBeam/Core/PCollection/PCollection.swift
new file mode 100644
index 0000000..d5c0dcc
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PCollection/PCollection.swift
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public enum StreamType {
+ case bounded
+ case unbounded
+ case unspecified
+}
+
+public enum WindowingStrategy {
+ case unspecified
+}
+
+public protocol PCollectionProtocol {
+ associatedtype Of
+
+ typealias Stream = PCollectionStream<Of>
+
+ var parent: PipelineTransform? { get }
+ var consumers: [PipelineTransform] { get }
+ var coder: Coder { get }
+ var stream: Stream { get }
+ var streamType: StreamType { get }
+
+ @discardableResult
+ func apply(_ transform: PipelineTransform) -> PipelineTransform
+}
+
+public final class PCollection<Of>: PCollectionProtocol {
+ public let coder: Coder
+ public let streamType: StreamType
+ public var consumers: [PipelineTransform]
+ public private(set) var parent: PipelineTransform?
+
+ private let staticStream: PCollectionStream<Of>?
+
+ public init(coder: Coder = .of(type: Of.self)!,
+ type: StreamType = .unspecified,
+ parent: PipelineTransform? = nil,
+ consumers: [PipelineTransform] = [],
+ stream: PCollectionStream<Of>? = nil)
+ {
+ self.coder = coder
+ self.consumers = consumers
+ self.parent = parent
+ staticStream = stream
+ streamType = type
+ }
+
+ public var stream: PCollectionStream<Of> {
+ staticStream ?? PCollectionStream<Of>()
+ }
+
+ @discardableResult
+ public func apply(_ transform: PipelineTransform) -> PipelineTransform {
+ consumers.append(transform)
+ return transform
+ }
+
+ func parent(_ transform: PipelineTransform) {
+ parent = transform
+ }
+}
+
+public typealias PipelineRoot = PCollection<Never>
+
+extension PCollection: PipelineMember {
+ var roots: [PipelineRoot] {
+ if let p = parent {
+ return p.roots
+ } else if let p = self as? PipelineRoot {
+ return [p]
+ } else {
+ return []
+ }
+ }
+}
+
+public extension PCollection {
+ static func empty() -> PCollection<Of> {
+ PCollection<Of>()
+ }
+}
diff --git a/Sources/ApacheBeam/Core/PCollection/PCollectionGroup.swift b/Sources/ApacheBeam/Core/PCollection/PCollectionGroup.swift
new file mode 100644
index 0000000..e4b43cf
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PCollection/PCollectionGroup.swift
@@ -0,0 +1,16 @@
+//
+// File.swift
+//
+//
+// Created by Byron Ellis on 9/21/23.
+//
+
+import Foundation
+
+@dynamicMemberLookup
+public protocol PCollectionGroup {
+ subscript(dynamicMember member:String) -> AnyPCollection? { get }
+ subscript<Of>(of:Of.Type) -> PCollection<Of>? { get }
+ subscript(named: String) -> AnyPCollection? { get throws }
+ subscript<Of>(named: String,of:Of.Type) -> PCollection<Of>? { get }
+}
diff --git a/Sources/ApacheBeam/Core/PCollection/PCollectionStream.swift b/Sources/ApacheBeam/Core/PCollection/PCollectionStream.swift
new file mode 100644
index 0000000..76b63ac
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PCollection/PCollectionStream.swift
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+/// The worker side realization of a PCollection that supports reading and writing
+public final class PCollectionStream<Of>: AsyncSequence {
+ public typealias Element = (Of, Date, Window)
+
+ private let stream: AsyncStream<Element>
+ private let emitter: AsyncStream<Element>.Continuation
+
+ public init() {
+ (stream, emitter) = AsyncStream.makeStream(of: Element.self)
+ }
+
+ public func makeAsyncIterator() -> AsyncStream<Element>.Iterator {
+ stream.makeAsyncIterator()
+ }
+
+ public func finish() {
+ emitter.finish()
+ }
+
+ public func emit(_ value: Element) {
+ emitter.yield(value)
+ }
+
+ public func emit(_ value: Of, timestamp: Date, window: Window) {
+ emit((value, timestamp, window))
+ }
+
+ // Implementing key-value pair conversion is a little more complicated because we need
+ // to convert to a KV<K,V> from what is essentially a KV<Any,Any> which requires us to
+ // cast the key and the value first and then construct the KV from that. There might be
+ // a more clever way of doing this, but I don't know what it is.
+
+ func emit(key: some Any, value: [some Any], timestamp: Date, window: Window) {
+ emit(KV(key, values: value) as! Of, timestamp: timestamp, window: window)
+ }
+
+ func emit(key: some Any, value: BeamValue, timestamp: Date, window: Window) throws {
+ // We overload the key value type as both (K,[V]) and (K,V). It may be worth considering
+ // having an explicit Pair type in addition to KV to simplify this decoding a little bit.
+ //
+ // On the other hand, the code is already written and pretty straightforward and there
+ // won't be much in the way of new scalar values.
+ if case let .array(array) = value {
+ switch array.first {
+ case .boolean: emit(key: key, value: array.map { $0.baseValue! as! Bool }, timestamp: timestamp, window: window)
+ case .bytes: emit(key: key, value: array.map { $0.baseValue! as! Data }, timestamp: timestamp, window: window)
+ case .double: emit(key: key, value: array.map { $0.baseValue! as! Double }, timestamp: timestamp, window: window)
+ case .integer: emit(key: key, value: array.map { $0.baseValue! as! Int }, timestamp: timestamp, window: window)
+ case .string: emit(key: key, value: array.map { $0.baseValue! as! String }, timestamp: timestamp, window: window)
+ default:
+ throw ApacheBeamError.runtimeError("Can't use \(String(describing: array.first)) as a value in a key value pair")
+ }
+ } else {
+ switch value {
+ case let .boolean(v): emit(key: key, value: [v!], timestamp: timestamp, window: window)
+ case let .bytes(v): emit(key: key, value: [v!], timestamp: timestamp, window: window)
+ case let .double(v): emit(key: key, value: [v!], timestamp: timestamp, window: window)
+ case let .integer(v): emit(key: key, value: [v!], timestamp: timestamp, window: window)
+ case let .string(v): emit(key: key, value: [v!], timestamp: timestamp, window: window)
+ default:
+ throw ApacheBeamError.runtimeError("Can't use \(value) as a value in a key value pair")
+ }
+ }
+ }
+
+ // Unwrap all of the actual value types (not windows or windowed elements)
+ func emit(_ value: BeamValue, timestamp: Date, window: Window) throws {
+ if case let .kv(key, value) = value {
+ // Unwrap the key first
+ switch key {
+ case let .boolean(v): try emit(key: v!, value: value, timestamp: timestamp, window: window)
+ case let .bytes(v): try emit(key: v!, value: value, timestamp: timestamp, window: window)
+ case let .double(v): try emit(key: v!, value: value, timestamp: timestamp, window: window)
+ case let .integer(v): try emit(key: v!, value: value, timestamp: timestamp, window: window)
+ case let .string(v): try emit(key: v!, value: value, timestamp: timestamp, window: window)
+ default:
+ throw ApacheBeamError.runtimeError("Can't use \(value) as a value in a key value pair")
+ }
+ } else {
+ emit(value.baseValue as! Of, timestamp: timestamp, window: window)
+ }
+ }
+
+ /// Mostly intended as a convenience function for bundle processing this emit unwraps a windowed value
+ /// for further conversion in (non-public) versions of the function.
+ public func emit(_ value: BeamValue) throws {
+ switch value {
+ case let .windowed(value, timestamp, _, window):
+ try emit(value, timestamp: timestamp, window: window.baseValue as! Window)
+ default:
+ throw ApacheBeamError.runtimeError("Only windowed values can be sent directly to a PCollectionStream, not \(value)")
+ }
+ }
+}
diff --git a/Sources/ApacheBeam/Core/PTransform/AnyPTransform.swift b/Sources/ApacheBeam/Core/PTransform/AnyPTransform.swift
new file mode 100644
index 0000000..3989f64
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PTransform/AnyPTransform.swift
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public struct AnyPTransform: _PrimitivePTransform {
+ let type: Any.Type
+ var transform: Any
+
+ let expandClosure: (Any) -> AnyPTransform
+ let expansionType: Any.Type
+
+ public init<T>(_ transform: T) where T: PTransform {
+ if let anyTransform = transform as? AnyPTransform {
+ self = anyTransform
+ } else {
+ type = T.self
+ expansionType = T.Expansion.self
+ self.transform = transform
+ expandClosure = { AnyPTransform(($0 as! T).expand) }
+ }
+ }
+}
+
+public func mapAnyPTransform<T, P>(_ anyPTransform: AnyPTransform, transform: (P) -> T) -> T? {
+ guard let ptransform = anyPTransform.transform as? P else { return nil }
+ return transform(ptransform)
+}
+
+extension AnyPTransform: ParentPTransform {
+ public var children: [AnyPTransform] {
+ (transform as? ParentPTransform)?.children ?? []
+ }
+}
diff --git a/Sources/ApacheBeam/Core/PTransform/Group.swift b/Sources/ApacheBeam/Core/PTransform/Group.swift
new file mode 100644
index 0000000..861901a
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PTransform/Group.swift
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+public struct Group<Content> {
+ let content: Content
+ public init(@PTransformBuilder content: () -> Content) {
+ self.content = content()
+ }
+}
+
+extension Group: _PrimitivePTransform, PTransform where Content: PTransform {
+ public func _visitChildren(_ visitor: some PTransformVisitor) {
+ visitor.visit(content)
+ }
+}
+
+extension Group: ParentPTransform where Content: PTransform {
+ public var children: [AnyPTransform] { (content as? ParentPTransform)?.children ?? [AnyPTransform(content)] }
+}
+
+extension Group: GroupPTransform where Content: PTransform {}
diff --git a/Sources/ApacheBeam/Core/PTransform/PTransform.swift b/Sources/ApacheBeam/Core/PTransform/PTransform.swift
new file mode 100644
index 0000000..35303b2
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PTransform/PTransform.swift
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/// Represents a composite transform
+public protocol PTransform {
+ associatedtype Expansion: PTransform
+
+ @PTransformBuilder
+ var expand: Expansion { get }
+}
+
+public extension Never {
+ var expand: Never {
+ fatalError()
+ }
+}
+
+extension Never: PTransform {}
+
+/// Represents PTransforms that can't be expanded further. When constructing the pipeline the expansion
+/// happens until we hit this point
+public protocol _PrimitivePTransform: PTransform where Expansion == Never {}
+public extension _PrimitivePTransform {
+ var expand: Never {
+ neverExpand(String(reflecting: Self.self))
+ }
+}
+
+public protocol ParentPTransform {
+ var children: [AnyPTransform] { get }
+}
+
+public protocol GroupPTransform: ParentPTransform {}
+
+public func neverExpand(_ type: String) -> Never {
+ fatalError("\(type) is a primitive PTransform and cannot be expanded.")
+}
+
+public extension ParentPTransform {
+ subscript(pcollection: String) -> AnyPCollection? {
+ for child in children {
+ if let named = child.transform as? NamedCollectionPTransform {
+ if named.name == pcollection {
+ return named.collection
+ }
+ }
+ }
+ return nil
+ }
+}
diff --git a/Sources/ApacheBeam/Core/PTransform/PTransformBuilder.swift b/Sources/ApacheBeam/Core/PTransform/PTransformBuilder.swift
new file mode 100644
index 0000000..3fa3e28
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PTransform/PTransformBuilder.swift
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public struct EmptyPTransform: _PrimitivePTransform {
+ public init() {}
+}
+
+public struct _ConditionalPTransform<TruePTransform, FalsePTransform>: _PrimitivePTransform
+ where TruePTransform: PTransform, FalsePTransform: PTransform
+{
+ enum Storage {
+ case trueTransform(TruePTransform)
+ case falseTransform(FalsePTransform)
+ }
+
+ let storage: Storage
+}
+
+public extension _ConditionalPTransform {
+ var children: [AnyPTransform] {
+ switch storage {
+ case let .trueTransform(transform):
+ return [AnyPTransform(transform)]
+ case let .falseTransform(transform):
+ return [AnyPTransform(transform)]
+ }
+ }
+}
+
+@resultBuilder
+public enum PTransformBuilder {
+ public static func buildBlock() -> EmptyPTransform {
+ EmptyPTransform()
+ }
+
+ public static func buildBlock<Transform>(_ transform: Transform) -> Transform where Transform: PTransform {
+ transform
+ }
+
+ public static func buildEither<TrueT, FalseT>(first: TrueT) -> _ConditionalPTransform<TrueT, FalseT> where TrueT: PTransform, FalseT: PTransform {
+ .init(storage: .trueTransform(first))
+ }
+
+ public static func buildEither<TrueT, FalseT>(second: FalseT) -> _ConditionalPTransform<TrueT, FalseT> where TrueT: PTransform, FalseT: PTransform {
+ .init(storage: .falseTransform(second))
+ }
+}
+
+public extension PTransformBuilder {
+ static func buildBlock<T0, T1>(_ t0: T0, _ t1: T1) -> TuplePTransform<(T0, T1)> where T0: PTransform, T1: PTransform {
+ TuplePTransform<(T0, T1)>(t0, t1)
+ }
+}
diff --git a/Sources/ApacheBeam/Core/PTransform/PTransformVisitor.swift b/Sources/ApacheBeam/Core/PTransform/PTransformVisitor.swift
new file mode 100644
index 0000000..8860d09
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PTransform/PTransformVisitor.swift
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+public protocol PTransformVisitor {
+ func visit<T: PTransform>(_ transform: T)
+}
+
+public extension PTransform {
+ func _visitChildren(_ visitor: some PTransformVisitor) {
+ visitor.visit(expand)
+ }
+}
+
+public typealias PTransformVisitorFn<V: PTransformVisitor> = (V) -> Void
+
+protocol PTransformReducer {
+ associatedtype Result
+
+ static func reduce<T: PTransform>(into partialResult: inout Result, nextTransform: T)
+ static func reduce<T: PTransform>(partialResult: Result, nextTransform: T) -> Result
+}
+
+extension PTransformReducer {
+ static func reduce(into partialResult: inout Result, nextTransform: some PTransform) {
+ partialResult = reduce(partialResult: partialResult, nextTransform: nextTransform)
+ }
+
+ static func reduce(partialResult: Result, nextTransform: some PTransform) -> Result {
+ var result = partialResult
+ Self.reduce(into: &result, nextTransform: nextTransform)
+ return result
+ }
+}
+
+final class ReducerVisitor<R: PTransformReducer>: PTransformVisitor {
+ var result: R.Result
+ init(initialResult: R.Result) {
+ result = initialResult
+ }
+
+ func visit(_ transform: some PTransform) {
+ R.reduce(into: &result, nextTransform: transform)
+ }
+}
+
+extension PTransformReducer {
+ typealias Visitor = ReducerVisitor<Self>
+}
diff --git a/Sources/ApacheBeam/Core/PTransform/PValue.swift b/Sources/ApacheBeam/Core/PTransform/PValue.swift
new file mode 100644
index 0000000..a046002
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PTransform/PValue.swift
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+@propertyWrapper
+public struct PValue<Value> : DynamicProperty {
+
+ enum UpdateStrategy {
+ case pipeline
+ case first(Any.Type)
+ case named(String,Any.Type)
+ }
+
+ enum Storage {
+ case uninitialized
+ case constant(Value)
+ }
+
+ private let updater:UpdateStrategy
+ private var storage:Storage = .uninitialized
+
+ public var wrappedValue: Value {
+ get {
+ switch storage {
+ case .constant(let v):
+ return v
+ case .uninitialized:
+ fatalError("Storage component of a PValue is uninitialized.")
+ }
+ }
+ nonmutating set {
+ }
+ }
+
+ public var projectedValue: PValue<Value> { self }
+
+ public init() where Value == PipelineRoot {
+ updater = .pipeline
+ }
+
+ public init<Of>() where Value == PCollection<Of> {
+ updater = .first(Of.self)
+ }
+
+ public init<Of>(named: String) where Value == PCollection<Of> {
+ updater = .named(named,Of.self)
+ }
+
+ public mutating func update(from root: PipelineRoot) throws where Value == PipelineRoot {
+ if case .pipeline = updater {
+ storage = .constant(root)
+ }
+ }
+
+ public mutating func update<Of>(from collection: PCollection<Of>) throws where Value == PipelineRoot {
+ if case .pipeline = updater {
+ if let root = collection.roots.first {
+ storage = .constant(root)
+ } else {
+ throw ApacheBeamError.runtimeError("Unable to retrieve pipeline from PCollection")
+ }
+ }
+ }
+
+ public mutating func update<Of>(from collection: PCollection<Of>) throws where Value == PCollection<Of> {
+ if case let .first(type) = updater {
+ if type == Of.self {
+ storage = .constant(collection)
+ }
+ }
+ }
+
+}
diff --git a/Sources/ApacheBeam/Core/PTransform/Tag.swift b/Sources/ApacheBeam/Core/PTransform/Tag.swift
new file mode 100644
index 0000000..640c350
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PTransform/Tag.swift
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public struct NamedCollectionPTransform: _PrimitivePTransform {
+ let name: String
+ let collection: AnyPCollection
+}
+
+/// Captures a PCollection and gives it a name so it can be referenced
+public struct Tag<Of>: PTransform {
+ let name: String
+ let fn: () -> PCollection<Of>
+ public init(_ name: String, _ fn: @escaping () -> PCollection<Of>) {
+ self.name = name
+ self.fn = fn
+ }
+
+ public var expand: NamedCollectionPTransform {
+ NamedCollectionPTransform(name: name, collection: AnyPCollection(fn()))
+ }
+}
+
+/// Convenience mapping for more readable composites
+public typealias Output = Tag
diff --git a/Sources/ApacheBeam/Core/PTransform/Transform.swift b/Sources/ApacheBeam/Core/PTransform/Transform.swift
new file mode 100644
index 0000000..062e82f
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PTransform/Transform.swift
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/// Groups transforms together. Does not expose any of the pcollections as accessible outputs
+public struct Transform<Subtransform> {
+ let subtransform: Subtransform
+ public init(@PTransformBuilder subtransform: () -> Subtransform) {
+ self.subtransform = subtransform()
+ }
+}
diff --git a/Sources/ApacheBeam/Core/PTransform/TuplePTransform.swift b/Sources/ApacheBeam/Core/PTransform/TuplePTransform.swift
new file mode 100644
index 0000000..6d9b8b8
--- /dev/null
+++ b/Sources/ApacheBeam/Core/PTransform/TuplePTransform.swift
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public struct TuplePTransform<T>: _PrimitivePTransform {
+ public let value: T
+ let _children: [AnyPTransform]
+
+ public init(value: T) {
+ self.value = value
+ _children = []
+ }
+
+ public init(value: T, children: [AnyPTransform]) {
+ self.value = value
+ _children = children
+ }
+
+ init<T0: PTransform, T1: PTransform>(_ t0: T0, _ t1: T1) where T == (T0, T1) {
+ value = (t0, t1)
+ _children = [AnyPTransform(t0), AnyPTransform(t1)]
+ }
+}
diff --git a/Sources/ApacheBeam/Core/Pipeline/Pipeline.swift b/Sources/ApacheBeam/Core/Pipeline/Pipeline.swift
new file mode 100644
index 0000000..e5ef5ea
--- /dev/null
+++ b/Sources/ApacheBeam/Core/Pipeline/Pipeline.swift
@@ -0,0 +1,290 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import GRPC
+import Logging
+
+protocol PipelineMember {
+ var roots: [PCollection<Never>] { get }
+}
+
+extension StreamType {
+ var isBounded: Org_Apache_Beam_Model_Pipeline_V1_IsBounded.Enum {
+ get throws {
+ switch self {
+ case .bounded:
+ .bounded
+ case .unbounded:
+ .unbounded
+ case .unspecified:
+ throw ApacheBeamError.runtimeError("isBounded must be specified at pipeline construction time.")
+ }
+ }
+ }
+}
+
+public final class Pipeline {
+ let content: (inout PCollection<Never>) -> Void
+ let log: Logging.Logger
+
+ public init(log: Logging.Logger = .init(label: "Pipeline"), _ content: @escaping (inout PCollection<Never>) -> Void) {
+ self.log = log
+ self.content = content
+ }
+
+ public init(log: Logging.Logger = .init(label: "Pipeline"), @PTransformBuilder content: () -> some PTransform) {
+ self.log = log
+
+ let transform = content()
+ self.content = { root in
+ _ = root.apply(transform)
+ }
+ }
+
+ public func run(_ runner: PipelineRunner) async throws {
+ try await runner.run(context)
+ }
+
+ /// For managing the pipeline items to visit
+ enum Visit {
+ case transform([PipelineComponent], PipelineTransform)
+ case collection(AnyPCollection)
+ }
+
+ var context: PipelineContext {
+ get throws {
+ // Grab the pipeline content using an new root
+ var root = PCollection<Never>(coder: .unknown(.coderUrn("never")), type: .bounded)
+ _ = content(&root)
+
+ // These get passed to the pipeline context
+ var collections: [String: AnyPCollection] = [:]
+ var fns: [String: SerializableFn] = [:]
+ var coders: [String: Coder] = [:]
+ var counter: Int = 1
+
+ // These caches are just used internally
+ var collectionCache: [AnyPCollection: PipelineComponent] = [:]
+ var coderCache: [Coder: PipelineComponent] = [:]
+ var rootIds: [String] = []
+
+ var defaultEnvironment: PipelineComponent = .none
+
+ // TODO: Support for composite PTransforms
+ let pipeline: PipelineProto = try .with { proto in
+
+ func uniqueName(_ prefix: String = "id") -> String {
+ let output = "\(prefix)\(counter)"
+ counter = counter + 1
+ return output
+ }
+
+ /// We need to define this inside the with to prevent concurrent access errors.
+ func coder(from: Coder) -> PipelineComponent {
+ if let cached = coderCache[from] {
+ return cached
+ }
+ let componentCoders: [String] = switch from {
+ case let .keyvalue(keyCoder, valueCoder):
+ [coder(from: keyCoder).name, coder(from: valueCoder).name]
+ case let .iterable(valueCoder):
+ [coder(from: valueCoder).name]
+ case let .lengthprefix(valueCoder):
+ [coder(from: valueCoder).name]
+ case let .windowedvalue(valueCoder, windowCoder):
+ [coder(from: valueCoder).name, coder(from: windowCoder).name]
+ default:
+ []
+ }
+ let baseCoder = proto.coder { _ in
+ .with {
+ $0.spec = .with {
+ $0.urn = from.urn
+ if case let .custom(data) = from {
+ $0.payload = data
+ }
+ }
+ $0.componentCoderIds = componentCoders
+ }
+ }
+ coderCache[from] = baseCoder
+ coders[baseCoder.name] = from
+ return baseCoder
+ }
+
+ /// Define the default environment for this pipeline
+ defaultEnvironment = try proto.environment(from: .init(.docker("swift:image"),
+ capabilities: Coder.capabilities,
+ dependencies: []))
+
+ /// Define the default strategy
+ let globalWindow = coder(from: .globalwindow)
+ let defaultStrategy = proto.windowingStrategy { _ in
+ .with {
+ $0.windowCoderID = globalWindow.name
+ $0.windowFn = .with {
+ $0.urn = .beamUrn("global_windows", type: "window_fn")
+ }
+ $0.mergeStatus = .nonMerging
+ $0.trigger = .with {
+ $0.default = .init()
+ }
+ $0.accumulationMode = .discarding
+ $0.outputTime = .endOfWindow
+ $0.closingBehavior = .emitIfNonempty
+ $0.onTimeBehavior = .fireIfNonempty
+ $0.environmentID = defaultEnvironment.name
+ }
+ }
+
+ // As above we define this within the "with" to prevent concurrent access errors.
+ func collection(from collection: AnyPCollection) throws -> PipelineComponent {
+ if let cached = collectionCache[collection] {
+ return cached
+ }
+ let coder = coder(from: collection.coder)
+ let output = try proto.collection { _ in
+ try .with {
+ $0.uniqueName = uniqueName("c")
+ $0.coderID = coder.name
+ $0.windowingStrategyID = defaultStrategy.name
+ $0.isBounded = try collection.streamType.isBounded
+ }
+ }
+ collectionCache[collection] = output
+ collections[output.name] = collection
+ return output
+ }
+
+ func transform(name: String = "", _ fn: @escaping (String, String) throws -> PTransformProto) throws -> PipelineComponent {
+ try proto.transform { ref in
+ try fn(ref, name.count > 0 ? name + uniqueName(".t") : uniqueName("t"))
+ }
+ }
+
+ var toVisit: [Visit] = root.consumers.map { .transform([], $0) }
+ var visited = Set<AnyPCollection>() // Cycle detection, etc
+
+ while toVisit.count > 0 {
+ let item = toVisit.removeFirst()
+ if case let .transform(parents, pipelineTransform) = item {
+ let inputs = parents.enumerated().map { ("\($0)", "\($1.name)") }.dict()
+ switch pipelineTransform {
+ case let .pardo(_, n, fn, o):
+ let outputs = try o.enumerated().map {
+ try ("\($0)", collection(from: $1).name)
+ }.dict()
+ let p = try transform(name: n) { _, name in
+ try .with {
+ $0.uniqueName = name
+ $0.inputs = inputs
+ $0.outputs = outputs
+ $0.spec = try .with {
+ $0.urn = .transformUrn("pardo")
+ $0.payload = try Org_Apache_Beam_Model_Pipeline_V1_ParDoPayload.with {
+ $0.doFn = try .with {
+ $0.urn = fn.urn
+ $0.payload = try fn.payload
+ }
+ }.serializedData()
+ }
+ $0.environmentID = defaultEnvironment.name
+ }
+ }
+ rootIds.append(p.name) // TODO: Composite transform handling
+ fns[p.transform!.uniqueName] = fn
+ toVisit.append(contentsOf: o.map { .collection($0) })
+ case let .impulse(_, o):
+ let outputs = try [o].enumerated().map {
+ try ("\($0)", collection(from: $1).name)
+ }.dict()
+ let p = try transform { _, name in
+ .with {
+ $0.uniqueName = name
+ $0.outputs = outputs
+ $0.spec = .with {
+ $0.urn = .transformUrn("impulse")
+ }
+ }
+ }
+ rootIds.append(p.name)
+ toVisit.append(.collection(o))
+ case .flatten:
+ throw ApacheBeamError.runtimeError("flatten not implemented yet")
+ case .external:
+ throw ApacheBeamError.runtimeError("External Transforms not implemented yet")
+ case let .groupByKey(_, o):
+ let outputs = try [o].enumerated().map {
+ try ("\($0)", collection(from: $1).name)
+ }.dict()
+ let p = try transform { _, name in
+ .with {
+ $0.uniqueName = name
+ $0.inputs = inputs
+ $0.outputs = outputs
+ $0.spec = .with {
+ $0.urn = .transformUrn("group_by_key")
+ }
+ }
+ }
+ rootIds.append(p.name)
+ toVisit.append(.collection(o))
+ case let .custom(_, urn, payload, env, o):
+ let outputs = try o.enumerated().map {
+ try ("\($0)", collection(from: $1).name)
+ }.dict()
+ let environment = if let e = env {
+ try proto.environment(from: e)
+ } else {
+ defaultEnvironment
+ }
+ let p = try transform { _, name in
+ .with {
+ $0.uniqueName = name
+ $0.inputs = inputs
+ $0.outputs = outputs
+ $0.spec = .with {
+ $0.urn = urn
+ $0.payload = payload
+ }
+ $0.environmentID = environment.name
+ }
+ }
+ rootIds.append(p.name)
+ toVisit.append(contentsOf: o.map { .collection($0) })
+ case .composite:
+ throw ApacheBeamError.runtimeError("Composite transforms are not yet implemented")
+ }
+ } else if case let .collection(anyPCollection) = item {
+ if visited.contains(anyPCollection) {
+ throw ApacheBeamError.runtimeError("Pipeline definition contains a cycle.")
+ }
+ visited.insert(anyPCollection)
+ // TODO: Remove this to see if we can recreate the error I was seeing earlier for robertwb
+ if anyPCollection.consumers.count > 0 {
+ let me = try collection(from: anyPCollection)
+ toVisit.append(contentsOf: anyPCollection.consumers.map { .transform([me], $0) })
+ }
+ }
+ }
+ proto.rootTransformIds = rootIds
+ }
+ return PipelineContext(pipeline, defaultEnvironment.name, collections, fns)
+ }
+ }
+}
diff --git a/Sources/ApacheBeam/Core/Pipeline/PipelineContext.swift b/Sources/ApacheBeam/Core/Pipeline/PipelineContext.swift
new file mode 100644
index 0000000..7190ad8
--- /dev/null
+++ b/Sources/ApacheBeam/Core/Pipeline/PipelineContext.swift
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public protocol StatusProtocol {}
+
+public final class PipelineContext {
+ var proto: PipelineProto
+ let defaultEnvironmentId: String
+ let collections: [String: AnyPCollection]
+ let pardoFns: [String: SerializableFn]
+
+ init(_ proto: PipelineProto, _ defaultEnvironmentId: String, _ collections: [String: AnyPCollection], _ fns: [String: SerializableFn]) {
+ self.proto = proto
+ self.defaultEnvironmentId = defaultEnvironmentId
+ self.collections = collections
+ pardoFns = fns
+ }
+}
diff --git a/Sources/ApacheBeam/Core/Pipeline/PipelineRunner.swift b/Sources/ApacheBeam/Core/Pipeline/PipelineRunner.swift
new file mode 100644
index 0000000..08a0801
--- /dev/null
+++ b/Sources/ApacheBeam/Core/Pipeline/PipelineRunner.swift
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public protocol PipelineRunner {
+ func run(_ context: PipelineContext) async throws
+}
diff --git a/Sources/ApacheBeam/Core/Pipeline/PipelineTransform.swift b/Sources/ApacheBeam/Core/Pipeline/PipelineTransform.swift
new file mode 100644
index 0000000..c185c6e
--- /dev/null
+++ b/Sources/ApacheBeam/Core/Pipeline/PipelineTransform.swift
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+/// Enum for pipeline representable transforms as opposed to composite transforms
+/// which are a user-side construct represented by PTransform
+public enum PipelineTransform {
+ case pardo(AnyPCollection, String, SerializableFn, [AnyPCollection])
+ case impulse(AnyPCollection, AnyPCollection)
+ case flatten([AnyPCollection], AnyPCollection)
+ case groupByKey(AnyPCollection, AnyPCollection)
+ case custom(AnyPCollection, String, Data, Environment?, [AnyPCollection])
+ case composite(AnyPCollection, AnyPTransform)
+ case external(AnyPCollection, String, FieldValue, [AnyPCollection])
+}
+
+extension PipelineTransform: PipelineMember {
+ var roots: [PCollection<Never>] {
+ switch self {
+ case let .pardo(p, _, _, _):
+ p.roots
+ case let .impulse(p, _):
+ p.roots
+ case let .flatten(p, _):
+ p.flatMap(\.roots)
+ case let .groupByKey(p, _):
+ p.roots
+ case let .custom(p, _, _, _, _):
+ p.roots
+ case let .composite(p, _):
+ p.roots
+ case let .external(p, _, _, _):
+ p.roots
+ }
+ }
+}
diff --git a/Sources/ApacheBeam/Core/Windowing.swift b/Sources/ApacheBeam/Core/Windowing.swift
new file mode 100644
index 0000000..7f9cff5
--- /dev/null
+++ b/Sources/ApacheBeam/Core/Windowing.swift
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Foundation
+
+public enum WindowFn {
+ case global
+ case fixed(Duration)
+ case sliding(Duration, Duration)
+ case session(Duration)
+}
+
+public enum Window {
+ case global
+ case bounded(Date)
+ case interval(Date, Date)
+
+ var maxTimestamp: Date {
+ switch self {
+ case .global: Date.distantFuture
+ case let .bounded(end): end
+ case let .interval(_, end): end
+ }
+ }
+}
+
+public enum Timing: String {
+ case early = "EARLY"
+ case onTime = "ON_TIME"
+ case late = "LATE"
+ case unknown = "UNKNOWN"
+}
+
+public protocol PaneInfo {
+ var timing: Timing { get }
+ var index: UInt64 { get }
+ var onTimeIndex: UInt64 { get }
+ var first: Bool { get }
+ var last: Bool { get }
+}
diff --git a/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/fn_execution/v1/beam_fn_api.grpc.swift b/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/fn_execution/v1/beam_fn_api.grpc.swift
new file mode 100644
index 0000000..c636677
--- /dev/null
+++ b/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/fn_execution/v1/beam_fn_api.grpc.swift
@@ -0,0 +1,2061 @@
+//
+// DO NOT EDIT.
+// swift-format-ignore-file
+//
+// Generated by the protocol buffer compiler.
+// Source: org/apache/beam/model/fn_execution/v1/beam_fn_api.proto
+//
+
+//
+// Copyright 2018, gRPC Authors All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+//
+import GRPC
+import NIO
+import NIOConcurrencyHelpers
+import SwiftProtobuf
+
+
+/// An API that describes the work that a SDK harness is meant to do.
+/// Stable
+///
+/// Usage: instantiate `Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientInterceptorFactoryProtocol? { get }
+
+ func control(
+ callOptions: CallOptions?,
+ handler: @escaping (Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest) -> Void
+ ) -> BidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse, Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest>
+
+ func getProcessBundleDescriptor(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest, Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.fn_execution.v1.BeamFnControl"
+ }
+
+ /// Instructions sent by the runner to the SDK requesting different types
+ /// of work.
+ ///
+ /// Callers should use the `send` method on the returned object to send messages
+ /// to the server. The caller should send an `.end` after the final message has been sent.
+ ///
+ /// - Parameters:
+ /// - callOptions: Call options.
+ /// - handler: A closure called when each response is received from the server.
+ /// - Returns: A `ClientStreamingCall` with futures for the metadata and status.
+ internal func control(
+ callOptions: CallOptions? = nil,
+ handler: @escaping (Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest) -> Void
+ ) -> BidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse, Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest> {
+ return self.makeBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientMetadata.Methods.control.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeControlInterceptors() ?? [],
+ handler: handler
+ )
+ }
+
+ /// Used to get the full process bundle descriptors for bundles one
+ /// is asked to process.
+ ///
+ /// - Parameters:
+ /// - request: Request to send to GetProcessBundleDescriptor.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func getProcessBundleDescriptor(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest, Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientMetadata.Methods.getProcessBundleDescriptor.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetProcessBundleDescriptorInterceptors() ?? []
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlNIOClient")
+internal final class Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.BeamFnControl service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlNIOClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.BeamFnControl service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+/// An API that describes the work that a SDK harness is meant to do.
+/// Stable
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientInterceptorFactoryProtocol? { get }
+
+ func makeControlCall(
+ callOptions: CallOptions?
+ ) -> GRPCAsyncBidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse, Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest>
+
+ func makeGetProcessBundleDescriptorCall(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest, Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makeControlCall(
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncBidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse, Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest> {
+ return self.makeAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientMetadata.Methods.control.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeControlInterceptors() ?? []
+ )
+ }
+
+ internal func makeGetProcessBundleDescriptorCall(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest, Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientMetadata.Methods.getProcessBundleDescriptor.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetProcessBundleDescriptorInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlAsyncClientProtocol {
+ internal func control<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest> where RequestStream: Sequence, RequestStream.Element == Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse {
+ return self.performAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientMetadata.Methods.control.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeControlInterceptors() ?? []
+ )
+ }
+
+ internal func control<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest> where RequestStream: AsyncSequence & Sendable, RequestStream.Element == Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse {
+ return self.performAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientMetadata.Methods.control.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeControlInterceptors() ?? []
+ )
+ }
+
+ internal func getProcessBundleDescriptor(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientMetadata.Methods.getProcessBundleDescriptor.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetProcessBundleDescriptorInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlAsyncClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'control'.
+ func makeControlInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse, Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest>]
+
+ /// - Returns: Interceptors to use when invoking 'getProcessBundleDescriptor'.
+ func makeGetProcessBundleDescriptorInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest, Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "BeamFnControl",
+ fullName: "org.apache.beam.model.fn_execution.v1.BeamFnControl",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientMetadata.Methods.control,
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlClientMetadata.Methods.getProcessBundleDescriptor,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let control = GRPCMethodDescriptor(
+ name: "Control",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnControl/Control",
+ type: GRPCCallType.bidirectionalStreaming
+ )
+
+ internal static let getProcessBundleDescriptor = GRPCMethodDescriptor(
+ name: "GetProcessBundleDescriptor",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnControl/GetProcessBundleDescriptor",
+ type: GRPCCallType.unary
+ )
+ }
+}
+
+/// Stable
+///
+/// Usage: instantiate `Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientInterceptorFactoryProtocol? { get }
+
+ func data(
+ callOptions: CallOptions?,
+ handler: @escaping (Org_Apache_Beam_Model_FnExecution_V1_Elements) -> Void
+ ) -> BidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_Elements, Org_Apache_Beam_Model_FnExecution_V1_Elements>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.fn_execution.v1.BeamFnData"
+ }
+
+ /// Used to send data between harnesses.
+ ///
+ /// Callers should use the `send` method on the returned object to send messages
+ /// to the server. The caller should send an `.end` after the final message has been sent.
+ ///
+ /// - Parameters:
+ /// - callOptions: Call options.
+ /// - handler: A closure called when each response is received from the server.
+ /// - Returns: A `ClientStreamingCall` with futures for the metadata and status.
+ internal func data(
+ callOptions: CallOptions? = nil,
+ handler: @escaping (Org_Apache_Beam_Model_FnExecution_V1_Elements) -> Void
+ ) -> BidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_Elements, Org_Apache_Beam_Model_FnExecution_V1_Elements> {
+ return self.makeBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientMetadata.Methods.data.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeDataInterceptors() ?? [],
+ handler: handler
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataNIOClient")
+internal final class Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.BeamFnData service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataNIOClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.BeamFnData service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+/// Stable
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientInterceptorFactoryProtocol? { get }
+
+ func makeDataCall(
+ callOptions: CallOptions?
+ ) -> GRPCAsyncBidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_Elements, Org_Apache_Beam_Model_FnExecution_V1_Elements>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makeDataCall(
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncBidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_Elements, Org_Apache_Beam_Model_FnExecution_V1_Elements> {
+ return self.makeAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientMetadata.Methods.data.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeDataInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataAsyncClientProtocol {
+ internal func data<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_FnExecution_V1_Elements> where RequestStream: Sequence, RequestStream.Element == Org_Apache_Beam_Model_FnExecution_V1_Elements {
+ return self.performAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientMetadata.Methods.data.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeDataInterceptors() ?? []
+ )
+ }
+
+ internal func data<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_FnExecution_V1_Elements> where RequestStream: AsyncSequence & Sendable, RequestStream.Element == Org_Apache_Beam_Model_FnExecution_V1_Elements {
+ return self.performAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientMetadata.Methods.data.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeDataInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataAsyncClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'data'.
+ func makeDataInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_FnExecution_V1_Elements, Org_Apache_Beam_Model_FnExecution_V1_Elements>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "BeamFnData",
+ fullName: "org.apache.beam.model.fn_execution.v1.BeamFnData",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataClientMetadata.Methods.data,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let data = GRPCMethodDescriptor(
+ name: "Data",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnData/Data",
+ type: GRPCCallType.bidirectionalStreaming
+ )
+ }
+}
+
+/// Usage: instantiate `Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientInterceptorFactoryProtocol? { get }
+
+ func state(
+ callOptions: CallOptions?,
+ handler: @escaping (Org_Apache_Beam_Model_FnExecution_V1_StateResponse) -> Void
+ ) -> BidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_StateRequest, Org_Apache_Beam_Model_FnExecution_V1_StateResponse>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.fn_execution.v1.BeamFnState"
+ }
+
+ /// Used to get/append/clear state stored by the runner on behalf of the SDK.
+ ///
+ /// Callers should use the `send` method on the returned object to send messages
+ /// to the server. The caller should send an `.end` after the final message has been sent.
+ ///
+ /// - Parameters:
+ /// - callOptions: Call options.
+ /// - handler: A closure called when each response is received from the server.
+ /// - Returns: A `ClientStreamingCall` with futures for the metadata and status.
+ internal func state(
+ callOptions: CallOptions? = nil,
+ handler: @escaping (Org_Apache_Beam_Model_FnExecution_V1_StateResponse) -> Void
+ ) -> BidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_StateRequest, Org_Apache_Beam_Model_FnExecution_V1_StateResponse> {
+ return self.makeBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientMetadata.Methods.state.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeStateInterceptors() ?? [],
+ handler: handler
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateNIOClient")
+internal final class Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.BeamFnState service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateNIOClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.BeamFnState service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientInterceptorFactoryProtocol? { get }
+
+ func makeStateCall(
+ callOptions: CallOptions?
+ ) -> GRPCAsyncBidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_StateRequest, Org_Apache_Beam_Model_FnExecution_V1_StateResponse>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makeStateCall(
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncBidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_StateRequest, Org_Apache_Beam_Model_FnExecution_V1_StateResponse> {
+ return self.makeAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientMetadata.Methods.state.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeStateInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateAsyncClientProtocol {
+ internal func state<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_FnExecution_V1_StateResponse> where RequestStream: Sequence, RequestStream.Element == Org_Apache_Beam_Model_FnExecution_V1_StateRequest {
+ return self.performAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientMetadata.Methods.state.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeStateInterceptors() ?? []
+ )
+ }
+
+ internal func state<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_FnExecution_V1_StateResponse> where RequestStream: AsyncSequence & Sendable, RequestStream.Element == Org_Apache_Beam_Model_FnExecution_V1_StateRequest {
+ return self.performAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientMetadata.Methods.state.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeStateInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateAsyncClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'state'.
+ func makeStateInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_FnExecution_V1_StateRequest, Org_Apache_Beam_Model_FnExecution_V1_StateResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "BeamFnState",
+ fullName: "org.apache.beam.model.fn_execution.v1.BeamFnState",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateClientMetadata.Methods.state,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let state = GRPCMethodDescriptor(
+ name: "State",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnState/State",
+ type: GRPCCallType.bidirectionalStreaming
+ )
+ }
+}
+
+/// Stable
+///
+/// Usage: instantiate `Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientInterceptorFactoryProtocol? { get }
+
+ func logging(
+ callOptions: CallOptions?,
+ handler: @escaping (Org_Apache_Beam_Model_FnExecution_V1_LogControl) -> Void
+ ) -> BidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List, Org_Apache_Beam_Model_FnExecution_V1_LogControl>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.fn_execution.v1.BeamFnLogging"
+ }
+
+ /// Allows for the SDK to emit log entries which the runner can
+ /// associate with the active job.
+ ///
+ /// Callers should use the `send` method on the returned object to send messages
+ /// to the server. The caller should send an `.end` after the final message has been sent.
+ ///
+ /// - Parameters:
+ /// - callOptions: Call options.
+ /// - handler: A closure called when each response is received from the server.
+ /// - Returns: A `ClientStreamingCall` with futures for the metadata and status.
+ internal func logging(
+ callOptions: CallOptions? = nil,
+ handler: @escaping (Org_Apache_Beam_Model_FnExecution_V1_LogControl) -> Void
+ ) -> BidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List, Org_Apache_Beam_Model_FnExecution_V1_LogControl> {
+ return self.makeBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientMetadata.Methods.logging.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeLoggingInterceptors() ?? [],
+ handler: handler
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingNIOClient")
+internal final class Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.BeamFnLogging service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingNIOClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.BeamFnLogging service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+/// Stable
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientInterceptorFactoryProtocol? { get }
+
+ func makeLoggingCall(
+ callOptions: CallOptions?
+ ) -> GRPCAsyncBidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List, Org_Apache_Beam_Model_FnExecution_V1_LogControl>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makeLoggingCall(
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncBidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List, Org_Apache_Beam_Model_FnExecution_V1_LogControl> {
+ return self.makeAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientMetadata.Methods.logging.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeLoggingInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingAsyncClientProtocol {
+ internal func logging<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_FnExecution_V1_LogControl> where RequestStream: Sequence, RequestStream.Element == Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List {
+ return self.performAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientMetadata.Methods.logging.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeLoggingInterceptors() ?? []
+ )
+ }
+
+ internal func logging<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_FnExecution_V1_LogControl> where RequestStream: AsyncSequence & Sendable, RequestStream.Element == Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List {
+ return self.performAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientMetadata.Methods.logging.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeLoggingInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingAsyncClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'logging'.
+ func makeLoggingInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List, Org_Apache_Beam_Model_FnExecution_V1_LogControl>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "BeamFnLogging",
+ fullName: "org.apache.beam.model.fn_execution.v1.BeamFnLogging",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingClientMetadata.Methods.logging,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let logging = GRPCMethodDescriptor(
+ name: "Logging",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnLogging/Logging",
+ type: GRPCCallType.bidirectionalStreaming
+ )
+ }
+}
+
+/// Usage: instantiate `Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientInterceptorFactoryProtocol? { get }
+
+ func startWorker(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest, Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse>
+
+ func stopWorker(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest, Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool"
+ }
+
+ /// Start the SDK worker with the given ID.
+ ///
+ /// - Parameters:
+ /// - request: Request to send to StartWorker.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func startWorker(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest, Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientMetadata.Methods.startWorker.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeStartWorkerInterceptors() ?? []
+ )
+ }
+
+ /// Stop the SDK worker.
+ ///
+ /// - Parameters:
+ /// - request: Request to send to StopWorker.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func stopWorker(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest, Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientMetadata.Methods.stopWorker.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeStopWorkerInterceptors() ?? []
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolNIOClient")
+internal final class Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolNIOClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientInterceptorFactoryProtocol? { get }
+
+ func makeStartWorkerCall(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest, Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse>
+
+ func makeStopWorkerCall(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest, Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makeStartWorkerCall(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest, Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientMetadata.Methods.startWorker.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeStartWorkerInterceptors() ?? []
+ )
+ }
+
+ internal func makeStopWorkerCall(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest, Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientMetadata.Methods.stopWorker.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeStopWorkerInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolAsyncClientProtocol {
+ internal func startWorker(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientMetadata.Methods.startWorker.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeStartWorkerInterceptors() ?? []
+ )
+ }
+
+ internal func stopWorker(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientMetadata.Methods.stopWorker.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeStopWorkerInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolAsyncClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'startWorker'.
+ func makeStartWorkerInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest, Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse>]
+
+ /// - Returns: Interceptors to use when invoking 'stopWorker'.
+ func makeStopWorkerInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest, Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "BeamFnExternalWorkerPool",
+ fullName: "org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientMetadata.Methods.startWorker,
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolClientMetadata.Methods.stopWorker,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let startWorker = GRPCMethodDescriptor(
+ name: "StartWorker",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool/StartWorker",
+ type: GRPCCallType.unary
+ )
+
+ internal static let stopWorker = GRPCMethodDescriptor(
+ name: "StopWorker",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool/StopWorker",
+ type: GRPCCallType.unary
+ )
+ }
+}
+
+/// API for SDKs to report debug-related statuses to runner during pipeline execution.
+///
+/// Usage: instantiate `Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientInterceptorFactoryProtocol? { get }
+
+ func workerStatus(
+ callOptions: CallOptions?,
+ handler: @escaping (Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest) -> Void
+ ) -> BidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse, Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus"
+ }
+
+ /// Bidirectional streaming call to WorkerStatus
+ ///
+ /// Callers should use the `send` method on the returned object to send messages
+ /// to the server. The caller should send an `.end` after the final message has been sent.
+ ///
+ /// - Parameters:
+ /// - callOptions: Call options.
+ /// - handler: A closure called when each response is received from the server.
+ /// - Returns: A `ClientStreamingCall` with futures for the metadata and status.
+ internal func workerStatus(
+ callOptions: CallOptions? = nil,
+ handler: @escaping (Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest) -> Void
+ ) -> BidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse, Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest> {
+ return self.makeBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientMetadata.Methods.workerStatus.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeWorkerStatusInterceptors() ?? [],
+ handler: handler
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusNIOClient")
+internal final class Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusNIOClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+/// API for SDKs to report debug-related statuses to runner during pipeline execution.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientInterceptorFactoryProtocol? { get }
+
+ func makeWorkerStatusCall(
+ callOptions: CallOptions?
+ ) -> GRPCAsyncBidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse, Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makeWorkerStatusCall(
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncBidirectionalStreamingCall<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse, Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest> {
+ return self.makeAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientMetadata.Methods.workerStatus.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeWorkerStatusInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusAsyncClientProtocol {
+ internal func workerStatus<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest> where RequestStream: Sequence, RequestStream.Element == Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse {
+ return self.performAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientMetadata.Methods.workerStatus.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeWorkerStatusInterceptors() ?? []
+ )
+ }
+
+ internal func workerStatus<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest> where RequestStream: AsyncSequence & Sendable, RequestStream.Element == Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse {
+ return self.performAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientMetadata.Methods.workerStatus.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeWorkerStatusInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusAsyncClient: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'workerStatus'.
+ func makeWorkerStatusInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse, Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "BeamFnWorkerStatus",
+ fullName: "org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusClientMetadata.Methods.workerStatus,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let workerStatus = GRPCMethodDescriptor(
+ name: "WorkerStatus",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus/WorkerStatus",
+ type: GRPCCallType.bidirectionalStreaming
+ )
+ }
+}
+
+/// An API that describes the work that a SDK harness is meant to do.
+/// Stable
+///
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlServerInterceptorFactoryProtocol? { get }
+
+ /// Instructions sent by the runner to the SDK requesting different types
+ /// of work.
+ func control(context: StreamingResponseCallContext<Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest>) -> EventLoopFuture<(StreamEvent<Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse>) -> Void>
+
+ /// Used to get the full process bundle descriptors for bundles one
+ /// is asked to process.
+ func getProcessBundleDescriptor(request: Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "Control":
+ return BidirectionalStreamingServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest>(),
+ interceptors: self.interceptors?.makeControlInterceptors() ?? [],
+ observerFactory: self.control(context:)
+ )
+
+ case "GetProcessBundleDescriptor":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor>(),
+ interceptors: self.interceptors?.makeGetProcessBundleDescriptorInterceptors() ?? [],
+ userFunction: self.getProcessBundleDescriptor(request:context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// An API that describes the work that a SDK harness is meant to do.
+/// Stable
+///
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlServerInterceptorFactoryProtocol? { get }
+
+ /// Instructions sent by the runner to the SDK requesting different types
+ /// of work.
+ func control(
+ requestStream: GRPCAsyncRequestStream<Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse>,
+ responseStream: GRPCAsyncResponseStreamWriter<Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest>,
+ context: GRPCAsyncServerCallContext
+ ) async throws
+
+ /// Used to get the full process bundle descriptors for bundles one
+ /// is asked to process.
+ func getProcessBundleDescriptor(
+ request: Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "Control":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest>(),
+ interceptors: self.interceptors?.makeControlInterceptors() ?? [],
+ wrapping: { try await self.control(requestStream: $0, responseStream: $1, context: $2) }
+ )
+
+ case "GetProcessBundleDescriptor":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor>(),
+ interceptors: self.interceptors?.makeGetProcessBundleDescriptorInterceptors() ?? [],
+ wrapping: { try await self.getProcessBundleDescriptor(request: $0, context: $1) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'control'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeControlInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse, Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest>]
+
+ /// - Returns: Interceptors to use when handling 'getProcessBundleDescriptor'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeGetProcessBundleDescriptorInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest, Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "BeamFnControl",
+ fullName: "org.apache.beam.model.fn_execution.v1.BeamFnControl",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlServerMetadata.Methods.control,
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnControlServerMetadata.Methods.getProcessBundleDescriptor,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let control = GRPCMethodDescriptor(
+ name: "Control",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnControl/Control",
+ type: GRPCCallType.bidirectionalStreaming
+ )
+
+ internal static let getProcessBundleDescriptor = GRPCMethodDescriptor(
+ name: "GetProcessBundleDescriptor",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnControl/GetProcessBundleDescriptor",
+ type: GRPCCallType.unary
+ )
+ }
+}
+/// Stable
+///
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataServerInterceptorFactoryProtocol? { get }
+
+ /// Used to send data between harnesses.
+ func data(context: StreamingResponseCallContext<Org_Apache_Beam_Model_FnExecution_V1_Elements>) -> EventLoopFuture<(StreamEvent<Org_Apache_Beam_Model_FnExecution_V1_Elements>) -> Void>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "Data":
+ return BidirectionalStreamingServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_Elements>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_Elements>(),
+ interceptors: self.interceptors?.makeDataInterceptors() ?? [],
+ observerFactory: self.data(context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// Stable
+///
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataServerInterceptorFactoryProtocol? { get }
+
+ /// Used to send data between harnesses.
+ func data(
+ requestStream: GRPCAsyncRequestStream<Org_Apache_Beam_Model_FnExecution_V1_Elements>,
+ responseStream: GRPCAsyncResponseStreamWriter<Org_Apache_Beam_Model_FnExecution_V1_Elements>,
+ context: GRPCAsyncServerCallContext
+ ) async throws
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "Data":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_Elements>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_Elements>(),
+ interceptors: self.interceptors?.makeDataInterceptors() ?? [],
+ wrapping: { try await self.data(requestStream: $0, responseStream: $1, context: $2) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'data'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeDataInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_FnExecution_V1_Elements, Org_Apache_Beam_Model_FnExecution_V1_Elements>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "BeamFnData",
+ fullName: "org.apache.beam.model.fn_execution.v1.BeamFnData",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnDataServerMetadata.Methods.data,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let data = GRPCMethodDescriptor(
+ name: "Data",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnData/Data",
+ type: GRPCCallType.bidirectionalStreaming
+ )
+ }
+}
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateServerInterceptorFactoryProtocol? { get }
+
+ /// Used to get/append/clear state stored by the runner on behalf of the SDK.
+ func state(context: StreamingResponseCallContext<Org_Apache_Beam_Model_FnExecution_V1_StateResponse>) -> EventLoopFuture<(StreamEvent<Org_Apache_Beam_Model_FnExecution_V1_StateRequest>) -> Void>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "State":
+ return BidirectionalStreamingServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_StateRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_StateResponse>(),
+ interceptors: self.interceptors?.makeStateInterceptors() ?? [],
+ observerFactory: self.state(context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateServerInterceptorFactoryProtocol? { get }
+
+ /// Used to get/append/clear state stored by the runner on behalf of the SDK.
+ func state(
+ requestStream: GRPCAsyncRequestStream<Org_Apache_Beam_Model_FnExecution_V1_StateRequest>,
+ responseStream: GRPCAsyncResponseStreamWriter<Org_Apache_Beam_Model_FnExecution_V1_StateResponse>,
+ context: GRPCAsyncServerCallContext
+ ) async throws
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "State":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_StateRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_StateResponse>(),
+ interceptors: self.interceptors?.makeStateInterceptors() ?? [],
+ wrapping: { try await self.state(requestStream: $0, responseStream: $1, context: $2) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'state'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeStateInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_FnExecution_V1_StateRequest, Org_Apache_Beam_Model_FnExecution_V1_StateResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "BeamFnState",
+ fullName: "org.apache.beam.model.fn_execution.v1.BeamFnState",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnStateServerMetadata.Methods.state,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let state = GRPCMethodDescriptor(
+ name: "State",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnState/State",
+ type: GRPCCallType.bidirectionalStreaming
+ )
+ }
+}
+/// Stable
+///
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingServerInterceptorFactoryProtocol? { get }
+
+ /// Allows for the SDK to emit log entries which the runner can
+ /// associate with the active job.
+ func logging(context: StreamingResponseCallContext<Org_Apache_Beam_Model_FnExecution_V1_LogControl>) -> EventLoopFuture<(StreamEvent<Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List>) -> Void>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "Logging":
+ return BidirectionalStreamingServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_LogControl>(),
+ interceptors: self.interceptors?.makeLoggingInterceptors() ?? [],
+ observerFactory: self.logging(context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// Stable
+///
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingServerInterceptorFactoryProtocol? { get }
+
+ /// Allows for the SDK to emit log entries which the runner can
+ /// associate with the active job.
+ func logging(
+ requestStream: GRPCAsyncRequestStream<Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List>,
+ responseStream: GRPCAsyncResponseStreamWriter<Org_Apache_Beam_Model_FnExecution_V1_LogControl>,
+ context: GRPCAsyncServerCallContext
+ ) async throws
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "Logging":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_LogControl>(),
+ interceptors: self.interceptors?.makeLoggingInterceptors() ?? [],
+ wrapping: { try await self.logging(requestStream: $0, responseStream: $1, context: $2) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'logging'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeLoggingInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List, Org_Apache_Beam_Model_FnExecution_V1_LogControl>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "BeamFnLogging",
+ fullName: "org.apache.beam.model.fn_execution.v1.BeamFnLogging",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnLoggingServerMetadata.Methods.logging,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let logging = GRPCMethodDescriptor(
+ name: "Logging",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnLogging/Logging",
+ type: GRPCCallType.bidirectionalStreaming
+ )
+ }
+}
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolServerInterceptorFactoryProtocol? { get }
+
+ /// Start the SDK worker with the given ID.
+ func startWorker(request: Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse>
+
+ /// Stop the SDK worker.
+ func stopWorker(request: Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "StartWorker":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse>(),
+ interceptors: self.interceptors?.makeStartWorkerInterceptors() ?? [],
+ userFunction: self.startWorker(request:context:)
+ )
+
+ case "StopWorker":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse>(),
+ interceptors: self.interceptors?.makeStopWorkerInterceptors() ?? [],
+ userFunction: self.stopWorker(request:context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolServerInterceptorFactoryProtocol? { get }
+
+ /// Start the SDK worker with the given ID.
+ func startWorker(
+ request: Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse
+
+ /// Stop the SDK worker.
+ func stopWorker(
+ request: Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "StartWorker":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse>(),
+ interceptors: self.interceptors?.makeStartWorkerInterceptors() ?? [],
+ wrapping: { try await self.startWorker(request: $0, context: $1) }
+ )
+
+ case "StopWorker":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse>(),
+ interceptors: self.interceptors?.makeStopWorkerInterceptors() ?? [],
+ wrapping: { try await self.stopWorker(request: $0, context: $1) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'startWorker'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeStartWorkerInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest, Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse>]
+
+ /// - Returns: Interceptors to use when handling 'stopWorker'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeStopWorkerInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest, Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "BeamFnExternalWorkerPool",
+ fullName: "org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolServerMetadata.Methods.startWorker,
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnExternalWorkerPoolServerMetadata.Methods.stopWorker,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let startWorker = GRPCMethodDescriptor(
+ name: "StartWorker",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool/StartWorker",
+ type: GRPCCallType.unary
+ )
+
+ internal static let stopWorker = GRPCMethodDescriptor(
+ name: "StopWorker",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool/StopWorker",
+ type: GRPCCallType.unary
+ )
+ }
+}
+/// API for SDKs to report debug-related statuses to runner during pipeline execution.
+///
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusServerInterceptorFactoryProtocol? { get }
+
+ func workerStatus(context: StreamingResponseCallContext<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest>) -> EventLoopFuture<(StreamEvent<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse>) -> Void>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "WorkerStatus":
+ return BidirectionalStreamingServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest>(),
+ interceptors: self.interceptors?.makeWorkerStatusInterceptors() ?? [],
+ observerFactory: self.workerStatus(context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// API for SDKs to report debug-related statuses to runner during pipeline execution.
+///
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusServerInterceptorFactoryProtocol? { get }
+
+ func workerStatus(
+ requestStream: GRPCAsyncRequestStream<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse>,
+ responseStream: GRPCAsyncResponseStreamWriter<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest>,
+ context: GRPCAsyncServerCallContext
+ ) async throws
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "WorkerStatus":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest>(),
+ interceptors: self.interceptors?.makeWorkerStatusInterceptors() ?? [],
+ wrapping: { try await self.workerStatus(requestStream: $0, responseStream: $1, context: $2) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'workerStatus'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeWorkerStatusInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse, Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "BeamFnWorkerStatus",
+ fullName: "org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_BeamFnWorkerStatusServerMetadata.Methods.workerStatus,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let workerStatus = GRPCMethodDescriptor(
+ name: "WorkerStatus",
+ path: "/org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus/WorkerStatus",
+ type: GRPCCallType.bidirectionalStreaming
+ )
+ }
+}
diff --git a/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/fn_execution/v1/beam_provision_api.grpc.swift b/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/fn_execution/v1/beam_provision_api.grpc.swift
new file mode 100644
index 0000000..254ef54
--- /dev/null
+++ b/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/fn_execution/v1/beam_provision_api.grpc.swift
@@ -0,0 +1,332 @@
+//
+// DO NOT EDIT.
+// swift-format-ignore-file
+//
+// Generated by the protocol buffer compiler.
+// Source: org/apache/beam/model/fn_execution/v1/beam_provision_api.proto
+//
+
+//
+// Copyright 2018, gRPC Authors All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+//
+import GRPC
+import NIO
+import NIOConcurrencyHelpers
+import SwiftProtobuf
+
+
+/// A service to provide runtime provisioning information to the SDK harness
+/// worker instances -- such as pipeline options, resource constraints and
+/// other job metadata -- needed by an SDK harness instance to initialize.
+///
+/// Usage: instantiate `Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientInterceptorFactoryProtocol? { get }
+
+ func getProvisionInfo(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest, Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoResponse>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.fn_execution.v1.ProvisionService"
+ }
+
+ /// Get provision information for the SDK harness worker instance.
+ ///
+ /// - Parameters:
+ /// - request: Request to send to GetProvisionInfo.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func getProvisionInfo(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest, Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientMetadata.Methods.getProvisionInfo.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetProvisionInfoInterceptors() ?? []
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceNIOClient")
+internal final class Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClient: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.ProvisionService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceNIOClient: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.fn_execution.v1.ProvisionService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+/// A service to provide runtime provisioning information to the SDK harness
+/// worker instances -- such as pipeline options, resource constraints and
+/// other job metadata -- needed by an SDK harness instance to initialize.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientInterceptorFactoryProtocol? { get }
+
+ func makeGetProvisionInfoCall(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest, Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoResponse>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makeGetProvisionInfoCall(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest, Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientMetadata.Methods.getProvisionInfo.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetProvisionInfoInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceAsyncClientProtocol {
+ internal func getProvisionInfo(
+ _ request: Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientMetadata.Methods.getProvisionInfo.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetProvisionInfoInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceAsyncClient: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'getProvisionInfo'.
+ func makeGetProvisionInfoInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest, Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "ProvisionService",
+ fullName: "org.apache.beam.model.fn_execution.v1.ProvisionService",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceClientMetadata.Methods.getProvisionInfo,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let getProvisionInfo = GRPCMethodDescriptor(
+ name: "GetProvisionInfo",
+ path: "/org.apache.beam.model.fn_execution.v1.ProvisionService/GetProvisionInfo",
+ type: GRPCCallType.unary
+ )
+ }
+}
+
+/// A service to provide runtime provisioning information to the SDK harness
+/// worker instances -- such as pipeline options, resource constraints and
+/// other job metadata -- needed by an SDK harness instance to initialize.
+///
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceServerInterceptorFactoryProtocol? { get }
+
+ /// Get provision information for the SDK harness worker instance.
+ func getProvisionInfo(request: Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoResponse>
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "GetProvisionInfo":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoResponse>(),
+ interceptors: self.interceptors?.makeGetProvisionInfoInterceptors() ?? [],
+ userFunction: self.getProvisionInfo(request:context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// A service to provide runtime provisioning information to the SDK harness
+/// worker instances -- such as pipeline options, resource constraints and
+/// other job metadata -- needed by an SDK harness instance to initialize.
+///
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceServerInterceptorFactoryProtocol? { get }
+
+ /// Get provision information for the SDK harness worker instance.
+ func getProvisionInfo(
+ request: Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoResponse
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "GetProvisionInfo":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoResponse>(),
+ interceptors: self.interceptors?.makeGetProvisionInfoInterceptors() ?? [],
+ wrapping: { try await self.getProvisionInfo(request: $0, context: $1) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'getProvisionInfo'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeGetProvisionInfoInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoRequest, Org_Apache_Beam_Model_FnExecution_V1_GetProvisionInfoResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "ProvisionService",
+ fullName: "org.apache.beam.model.fn_execution.v1.ProvisionService",
+ methods: [
+ Org_Apache_Beam_Model_FnExecution_V1_ProvisionServiceServerMetadata.Methods.getProvisionInfo,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let getProvisionInfo = GRPCMethodDescriptor(
+ name: "GetProvisionInfo",
+ path: "/org.apache.beam.model.fn_execution.v1.ProvisionService/GetProvisionInfo",
+ type: GRPCCallType.unary
+ )
+ }
+}
diff --git a/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/job_management/v1/beam_artifact_api.grpc.swift b/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/job_management/v1/beam_artifact_api.grpc.swift
new file mode 100644
index 0000000..643e636
--- /dev/null
+++ b/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/job_management/v1/beam_artifact_api.grpc.swift
@@ -0,0 +1,1579 @@
+//
+// DO NOT EDIT.
+// swift-format-ignore-file
+//
+// Generated by the protocol buffer compiler.
+// Source: org/apache/beam/model/job_management/v1/beam_artifact_api.proto
+//
+
+//
+// Copyright 2018, gRPC Authors All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+//
+import GRPC
+import NIO
+import NIOConcurrencyHelpers
+import SwiftProtobuf
+
+
+/// A service to retrieve artifacts for use in a Job.
+///
+/// Usage: instantiate `Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientInterceptorFactoryProtocol? { get }
+
+ func resolveArtifacts(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest, Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsResponse>
+
+ func getArtifact(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest,
+ callOptions: CallOptions?,
+ handler: @escaping (Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse) -> Void
+ ) -> ServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse>
+}
+
+extension Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.job_management.v1.ArtifactRetrievalService"
+ }
+
+ /// Resolves the given artifact references into one or more replacement
+ /// artifact references (e.g. a Maven dependency into a (transitive) set
+ /// of jars.
+ ///
+ /// - Parameters:
+ /// - request: Request to send to ResolveArtifacts.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func resolveArtifacts(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest, Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientMetadata.Methods.resolveArtifacts.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeResolveArtifactsInterceptors() ?? []
+ )
+ }
+
+ /// Retrieves the given artifact as a stream of bytes.
+ ///
+ /// - Parameters:
+ /// - request: Request to send to GetArtifact.
+ /// - callOptions: Call options.
+ /// - handler: A closure called when each response is received from the server.
+ /// - Returns: A `ServerStreamingCall` with futures for the metadata and status.
+ internal func getArtifact(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest,
+ callOptions: CallOptions? = nil,
+ handler: @escaping (Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse) -> Void
+ ) -> ServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse> {
+ return self.makeServerStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientMetadata.Methods.getArtifact.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetArtifactInterceptors() ?? [],
+ handler: handler
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceNIOClient")
+internal final class Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClient: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.job_management.v1.ArtifactRetrievalService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceNIOClient: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.job_management.v1.ArtifactRetrievalService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+/// A service to retrieve artifacts for use in a Job.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientInterceptorFactoryProtocol? { get }
+
+ func makeResolveArtifactsCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest, Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsResponse>
+
+ func makeGetArtifactCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makeResolveArtifactsCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest, Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientMetadata.Methods.resolveArtifacts.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeResolveArtifactsInterceptors() ?? []
+ )
+ }
+
+ internal func makeGetArtifactCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse> {
+ return self.makeAsyncServerStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientMetadata.Methods.getArtifact.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetArtifactInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceAsyncClientProtocol {
+ internal func resolveArtifacts(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientMetadata.Methods.resolveArtifacts.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeResolveArtifactsInterceptors() ?? []
+ )
+ }
+
+ internal func getArtifact(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse> {
+ return self.performAsyncServerStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientMetadata.Methods.getArtifact.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetArtifactInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceAsyncClient: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'resolveArtifacts'.
+ func makeResolveArtifactsInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest, Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsResponse>]
+
+ /// - Returns: Interceptors to use when invoking 'getArtifact'.
+ func makeGetArtifactInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "ArtifactRetrievalService",
+ fullName: "org.apache.beam.model.job_management.v1.ArtifactRetrievalService",
+ methods: [
+ Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientMetadata.Methods.resolveArtifacts,
+ Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceClientMetadata.Methods.getArtifact,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let resolveArtifacts = GRPCMethodDescriptor(
+ name: "ResolveArtifacts",
+ path: "/org.apache.beam.model.job_management.v1.ArtifactRetrievalService/ResolveArtifacts",
+ type: GRPCCallType.unary
+ )
+
+ internal static let getArtifact = GRPCMethodDescriptor(
+ name: "GetArtifact",
+ path: "/org.apache.beam.model.job_management.v1.ArtifactRetrievalService/GetArtifact",
+ type: GRPCCallType.serverStreaming
+ )
+ }
+}
+
+/// A service that allows the client to act as an ArtifactRetrievalService,
+/// for a particular job with the server initiating requests and receiving
+/// responses.
+///
+/// A client calls the service with an ArtifactResponseWrapper that has the
+/// staging token set, and thereafter responds to the server's requests.
+///
+/// Usage: instantiate `Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientInterceptorFactoryProtocol? { get }
+
+ func reverseArtifactRetrievalService(
+ callOptions: CallOptions?,
+ handler: @escaping (Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper) -> Void
+ ) -> BidirectionalStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_ArtifactResponseWrapper, Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper>
+}
+
+extension Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.job_management.v1.ArtifactStagingService"
+ }
+
+ /// Bidirectional streaming call to ReverseArtifactRetrievalService
+ ///
+ /// Callers should use the `send` method on the returned object to send messages
+ /// to the server. The caller should send an `.end` after the final message has been sent.
+ ///
+ /// - Parameters:
+ /// - callOptions: Call options.
+ /// - handler: A closure called when each response is received from the server.
+ /// - Returns: A `ClientStreamingCall` with futures for the metadata and status.
+ internal func reverseArtifactRetrievalService(
+ callOptions: CallOptions? = nil,
+ handler: @escaping (Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper) -> Void
+ ) -> BidirectionalStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_ArtifactResponseWrapper, Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper> {
+ return self.makeBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientMetadata.Methods.reverseArtifactRetrievalService.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeReverseArtifactRetrievalServiceInterceptors() ?? [],
+ handler: handler
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceNIOClient")
+internal final class Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClient: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.job_management.v1.ArtifactStagingService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceNIOClient: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.job_management.v1.ArtifactStagingService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+/// A service that allows the client to act as an ArtifactRetrievalService,
+/// for a particular job with the server initiating requests and receiving
+/// responses.
+///
+/// A client calls the service with an ArtifactResponseWrapper that has the
+/// staging token set, and thereafter responds to the server's requests.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientInterceptorFactoryProtocol? { get }
+
+ func makeReverseArtifactRetrievalServiceCall(
+ callOptions: CallOptions?
+ ) -> GRPCAsyncBidirectionalStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_ArtifactResponseWrapper, Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makeReverseArtifactRetrievalServiceCall(
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncBidirectionalStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_ArtifactResponseWrapper, Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper> {
+ return self.makeAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientMetadata.Methods.reverseArtifactRetrievalService.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeReverseArtifactRetrievalServiceInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceAsyncClientProtocol {
+ internal func reverseArtifactRetrievalService<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper> where RequestStream: Sequence, RequestStream.Element == Org_Apache_Beam_Model_JobManagement_V1_ArtifactResponseWrapper {
+ return self.performAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientMetadata.Methods.reverseArtifactRetrievalService.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeReverseArtifactRetrievalServiceInterceptors() ?? []
+ )
+ }
+
+ internal func reverseArtifactRetrievalService<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper> where RequestStream: AsyncSequence & Sendable, RequestStream.Element == Org_Apache_Beam_Model_JobManagement_V1_ArtifactResponseWrapper {
+ return self.performAsyncBidirectionalStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientMetadata.Methods.reverseArtifactRetrievalService.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeReverseArtifactRetrievalServiceInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceAsyncClient: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'reverseArtifactRetrievalService'.
+ func makeReverseArtifactRetrievalServiceInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_ArtifactResponseWrapper, Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper>]
+}
+
+internal enum Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "ArtifactStagingService",
+ fullName: "org.apache.beam.model.job_management.v1.ArtifactStagingService",
+ methods: [
+ Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceClientMetadata.Methods.reverseArtifactRetrievalService,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let reverseArtifactRetrievalService = GRPCMethodDescriptor(
+ name: "ReverseArtifactRetrievalService",
+ path: "/org.apache.beam.model.job_management.v1.ArtifactStagingService/ReverseArtifactRetrievalService",
+ type: GRPCCallType.bidirectionalStreaming
+ )
+ }
+}
+
+/// A service to stage artifacts for use in a Job.
+///
+/// Usage: instantiate `Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientInterceptorFactoryProtocol? { get }
+
+ func putArtifact(
+ callOptions: CallOptions?
+ ) -> ClientStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_PutArtifactResponse>
+
+ func commitManifest(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest, Org_Apache_Beam_Model_JobManagement_V1_CommitManifestResponse>
+}
+
+extension Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.job_management.v1.LegacyArtifactStagingService"
+ }
+
+ /// Stage an artifact to be available during job execution. The first request must contain the
+ /// name of the artifact. All future requests must contain sequential chunks of the content of
+ /// the artifact.
+ ///
+ /// Callers should use the `send` method on the returned object to send messages
+ /// to the server. The caller should send an `.end` after the final message has been sent.
+ ///
+ /// - Parameters:
+ /// - callOptions: Call options.
+ /// - Returns: A `ClientStreamingCall` with futures for the metadata, status and response.
+ internal func putArtifact(
+ callOptions: CallOptions? = nil
+ ) -> ClientStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_PutArtifactResponse> {
+ return self.makeClientStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientMetadata.Methods.putArtifact.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makePutArtifactInterceptors() ?? []
+ )
+ }
+
+ /// Commit the manifest for a Job. All artifacts must have been successfully uploaded
+ /// before this call is made.
+ ///
+ /// Throws error INVALID_ARGUMENT if not all of the members of the manifest are present
+ ///
+ /// - Parameters:
+ /// - request: Request to send to CommitManifest.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func commitManifest(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest, Org_Apache_Beam_Model_JobManagement_V1_CommitManifestResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientMetadata.Methods.commitManifest.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeCommitManifestInterceptors() ?? []
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceNIOClient")
+internal final class Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClient: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.job_management.v1.LegacyArtifactStagingService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceNIOClient: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.job_management.v1.LegacyArtifactStagingService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+/// A service to stage artifacts for use in a Job.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientInterceptorFactoryProtocol? { get }
+
+ func makePutArtifactCall(
+ callOptions: CallOptions?
+ ) -> GRPCAsyncClientStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_PutArtifactResponse>
+
+ func makeCommitManifestCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest, Org_Apache_Beam_Model_JobManagement_V1_CommitManifestResponse>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makePutArtifactCall(
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncClientStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_PutArtifactResponse> {
+ return self.makeAsyncClientStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientMetadata.Methods.putArtifact.path,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makePutArtifactInterceptors() ?? []
+ )
+ }
+
+ internal func makeCommitManifestCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest, Org_Apache_Beam_Model_JobManagement_V1_CommitManifestResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientMetadata.Methods.commitManifest.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeCommitManifestInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceAsyncClientProtocol {
+ internal func putArtifact<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_PutArtifactResponse where RequestStream: Sequence, RequestStream.Element == Org_Apache_Beam_Model_JobManagement_V1_PutArtifactRequest {
+ return try await self.performAsyncClientStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientMetadata.Methods.putArtifact.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makePutArtifactInterceptors() ?? []
+ )
+ }
+
+ internal func putArtifact<RequestStream>(
+ _ requests: RequestStream,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_PutArtifactResponse where RequestStream: AsyncSequence & Sendable, RequestStream.Element == Org_Apache_Beam_Model_JobManagement_V1_PutArtifactRequest {
+ return try await self.performAsyncClientStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientMetadata.Methods.putArtifact.path,
+ requests: requests,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makePutArtifactInterceptors() ?? []
+ )
+ }
+
+ internal func commitManifest(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_CommitManifestResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientMetadata.Methods.commitManifest.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeCommitManifestInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceAsyncClient: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'putArtifact'.
+ func makePutArtifactInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_PutArtifactResponse>]
+
+ /// - Returns: Interceptors to use when invoking 'commitManifest'.
+ func makeCommitManifestInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest, Org_Apache_Beam_Model_JobManagement_V1_CommitManifestResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "LegacyArtifactStagingService",
+ fullName: "org.apache.beam.model.job_management.v1.LegacyArtifactStagingService",
+ methods: [
+ Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientMetadata.Methods.putArtifact,
+ Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceClientMetadata.Methods.commitManifest,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let putArtifact = GRPCMethodDescriptor(
+ name: "PutArtifact",
+ path: "/org.apache.beam.model.job_management.v1.LegacyArtifactStagingService/PutArtifact",
+ type: GRPCCallType.clientStreaming
+ )
+
+ internal static let commitManifest = GRPCMethodDescriptor(
+ name: "CommitManifest",
+ path: "/org.apache.beam.model.job_management.v1.LegacyArtifactStagingService/CommitManifest",
+ type: GRPCCallType.unary
+ )
+ }
+}
+
+/// A service to retrieve artifacts for use in a Job.
+///
+/// Usage: instantiate `Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientInterceptorFactoryProtocol? { get }
+
+ func getManifest(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest, Org_Apache_Beam_Model_JobManagement_V1_GetManifestResponse>
+
+ func getArtifact(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest,
+ callOptions: CallOptions?,
+ handler: @escaping (Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk) -> Void
+ ) -> ServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk>
+}
+
+extension Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService"
+ }
+
+ /// Get the manifest for the job
+ ///
+ /// - Parameters:
+ /// - request: Request to send to GetManifest.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func getManifest(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest, Org_Apache_Beam_Model_JobManagement_V1_GetManifestResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientMetadata.Methods.getManifest.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetManifestInterceptors() ?? []
+ )
+ }
+
+ /// Get an artifact staged for the job. The requested artifact must be within the manifest
+ ///
+ /// - Parameters:
+ /// - request: Request to send to GetArtifact.
+ /// - callOptions: Call options.
+ /// - handler: A closure called when each response is received from the server.
+ /// - Returns: A `ServerStreamingCall` with futures for the metadata and status.
+ internal func getArtifact(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest,
+ callOptions: CallOptions? = nil,
+ handler: @escaping (Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk) -> Void
+ ) -> ServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk> {
+ return self.makeServerStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientMetadata.Methods.getArtifact.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetArtifactInterceptors() ?? [],
+ handler: handler
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceNIOClient")
+internal final class Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClient: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceNIOClient: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+/// A service to retrieve artifacts for use in a Job.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientInterceptorFactoryProtocol? { get }
+
+ func makeGetManifestCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest, Org_Apache_Beam_Model_JobManagement_V1_GetManifestResponse>
+
+ func makeGetArtifactCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makeGetManifestCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest, Org_Apache_Beam_Model_JobManagement_V1_GetManifestResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientMetadata.Methods.getManifest.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetManifestInterceptors() ?? []
+ )
+ }
+
+ internal func makeGetArtifactCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk> {
+ return self.makeAsyncServerStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientMetadata.Methods.getArtifact.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetArtifactInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceAsyncClientProtocol {
+ internal func getManifest(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_GetManifestResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientMetadata.Methods.getManifest.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetManifestInterceptors() ?? []
+ )
+ }
+
+ internal func getArtifact(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk> {
+ return self.performAsyncServerStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientMetadata.Methods.getArtifact.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetArtifactInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceAsyncClient: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'getManifest'.
+ func makeGetManifestInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest, Org_Apache_Beam_Model_JobManagement_V1_GetManifestResponse>]
+
+ /// - Returns: Interceptors to use when invoking 'getArtifact'.
+ func makeGetArtifactInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk>]
+}
+
+internal enum Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "LegacyArtifactRetrievalService",
+ fullName: "org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService",
+ methods: [
+ Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientMetadata.Methods.getManifest,
+ Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceClientMetadata.Methods.getArtifact,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let getManifest = GRPCMethodDescriptor(
+ name: "GetManifest",
+ path: "/org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService/GetManifest",
+ type: GRPCCallType.unary
+ )
+
+ internal static let getArtifact = GRPCMethodDescriptor(
+ name: "GetArtifact",
+ path: "/org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService/GetArtifact",
+ type: GRPCCallType.serverStreaming
+ )
+ }
+}
+
+/// A service to retrieve artifacts for use in a Job.
+///
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceServerInterceptorFactoryProtocol? { get }
+
+ /// Resolves the given artifact references into one or more replacement
+ /// artifact references (e.g. a Maven dependency into a (transitive) set
+ /// of jars.
+ func resolveArtifacts(request: Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsResponse>
+
+ /// Retrieves the given artifact as a stream of bytes.
+ func getArtifact(request: Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest, context: StreamingResponseCallContext<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse>) -> EventLoopFuture<GRPCStatus>
+}
+
+extension Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "ResolveArtifacts":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsResponse>(),
+ interceptors: self.interceptors?.makeResolveArtifactsInterceptors() ?? [],
+ userFunction: self.resolveArtifacts(request:context:)
+ )
+
+ case "GetArtifact":
+ return ServerStreamingServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse>(),
+ interceptors: self.interceptors?.makeGetArtifactInterceptors() ?? [],
+ userFunction: self.getArtifact(request:context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// A service to retrieve artifacts for use in a Job.
+///
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceServerInterceptorFactoryProtocol? { get }
+
+ /// Resolves the given artifact references into one or more replacement
+ /// artifact references (e.g. a Maven dependency into a (transitive) set
+ /// of jars.
+ func resolveArtifacts(
+ request: Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsResponse
+
+ /// Retrieves the given artifact as a stream of bytes.
+ func getArtifact(
+ request: Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest,
+ responseStream: GRPCAsyncResponseStreamWriter<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse>,
+ context: GRPCAsyncServerCallContext
+ ) async throws
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "ResolveArtifacts":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsResponse>(),
+ interceptors: self.interceptors?.makeResolveArtifactsInterceptors() ?? [],
+ wrapping: { try await self.resolveArtifacts(request: $0, context: $1) }
+ )
+
+ case "GetArtifact":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse>(),
+ interceptors: self.interceptors?.makeGetArtifactInterceptors() ?? [],
+ wrapping: { try await self.getArtifact(request: $0, responseStream: $1, context: $2) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'resolveArtifacts'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeResolveArtifactsInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsRequest, Org_Apache_Beam_Model_JobManagement_V1_ResolveArtifactsResponse>]
+
+ /// - Returns: Interceptors to use when handling 'getArtifact'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeGetArtifactInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_GetArtifactResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "ArtifactRetrievalService",
+ fullName: "org.apache.beam.model.job_management.v1.ArtifactRetrievalService",
+ methods: [
+ Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceServerMetadata.Methods.resolveArtifacts,
+ Org_Apache_Beam_Model_JobManagement_V1_ArtifactRetrievalServiceServerMetadata.Methods.getArtifact,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let resolveArtifacts = GRPCMethodDescriptor(
+ name: "ResolveArtifacts",
+ path: "/org.apache.beam.model.job_management.v1.ArtifactRetrievalService/ResolveArtifacts",
+ type: GRPCCallType.unary
+ )
+
+ internal static let getArtifact = GRPCMethodDescriptor(
+ name: "GetArtifact",
+ path: "/org.apache.beam.model.job_management.v1.ArtifactRetrievalService/GetArtifact",
+ type: GRPCCallType.serverStreaming
+ )
+ }
+}
+/// A service that allows the client to act as an ArtifactRetrievalService,
+/// for a particular job with the server initiating requests and receiving
+/// responses.
+///
+/// A client calls the service with an ArtifactResponseWrapper that has the
+/// staging token set, and thereafter responds to the server's requests.
+///
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceServerInterceptorFactoryProtocol? { get }
+
+ func reverseArtifactRetrievalService(context: StreamingResponseCallContext<Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper>) -> EventLoopFuture<(StreamEvent<Org_Apache_Beam_Model_JobManagement_V1_ArtifactResponseWrapper>) -> Void>
+}
+
+extension Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "ReverseArtifactRetrievalService":
+ return BidirectionalStreamingServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_ArtifactResponseWrapper>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper>(),
+ interceptors: self.interceptors?.makeReverseArtifactRetrievalServiceInterceptors() ?? [],
+ observerFactory: self.reverseArtifactRetrievalService(context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// A service that allows the client to act as an ArtifactRetrievalService,
+/// for a particular job with the server initiating requests and receiving
+/// responses.
+///
+/// A client calls the service with an ArtifactResponseWrapper that has the
+/// staging token set, and thereafter responds to the server's requests.
+///
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceServerInterceptorFactoryProtocol? { get }
+
+ func reverseArtifactRetrievalService(
+ requestStream: GRPCAsyncRequestStream<Org_Apache_Beam_Model_JobManagement_V1_ArtifactResponseWrapper>,
+ responseStream: GRPCAsyncResponseStreamWriter<Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper>,
+ context: GRPCAsyncServerCallContext
+ ) async throws
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "ReverseArtifactRetrievalService":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_ArtifactResponseWrapper>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper>(),
+ interceptors: self.interceptors?.makeReverseArtifactRetrievalServiceInterceptors() ?? [],
+ wrapping: { try await self.reverseArtifactRetrievalService(requestStream: $0, responseStream: $1, context: $2) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'reverseArtifactRetrievalService'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeReverseArtifactRetrievalServiceInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_ArtifactResponseWrapper, Org_Apache_Beam_Model_JobManagement_V1_ArtifactRequestWrapper>]
+}
+
+internal enum Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "ArtifactStagingService",
+ fullName: "org.apache.beam.model.job_management.v1.ArtifactStagingService",
+ methods: [
+ Org_Apache_Beam_Model_JobManagement_V1_ArtifactStagingServiceServerMetadata.Methods.reverseArtifactRetrievalService,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let reverseArtifactRetrievalService = GRPCMethodDescriptor(
+ name: "ReverseArtifactRetrievalService",
+ path: "/org.apache.beam.model.job_management.v1.ArtifactStagingService/ReverseArtifactRetrievalService",
+ type: GRPCCallType.bidirectionalStreaming
+ )
+ }
+}
+/// A service to stage artifacts for use in a Job.
+///
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceServerInterceptorFactoryProtocol? { get }
+
+ /// Stage an artifact to be available during job execution. The first request must contain the
+ /// name of the artifact. All future requests must contain sequential chunks of the content of
+ /// the artifact.
+ func putArtifact(context: UnaryResponseCallContext<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactResponse>) -> EventLoopFuture<(StreamEvent<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactRequest>) -> Void>
+
+ /// Commit the manifest for a Job. All artifacts must have been successfully uploaded
+ /// before this call is made.
+ ///
+ /// Throws error INVALID_ARGUMENT if not all of the members of the manifest are present
+ func commitManifest(request: Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_JobManagement_V1_CommitManifestResponse>
+}
+
+extension Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "PutArtifact":
+ return ClientStreamingServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactResponse>(),
+ interceptors: self.interceptors?.makePutArtifactInterceptors() ?? [],
+ observerFactory: self.putArtifact(context:)
+ )
+
+ case "CommitManifest":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_CommitManifestResponse>(),
+ interceptors: self.interceptors?.makeCommitManifestInterceptors() ?? [],
+ userFunction: self.commitManifest(request:context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// A service to stage artifacts for use in a Job.
+///
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceServerInterceptorFactoryProtocol? { get }
+
+ /// Stage an artifact to be available during job execution. The first request must contain the
+ /// name of the artifact. All future requests must contain sequential chunks of the content of
+ /// the artifact.
+ func putArtifact(
+ requestStream: GRPCAsyncRequestStream<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactRequest>,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_PutArtifactResponse
+
+ /// Commit the manifest for a Job. All artifacts must have been successfully uploaded
+ /// before this call is made.
+ ///
+ /// Throws error INVALID_ARGUMENT if not all of the members of the manifest are present
+ func commitManifest(
+ request: Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_CommitManifestResponse
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "PutArtifact":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactResponse>(),
+ interceptors: self.interceptors?.makePutArtifactInterceptors() ?? [],
+ wrapping: { try await self.putArtifact(requestStream: $0, context: $1) }
+ )
+
+ case "CommitManifest":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_CommitManifestResponse>(),
+ interceptors: self.interceptors?.makeCommitManifestInterceptors() ?? [],
+ wrapping: { try await self.commitManifest(request: $0, context: $1) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'putArtifact'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makePutArtifactInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_PutArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_PutArtifactResponse>]
+
+ /// - Returns: Interceptors to use when handling 'commitManifest'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeCommitManifestInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_CommitManifestRequest, Org_Apache_Beam_Model_JobManagement_V1_CommitManifestResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "LegacyArtifactStagingService",
+ fullName: "org.apache.beam.model.job_management.v1.LegacyArtifactStagingService",
+ methods: [
+ Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceServerMetadata.Methods.putArtifact,
+ Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactStagingServiceServerMetadata.Methods.commitManifest,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let putArtifact = GRPCMethodDescriptor(
+ name: "PutArtifact",
+ path: "/org.apache.beam.model.job_management.v1.LegacyArtifactStagingService/PutArtifact",
+ type: GRPCCallType.clientStreaming
+ )
+
+ internal static let commitManifest = GRPCMethodDescriptor(
+ name: "CommitManifest",
+ path: "/org.apache.beam.model.job_management.v1.LegacyArtifactStagingService/CommitManifest",
+ type: GRPCCallType.unary
+ )
+ }
+}
+/// A service to retrieve artifacts for use in a Job.
+///
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceServerInterceptorFactoryProtocol? { get }
+
+ /// Get the manifest for the job
+ func getManifest(request: Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_JobManagement_V1_GetManifestResponse>
+
+ /// Get an artifact staged for the job. The requested artifact must be within the manifest
+ func getArtifact(request: Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest, context: StreamingResponseCallContext<Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk>) -> EventLoopFuture<GRPCStatus>
+}
+
+extension Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "GetManifest":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_GetManifestResponse>(),
+ interceptors: self.interceptors?.makeGetManifestInterceptors() ?? [],
+ userFunction: self.getManifest(request:context:)
+ )
+
+ case "GetArtifact":
+ return ServerStreamingServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk>(),
+ interceptors: self.interceptors?.makeGetArtifactInterceptors() ?? [],
+ userFunction: self.getArtifact(request:context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// A service to retrieve artifacts for use in a Job.
+///
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceServerInterceptorFactoryProtocol? { get }
+
+ /// Get the manifest for the job
+ func getManifest(
+ request: Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_GetManifestResponse
+
+ /// Get an artifact staged for the job. The requested artifact must be within the manifest
+ func getArtifact(
+ request: Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest,
+ responseStream: GRPCAsyncResponseStreamWriter<Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk>,
+ context: GRPCAsyncServerCallContext
+ ) async throws
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "GetManifest":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_GetManifestResponse>(),
+ interceptors: self.interceptors?.makeGetManifestInterceptors() ?? [],
+ wrapping: { try await self.getManifest(request: $0, context: $1) }
+ )
+
+ case "GetArtifact":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk>(),
+ interceptors: self.interceptors?.makeGetArtifactInterceptors() ?? [],
+ wrapping: { try await self.getArtifact(request: $0, responseStream: $1, context: $2) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'getManifest'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeGetManifestInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetManifestRequest, Org_Apache_Beam_Model_JobManagement_V1_GetManifestResponse>]
+
+ /// - Returns: Interceptors to use when handling 'getArtifact'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeGetArtifactInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_LegacyGetArtifactRequest, Org_Apache_Beam_Model_JobManagement_V1_ArtifactChunk>]
+}
+
+internal enum Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "LegacyArtifactRetrievalService",
+ fullName: "org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService",
+ methods: [
+ Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceServerMetadata.Methods.getManifest,
+ Org_Apache_Beam_Model_JobManagement_V1_LegacyArtifactRetrievalServiceServerMetadata.Methods.getArtifact,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let getManifest = GRPCMethodDescriptor(
+ name: "GetManifest",
+ path: "/org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService/GetManifest",
+ type: GRPCCallType.unary
+ )
+
+ internal static let getArtifact = GRPCMethodDescriptor(
+ name: "GetArtifact",
+ path: "/org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService/GetArtifact",
+ type: GRPCCallType.serverStreaming
+ )
+ }
+}
diff --git a/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/job_management/v1/beam_expansion_api.grpc.swift b/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/job_management/v1/beam_expansion_api.grpc.swift
new file mode 100644
index 0000000..9088730
--- /dev/null
+++ b/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/job_management/v1/beam_expansion_api.grpc.swift
@@ -0,0 +1,425 @@
+//
+// DO NOT EDIT.
+// swift-format-ignore-file
+//
+// Generated by the protocol buffer compiler.
+// Source: org/apache/beam/model/job_management/v1/beam_expansion_api.proto
+//
+
+//
+// Copyright 2018, gRPC Authors All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+//
+import GRPC
+import NIO
+import NIOConcurrencyHelpers
+import SwiftProtobuf
+
+
+/// Job Service for constructing pipelines
+///
+/// Usage: instantiate `Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientInterceptorFactoryProtocol? { get }
+
+ func expand(
+ _ request: Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest, Org_Apache_Beam_Model_Expansion_V1_ExpansionResponse>
+
+ func discoverSchemaTransform(
+ _ request: Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest, Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformResponse>
+}
+
+extension Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.expansion.v1.ExpansionService"
+ }
+
+ /// Unary call to Expand
+ ///
+ /// - Parameters:
+ /// - request: Request to send to Expand.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func expand(
+ _ request: Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest, Org_Apache_Beam_Model_Expansion_V1_ExpansionResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientMetadata.Methods.expand.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeExpandInterceptors() ?? []
+ )
+ }
+
+ ///A RPC to discover already registered SchemaTransformProviders.
+ /// See https://s.apache.org/easy-multi-language for more details.
+ ///
+ /// - Parameters:
+ /// - request: Request to send to DiscoverSchemaTransform.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func discoverSchemaTransform(
+ _ request: Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest, Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientMetadata.Methods.discoverSchemaTransform.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeDiscoverSchemaTransformInterceptors() ?? []
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceNIOClient")
+internal final class Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClient: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.expansion.v1.ExpansionService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceNIOClient: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.expansion.v1.ExpansionService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+/// Job Service for constructing pipelines
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientInterceptorFactoryProtocol? { get }
+
+ func makeExpandCall(
+ _ request: Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest, Org_Apache_Beam_Model_Expansion_V1_ExpansionResponse>
+
+ func makeDiscoverSchemaTransformCall(
+ _ request: Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest, Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformResponse>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makeExpandCall(
+ _ request: Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest, Org_Apache_Beam_Model_Expansion_V1_ExpansionResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientMetadata.Methods.expand.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeExpandInterceptors() ?? []
+ )
+ }
+
+ internal func makeDiscoverSchemaTransformCall(
+ _ request: Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest, Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientMetadata.Methods.discoverSchemaTransform.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeDiscoverSchemaTransformInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceAsyncClientProtocol {
+ internal func expand(
+ _ request: Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_Expansion_V1_ExpansionResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientMetadata.Methods.expand.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeExpandInterceptors() ?? []
+ )
+ }
+
+ internal func discoverSchemaTransform(
+ _ request: Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientMetadata.Methods.discoverSchemaTransform.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeDiscoverSchemaTransformInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceAsyncClient: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'expand'.
+ func makeExpandInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest, Org_Apache_Beam_Model_Expansion_V1_ExpansionResponse>]
+
+ /// - Returns: Interceptors to use when invoking 'discoverSchemaTransform'.
+ func makeDiscoverSchemaTransformInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest, Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "ExpansionService",
+ fullName: "org.apache.beam.model.expansion.v1.ExpansionService",
+ methods: [
+ Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientMetadata.Methods.expand,
+ Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceClientMetadata.Methods.discoverSchemaTransform,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let expand = GRPCMethodDescriptor(
+ name: "Expand",
+ path: "/org.apache.beam.model.expansion.v1.ExpansionService/Expand",
+ type: GRPCCallType.unary
+ )
+
+ internal static let discoverSchemaTransform = GRPCMethodDescriptor(
+ name: "DiscoverSchemaTransform",
+ path: "/org.apache.beam.model.expansion.v1.ExpansionService/DiscoverSchemaTransform",
+ type: GRPCCallType.unary
+ )
+ }
+}
+
+/// Job Service for constructing pipelines
+///
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceServerInterceptorFactoryProtocol? { get }
+
+ func expand(request: Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_Expansion_V1_ExpansionResponse>
+
+ ///A RPC to discover already registered SchemaTransformProviders.
+ /// See https://s.apache.org/easy-multi-language for more details.
+ func discoverSchemaTransform(request: Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformResponse>
+}
+
+extension Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "Expand":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_Expansion_V1_ExpansionResponse>(),
+ interceptors: self.interceptors?.makeExpandInterceptors() ?? [],
+ userFunction: self.expand(request:context:)
+ )
+
+ case "DiscoverSchemaTransform":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformResponse>(),
+ interceptors: self.interceptors?.makeDiscoverSchemaTransformInterceptors() ?? [],
+ userFunction: self.discoverSchemaTransform(request:context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// Job Service for constructing pipelines
+///
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceServerInterceptorFactoryProtocol? { get }
+
+ func expand(
+ request: Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_Expansion_V1_ExpansionResponse
+
+ ///A RPC to discover already registered SchemaTransformProviders.
+ /// See https://s.apache.org/easy-multi-language for more details.
+ func discoverSchemaTransform(
+ request: Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformResponse
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "Expand":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_Expansion_V1_ExpansionResponse>(),
+ interceptors: self.interceptors?.makeExpandInterceptors() ?? [],
+ wrapping: { try await self.expand(request: $0, context: $1) }
+ )
+
+ case "DiscoverSchemaTransform":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformResponse>(),
+ interceptors: self.interceptors?.makeDiscoverSchemaTransformInterceptors() ?? [],
+ wrapping: { try await self.discoverSchemaTransform(request: $0, context: $1) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'expand'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeExpandInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_Expansion_V1_ExpansionRequest, Org_Apache_Beam_Model_Expansion_V1_ExpansionResponse>]
+
+ /// - Returns: Interceptors to use when handling 'discoverSchemaTransform'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeDiscoverSchemaTransformInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformRequest, Org_Apache_Beam_Model_Expansion_V1_DiscoverSchemaTransformResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "ExpansionService",
+ fullName: "org.apache.beam.model.expansion.v1.ExpansionService",
+ methods: [
+ Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceServerMetadata.Methods.expand,
+ Org_Apache_Beam_Model_Expansion_V1_ExpansionServiceServerMetadata.Methods.discoverSchemaTransform,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let expand = GRPCMethodDescriptor(
+ name: "Expand",
+ path: "/org.apache.beam.model.expansion.v1.ExpansionService/Expand",
+ type: GRPCCallType.unary
+ )
+
+ internal static let discoverSchemaTransform = GRPCMethodDescriptor(
+ name: "DiscoverSchemaTransform",
+ path: "/org.apache.beam.model.expansion.v1.ExpansionService/DiscoverSchemaTransform",
+ type: GRPCCallType.unary
+ )
+ }
+}
diff --git a/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/job_management/v1/beam_job_api.grpc.swift b/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/job_management/v1/beam_job_api.grpc.swift
new file mode 100644
index 0000000..75949ff
--- /dev/null
+++ b/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/job_management/v1/beam_job_api.grpc.swift
@@ -0,0 +1,1237 @@
+//
+// DO NOT EDIT.
+// swift-format-ignore-file
+//
+// Generated by the protocol buffer compiler.
+// Source: org/apache/beam/model/job_management/v1/beam_job_api.proto
+//
+
+//
+// Copyright 2018, gRPC Authors All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+//
+import GRPC
+import NIO
+import NIOConcurrencyHelpers
+import SwiftProtobuf
+
+
+/// Job Service for running RunnerAPI pipelines
+///
+/// Usage: instantiate `Org_Apache_Beam_Model_JobManagement_V1_JobServiceClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientInterceptorFactoryProtocol? { get }
+
+ func prepare(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest, Org_Apache_Beam_Model_JobManagement_V1_PrepareJobResponse>
+
+ func run(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest, Org_Apache_Beam_Model_JobManagement_V1_RunJobResponse>
+
+ func getJobs(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobsResponse>
+
+ func getState(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>
+
+ func getPipeline(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineResponse>
+
+ func cancel(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest, Org_Apache_Beam_Model_JobManagement_V1_CancelJobResponse>
+
+ func getStateStream(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest,
+ callOptions: CallOptions?,
+ handler: @escaping (Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent) -> Void
+ ) -> ServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>
+
+ func getMessageStream(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest,
+ callOptions: CallOptions?,
+ handler: @escaping (Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse) -> Void
+ ) -> ServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest, Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse>
+
+ func getJobMetrics(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsResponse>
+
+ func describePipelineOptions(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest,
+ callOptions: CallOptions?
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest, Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsResponse>
+}
+
+extension Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.job_management.v1.JobService"
+ }
+
+ /// Prepare a job for execution. The job will not be executed until a call is made to run with the
+ /// returned preparationId.
+ ///
+ /// - Parameters:
+ /// - request: Request to send to Prepare.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func prepare(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest, Org_Apache_Beam_Model_JobManagement_V1_PrepareJobResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.prepare.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makePrepareInterceptors() ?? []
+ )
+ }
+
+ /// Submit the job for execution
+ ///
+ /// - Parameters:
+ /// - request: Request to send to Run.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func run(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest, Org_Apache_Beam_Model_JobManagement_V1_RunJobResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.run.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeRunInterceptors() ?? []
+ )
+ }
+
+ /// Get a list of all invoked jobs
+ ///
+ /// - Parameters:
+ /// - request: Request to send to GetJobs.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func getJobs(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobsResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getJobs.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetJobsInterceptors() ?? []
+ )
+ }
+
+ /// Get the current state of the job
+ ///
+ /// - Parameters:
+ /// - request: Request to send to GetState.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func getState(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getState.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetStateInterceptors() ?? []
+ )
+ }
+
+ /// Get the job's pipeline
+ ///
+ /// - Parameters:
+ /// - request: Request to send to GetPipeline.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func getPipeline(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getPipeline.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetPipelineInterceptors() ?? []
+ )
+ }
+
+ /// Cancel the job
+ ///
+ /// - Parameters:
+ /// - request: Request to send to Cancel.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func cancel(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest, Org_Apache_Beam_Model_JobManagement_V1_CancelJobResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.cancel.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeCancelInterceptors() ?? []
+ )
+ }
+
+ /// Subscribe to a stream of state changes of the job, will immediately return the current state of the job as the first response.
+ ///
+ /// - Parameters:
+ /// - request: Request to send to GetStateStream.
+ /// - callOptions: Call options.
+ /// - handler: A closure called when each response is received from the server.
+ /// - Returns: A `ServerStreamingCall` with futures for the metadata and status.
+ internal func getStateStream(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest,
+ callOptions: CallOptions? = nil,
+ handler: @escaping (Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent) -> Void
+ ) -> ServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent> {
+ return self.makeServerStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getStateStream.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetStateStreamInterceptors() ?? [],
+ handler: handler
+ )
+ }
+
+ /// Subscribe to a stream of state changes and messages from the job
+ ///
+ /// - Parameters:
+ /// - request: Request to send to GetMessageStream.
+ /// - callOptions: Call options.
+ /// - handler: A closure called when each response is received from the server.
+ /// - Returns: A `ServerStreamingCall` with futures for the metadata and status.
+ internal func getMessageStream(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest,
+ callOptions: CallOptions? = nil,
+ handler: @escaping (Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse) -> Void
+ ) -> ServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest, Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse> {
+ return self.makeServerStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getMessageStream.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetMessageStreamInterceptors() ?? [],
+ handler: handler
+ )
+ }
+
+ /// Fetch metrics for a given job
+ ///
+ /// - Parameters:
+ /// - request: Request to send to GetJobMetrics.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func getJobMetrics(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getJobMetrics.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetJobMetricsInterceptors() ?? []
+ )
+ }
+
+ /// Get the supported pipeline options of the runner
+ ///
+ /// - Parameters:
+ /// - request: Request to send to DescribePipelineOptions.
+ /// - callOptions: Call options.
+ /// - Returns: A `UnaryCall` with futures for the metadata, status and response.
+ internal func describePipelineOptions(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest,
+ callOptions: CallOptions? = nil
+ ) -> UnaryCall<Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest, Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsResponse> {
+ return self.makeUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.describePipelineOptions.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeDescribePipelineOptionsInterceptors() ?? []
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_JobManagement_V1_JobServiceClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_JobManagement_V1_JobServiceNIOClient")
+internal final class Org_Apache_Beam_Model_JobManagement_V1_JobServiceClient: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.job_management.v1.JobService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_JobManagement_V1_JobServiceNIOClient: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.job_management.v1.JobService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+/// Job Service for running RunnerAPI pipelines
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_JobServiceAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientInterceptorFactoryProtocol? { get }
+
+ func makePrepareCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest, Org_Apache_Beam_Model_JobManagement_V1_PrepareJobResponse>
+
+ func makeRunCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest, Org_Apache_Beam_Model_JobManagement_V1_RunJobResponse>
+
+ func makeGetJobsCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobsResponse>
+
+ func makeGetStateCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>
+
+ func makeGetPipelineCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineResponse>
+
+ func makeCancelCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest, Org_Apache_Beam_Model_JobManagement_V1_CancelJobResponse>
+
+ func makeGetStateStreamCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>
+
+ func makeGetMessageStreamCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest, Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse>
+
+ func makeGetJobMetricsCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsResponse>
+
+ func makeDescribePipelineOptionsCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest, Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsResponse>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_JobServiceAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makePrepareCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest, Org_Apache_Beam_Model_JobManagement_V1_PrepareJobResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.prepare.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makePrepareInterceptors() ?? []
+ )
+ }
+
+ internal func makeRunCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest, Org_Apache_Beam_Model_JobManagement_V1_RunJobResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.run.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeRunInterceptors() ?? []
+ )
+ }
+
+ internal func makeGetJobsCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobsResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getJobs.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetJobsInterceptors() ?? []
+ )
+ }
+
+ internal func makeGetStateCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getState.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetStateInterceptors() ?? []
+ )
+ }
+
+ internal func makeGetPipelineCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getPipeline.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetPipelineInterceptors() ?? []
+ )
+ }
+
+ internal func makeCancelCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest, Org_Apache_Beam_Model_JobManagement_V1_CancelJobResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.cancel.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeCancelInterceptors() ?? []
+ )
+ }
+
+ internal func makeGetStateStreamCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent> {
+ return self.makeAsyncServerStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getStateStream.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetStateStreamInterceptors() ?? []
+ )
+ }
+
+ internal func makeGetMessageStreamCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncServerStreamingCall<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest, Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse> {
+ return self.makeAsyncServerStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getMessageStream.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetMessageStreamInterceptors() ?? []
+ )
+ }
+
+ internal func makeGetJobMetricsCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getJobMetrics.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetJobMetricsInterceptors() ?? []
+ )
+ }
+
+ internal func makeDescribePipelineOptionsCall(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncUnaryCall<Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest, Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsResponse> {
+ return self.makeAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.describePipelineOptions.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeDescribePipelineOptionsInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_JobServiceAsyncClientProtocol {
+ internal func prepare(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_PrepareJobResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.prepare.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makePrepareInterceptors() ?? []
+ )
+ }
+
+ internal func run(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_RunJobResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.run.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeRunInterceptors() ?? []
+ )
+ }
+
+ internal func getJobs(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_GetJobsResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getJobs.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetJobsInterceptors() ?? []
+ )
+ }
+
+ internal func getState(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getState.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetStateInterceptors() ?? []
+ )
+ }
+
+ internal func getPipeline(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getPipeline.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetPipelineInterceptors() ?? []
+ )
+ }
+
+ internal func cancel(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_CancelJobResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.cancel.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeCancelInterceptors() ?? []
+ )
+ }
+
+ internal func getStateStream(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent> {
+ return self.performAsyncServerStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getStateStream.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetStateStreamInterceptors() ?? []
+ )
+ }
+
+ internal func getMessageStream(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse> {
+ return self.performAsyncServerStreamingCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getMessageStream.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetMessageStreamInterceptors() ?? []
+ )
+ }
+
+ internal func getJobMetrics(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getJobMetrics.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeGetJobMetricsInterceptors() ?? []
+ )
+ }
+
+ internal func describePipelineOptions(
+ _ request: Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest,
+ callOptions: CallOptions? = nil
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsResponse {
+ return try await self.performAsyncUnaryCall(
+ path: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.describePipelineOptions.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeDescribePipelineOptionsInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_JobManagement_V1_JobServiceAsyncClient: Org_Apache_Beam_Model_JobManagement_V1_JobServiceAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'prepare'.
+ func makePrepareInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest, Org_Apache_Beam_Model_JobManagement_V1_PrepareJobResponse>]
+
+ /// - Returns: Interceptors to use when invoking 'run'.
+ func makeRunInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest, Org_Apache_Beam_Model_JobManagement_V1_RunJobResponse>]
+
+ /// - Returns: Interceptors to use when invoking 'getJobs'.
+ func makeGetJobsInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobsResponse>]
+
+ /// - Returns: Interceptors to use when invoking 'getState'.
+ func makeGetStateInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>]
+
+ /// - Returns: Interceptors to use when invoking 'getPipeline'.
+ func makeGetPipelineInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineResponse>]
+
+ /// - Returns: Interceptors to use when invoking 'cancel'.
+ func makeCancelInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest, Org_Apache_Beam_Model_JobManagement_V1_CancelJobResponse>]
+
+ /// - Returns: Interceptors to use when invoking 'getStateStream'.
+ func makeGetStateStreamInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>]
+
+ /// - Returns: Interceptors to use when invoking 'getMessageStream'.
+ func makeGetMessageStreamInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest, Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse>]
+
+ /// - Returns: Interceptors to use when invoking 'getJobMetrics'.
+ func makeGetJobMetricsInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsResponse>]
+
+ /// - Returns: Interceptors to use when invoking 'describePipelineOptions'.
+ func makeDescribePipelineOptionsInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest, Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "JobService",
+ fullName: "org.apache.beam.model.job_management.v1.JobService",
+ methods: [
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.prepare,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.run,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getJobs,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getState,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getPipeline,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.cancel,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getStateStream,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getMessageStream,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.getJobMetrics,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceClientMetadata.Methods.describePipelineOptions,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let prepare = GRPCMethodDescriptor(
+ name: "Prepare",
+ path: "/org.apache.beam.model.job_management.v1.JobService/Prepare",
+ type: GRPCCallType.unary
+ )
+
+ internal static let run = GRPCMethodDescriptor(
+ name: "Run",
+ path: "/org.apache.beam.model.job_management.v1.JobService/Run",
+ type: GRPCCallType.unary
+ )
+
+ internal static let getJobs = GRPCMethodDescriptor(
+ name: "GetJobs",
+ path: "/org.apache.beam.model.job_management.v1.JobService/GetJobs",
+ type: GRPCCallType.unary
+ )
+
+ internal static let getState = GRPCMethodDescriptor(
+ name: "GetState",
+ path: "/org.apache.beam.model.job_management.v1.JobService/GetState",
+ type: GRPCCallType.unary
+ )
+
+ internal static let getPipeline = GRPCMethodDescriptor(
+ name: "GetPipeline",
+ path: "/org.apache.beam.model.job_management.v1.JobService/GetPipeline",
+ type: GRPCCallType.unary
+ )
+
+ internal static let cancel = GRPCMethodDescriptor(
+ name: "Cancel",
+ path: "/org.apache.beam.model.job_management.v1.JobService/Cancel",
+ type: GRPCCallType.unary
+ )
+
+ internal static let getStateStream = GRPCMethodDescriptor(
+ name: "GetStateStream",
+ path: "/org.apache.beam.model.job_management.v1.JobService/GetStateStream",
+ type: GRPCCallType.serverStreaming
+ )
+
+ internal static let getMessageStream = GRPCMethodDescriptor(
+ name: "GetMessageStream",
+ path: "/org.apache.beam.model.job_management.v1.JobService/GetMessageStream",
+ type: GRPCCallType.serverStreaming
+ )
+
+ internal static let getJobMetrics = GRPCMethodDescriptor(
+ name: "GetJobMetrics",
+ path: "/org.apache.beam.model.job_management.v1.JobService/GetJobMetrics",
+ type: GRPCCallType.unary
+ )
+
+ internal static let describePipelineOptions = GRPCMethodDescriptor(
+ name: "DescribePipelineOptions",
+ path: "/org.apache.beam.model.job_management.v1.JobService/DescribePipelineOptions",
+ type: GRPCCallType.unary
+ )
+ }
+}
+
+/// Job Service for running RunnerAPI pipelines
+///
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_JobServiceProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerInterceptorFactoryProtocol? { get }
+
+ /// Prepare a job for execution. The job will not be executed until a call is made to run with the
+ /// returned preparationId.
+ func prepare(request: Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_JobManagement_V1_PrepareJobResponse>
+
+ /// Submit the job for execution
+ func run(request: Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_JobManagement_V1_RunJobResponse>
+
+ /// Get a list of all invoked jobs
+ func getJobs(request: Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_JobManagement_V1_GetJobsResponse>
+
+ /// Get the current state of the job
+ func getState(request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>
+
+ /// Get the job's pipeline
+ func getPipeline(request: Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineResponse>
+
+ /// Cancel the job
+ func cancel(request: Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_JobManagement_V1_CancelJobResponse>
+
+ /// Subscribe to a stream of state changes of the job, will immediately return the current state of the job as the first response.
+ func getStateStream(request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, context: StreamingResponseCallContext<Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>) -> EventLoopFuture<GRPCStatus>
+
+ /// Subscribe to a stream of state changes and messages from the job
+ func getMessageStream(request: Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest, context: StreamingResponseCallContext<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse>) -> EventLoopFuture<GRPCStatus>
+
+ /// Fetch metrics for a given job
+ func getJobMetrics(request: Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsResponse>
+
+ /// Get the supported pipeline options of the runner
+ func describePipelineOptions(request: Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest, context: StatusOnlyCallContext) -> EventLoopFuture<Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsResponse>
+}
+
+extension Org_Apache_Beam_Model_JobManagement_V1_JobServiceProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "Prepare":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_PrepareJobResponse>(),
+ interceptors: self.interceptors?.makePrepareInterceptors() ?? [],
+ userFunction: self.prepare(request:context:)
+ )
+
+ case "Run":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_RunJobResponse>(),
+ interceptors: self.interceptors?.makeRunInterceptors() ?? [],
+ userFunction: self.run(request:context:)
+ )
+
+ case "GetJobs":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobsResponse>(),
+ interceptors: self.interceptors?.makeGetJobsInterceptors() ?? [],
+ userFunction: self.getJobs(request:context:)
+ )
+
+ case "GetState":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>(),
+ interceptors: self.interceptors?.makeGetStateInterceptors() ?? [],
+ userFunction: self.getState(request:context:)
+ )
+
+ case "GetPipeline":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineResponse>(),
+ interceptors: self.interceptors?.makeGetPipelineInterceptors() ?? [],
+ userFunction: self.getPipeline(request:context:)
+ )
+
+ case "Cancel":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_CancelJobResponse>(),
+ interceptors: self.interceptors?.makeCancelInterceptors() ?? [],
+ userFunction: self.cancel(request:context:)
+ )
+
+ case "GetStateStream":
+ return ServerStreamingServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>(),
+ interceptors: self.interceptors?.makeGetStateStreamInterceptors() ?? [],
+ userFunction: self.getStateStream(request:context:)
+ )
+
+ case "GetMessageStream":
+ return ServerStreamingServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse>(),
+ interceptors: self.interceptors?.makeGetMessageStreamInterceptors() ?? [],
+ userFunction: self.getMessageStream(request:context:)
+ )
+
+ case "GetJobMetrics":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsResponse>(),
+ interceptors: self.interceptors?.makeGetJobMetricsInterceptors() ?? [],
+ userFunction: self.getJobMetrics(request:context:)
+ )
+
+ case "DescribePipelineOptions":
+ return UnaryServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsResponse>(),
+ interceptors: self.interceptors?.makeDescribePipelineOptionsInterceptors() ?? [],
+ userFunction: self.describePipelineOptions(request:context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// Job Service for running RunnerAPI pipelines
+///
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_JobServiceAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerInterceptorFactoryProtocol? { get }
+
+ /// Prepare a job for execution. The job will not be executed until a call is made to run with the
+ /// returned preparationId.
+ func prepare(
+ request: Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_PrepareJobResponse
+
+ /// Submit the job for execution
+ func run(
+ request: Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_RunJobResponse
+
+ /// Get a list of all invoked jobs
+ func getJobs(
+ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_GetJobsResponse
+
+ /// Get the current state of the job
+ func getState(
+ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent
+
+ /// Get the job's pipeline
+ func getPipeline(
+ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineResponse
+
+ /// Cancel the job
+ func cancel(
+ request: Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_CancelJobResponse
+
+ /// Subscribe to a stream of state changes of the job, will immediately return the current state of the job as the first response.
+ func getStateStream(
+ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest,
+ responseStream: GRPCAsyncResponseStreamWriter<Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>,
+ context: GRPCAsyncServerCallContext
+ ) async throws
+
+ /// Subscribe to a stream of state changes and messages from the job
+ func getMessageStream(
+ request: Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest,
+ responseStream: GRPCAsyncResponseStreamWriter<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse>,
+ context: GRPCAsyncServerCallContext
+ ) async throws
+
+ /// Fetch metrics for a given job
+ func getJobMetrics(
+ request: Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsResponse
+
+ /// Get the supported pipeline options of the runner
+ func describePipelineOptions(
+ request: Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest,
+ context: GRPCAsyncServerCallContext
+ ) async throws -> Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsResponse
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_JobManagement_V1_JobServiceAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "Prepare":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_PrepareJobResponse>(),
+ interceptors: self.interceptors?.makePrepareInterceptors() ?? [],
+ wrapping: { try await self.prepare(request: $0, context: $1) }
+ )
+
+ case "Run":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_RunJobResponse>(),
+ interceptors: self.interceptors?.makeRunInterceptors() ?? [],
+ wrapping: { try await self.run(request: $0, context: $1) }
+ )
+
+ case "GetJobs":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobsResponse>(),
+ interceptors: self.interceptors?.makeGetJobsInterceptors() ?? [],
+ wrapping: { try await self.getJobs(request: $0, context: $1) }
+ )
+
+ case "GetState":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>(),
+ interceptors: self.interceptors?.makeGetStateInterceptors() ?? [],
+ wrapping: { try await self.getState(request: $0, context: $1) }
+ )
+
+ case "GetPipeline":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineResponse>(),
+ interceptors: self.interceptors?.makeGetPipelineInterceptors() ?? [],
+ wrapping: { try await self.getPipeline(request: $0, context: $1) }
+ )
+
+ case "Cancel":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_CancelJobResponse>(),
+ interceptors: self.interceptors?.makeCancelInterceptors() ?? [],
+ wrapping: { try await self.cancel(request: $0, context: $1) }
+ )
+
+ case "GetStateStream":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>(),
+ interceptors: self.interceptors?.makeGetStateStreamInterceptors() ?? [],
+ wrapping: { try await self.getStateStream(request: $0, responseStream: $1, context: $2) }
+ )
+
+ case "GetMessageStream":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse>(),
+ interceptors: self.interceptors?.makeGetMessageStreamInterceptors() ?? [],
+ wrapping: { try await self.getMessageStream(request: $0, responseStream: $1, context: $2) }
+ )
+
+ case "GetJobMetrics":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsResponse>(),
+ interceptors: self.interceptors?.makeGetJobMetricsInterceptors() ?? [],
+ wrapping: { try await self.getJobMetrics(request: $0, context: $1) }
+ )
+
+ case "DescribePipelineOptions":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsResponse>(),
+ interceptors: self.interceptors?.makeDescribePipelineOptionsInterceptors() ?? [],
+ wrapping: { try await self.describePipelineOptions(request: $0, context: $1) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'prepare'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makePrepareInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_PrepareJobRequest, Org_Apache_Beam_Model_JobManagement_V1_PrepareJobResponse>]
+
+ /// - Returns: Interceptors to use when handling 'run'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeRunInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_RunJobRequest, Org_Apache_Beam_Model_JobManagement_V1_RunJobResponse>]
+
+ /// - Returns: Interceptors to use when handling 'getJobs'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeGetJobsInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetJobsRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobsResponse>]
+
+ /// - Returns: Interceptors to use when handling 'getState'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeGetStateInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>]
+
+ /// - Returns: Interceptors to use when handling 'getPipeline'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeGetPipelineInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobPipelineResponse>]
+
+ /// - Returns: Interceptors to use when handling 'cancel'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeCancelInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_CancelJobRequest, Org_Apache_Beam_Model_JobManagement_V1_CancelJobResponse>]
+
+ /// - Returns: Interceptors to use when handling 'getStateStream'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeGetStateStreamInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetJobStateRequest, Org_Apache_Beam_Model_JobManagement_V1_JobStateEvent>]
+
+ /// - Returns: Interceptors to use when handling 'getMessageStream'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeGetMessageStreamInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_JobMessagesRequest, Org_Apache_Beam_Model_JobManagement_V1_JobMessagesResponse>]
+
+ /// - Returns: Interceptors to use when handling 'getJobMetrics'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeGetJobMetricsInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsRequest, Org_Apache_Beam_Model_JobManagement_V1_GetJobMetricsResponse>]
+
+ /// - Returns: Interceptors to use when handling 'describePipelineOptions'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeDescribePipelineOptionsInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsRequest, Org_Apache_Beam_Model_JobManagement_V1_DescribePipelineOptionsResponse>]
+}
+
+internal enum Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "JobService",
+ fullName: "org.apache.beam.model.job_management.v1.JobService",
+ methods: [
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.Methods.prepare,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.Methods.run,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.Methods.getJobs,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.Methods.getState,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.Methods.getPipeline,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.Methods.cancel,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.Methods.getStateStream,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.Methods.getMessageStream,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.Methods.getJobMetrics,
+ Org_Apache_Beam_Model_JobManagement_V1_JobServiceServerMetadata.Methods.describePipelineOptions,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let prepare = GRPCMethodDescriptor(
+ name: "Prepare",
+ path: "/org.apache.beam.model.job_management.v1.JobService/Prepare",
+ type: GRPCCallType.unary
+ )
+
+ internal static let run = GRPCMethodDescriptor(
+ name: "Run",
+ path: "/org.apache.beam.model.job_management.v1.JobService/Run",
+ type: GRPCCallType.unary
+ )
+
+ internal static let getJobs = GRPCMethodDescriptor(
+ name: "GetJobs",
+ path: "/org.apache.beam.model.job_management.v1.JobService/GetJobs",
+ type: GRPCCallType.unary
+ )
+
+ internal static let getState = GRPCMethodDescriptor(
+ name: "GetState",
+ path: "/org.apache.beam.model.job_management.v1.JobService/GetState",
+ type: GRPCCallType.unary
+ )
+
+ internal static let getPipeline = GRPCMethodDescriptor(
+ name: "GetPipeline",
+ path: "/org.apache.beam.model.job_management.v1.JobService/GetPipeline",
+ type: GRPCCallType.unary
+ )
+
+ internal static let cancel = GRPCMethodDescriptor(
+ name: "Cancel",
+ path: "/org.apache.beam.model.job_management.v1.JobService/Cancel",
+ type: GRPCCallType.unary
+ )
+
+ internal static let getStateStream = GRPCMethodDescriptor(
+ name: "GetStateStream",
+ path: "/org.apache.beam.model.job_management.v1.JobService/GetStateStream",
+ type: GRPCCallType.serverStreaming
+ )
+
+ internal static let getMessageStream = GRPCMethodDescriptor(
+ name: "GetMessageStream",
+ path: "/org.apache.beam.model.job_management.v1.JobService/GetMessageStream",
+ type: GRPCCallType.serverStreaming
+ )
+
+ internal static let getJobMetrics = GRPCMethodDescriptor(
+ name: "GetJobMetrics",
+ path: "/org.apache.beam.model.job_management.v1.JobService/GetJobMetrics",
+ type: GRPCCallType.unary
+ )
+
+ internal static let describePipelineOptions = GRPCMethodDescriptor(
+ name: "DescribePipelineOptions",
+ path: "/org.apache.beam.model.job_management.v1.JobService/DescribePipelineOptions",
+ type: GRPCCallType.unary
+ )
+ }
+}
diff --git a/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/pipeline/v1/beam_runner_api.grpc.swift b/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/pipeline/v1/beam_runner_api.grpc.swift
new file mode 100644
index 0000000..1af4ab5
--- /dev/null
+++ b/Sources/ApacheBeam/Generated/GRPC/org/apache/beam/model/pipeline/v1/beam_runner_api.grpc.swift
@@ -0,0 +1,322 @@
+//
+// DO NOT EDIT.
+// swift-format-ignore-file
+//
+// Generated by the protocol buffer compiler.
+// Source: org/apache/beam/model/pipeline/v1/beam_runner_api.proto
+//
+
+//
+// Copyright 2018, gRPC Authors All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+//
+import GRPC
+import NIO
+import NIOConcurrencyHelpers
+import SwiftProtobuf
+
+
+/// Usage: instantiate `Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClient`, then call methods of this protocol to make API calls.
+internal protocol Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientProtocol: GRPCClient {
+ var serviceName: String { get }
+ var interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientInterceptorFactoryProtocol? { get }
+
+ func events(
+ _ request: Org_Apache_Beam_Model_Pipeline_V1_EventsRequest,
+ callOptions: CallOptions?,
+ handler: @escaping (Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event) -> Void
+ ) -> ServerStreamingCall<Org_Apache_Beam_Model_Pipeline_V1_EventsRequest, Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event>
+}
+
+extension Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientProtocol {
+ internal var serviceName: String {
+ return "org.apache.beam.model.pipeline.v1.TestStreamService"
+ }
+
+ /// A TestStream will request for events using this RPC.
+ ///
+ /// - Parameters:
+ /// - request: Request to send to Events.
+ /// - callOptions: Call options.
+ /// - handler: A closure called when each response is received from the server.
+ /// - Returns: A `ServerStreamingCall` with futures for the metadata and status.
+ internal func events(
+ _ request: Org_Apache_Beam_Model_Pipeline_V1_EventsRequest,
+ callOptions: CallOptions? = nil,
+ handler: @escaping (Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event) -> Void
+ ) -> ServerStreamingCall<Org_Apache_Beam_Model_Pipeline_V1_EventsRequest, Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event> {
+ return self.makeServerStreamingCall(
+ path: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientMetadata.Methods.events.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeEventsInterceptors() ?? [],
+ handler: handler
+ )
+ }
+}
+
+@available(*, deprecated)
+extension Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClient: @unchecked Sendable {}
+
+@available(*, deprecated, renamed: "Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceNIOClient")
+internal final class Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClient: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientProtocol {
+ private let lock = Lock()
+ private var _defaultCallOptions: CallOptions
+ private var _interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientInterceptorFactoryProtocol?
+ internal let channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions {
+ get { self.lock.withLock { return self._defaultCallOptions } }
+ set { self.lock.withLockVoid { self._defaultCallOptions = newValue } }
+ }
+ internal var interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientInterceptorFactoryProtocol? {
+ get { self.lock.withLock { return self._interceptors } }
+ set { self.lock.withLockVoid { self._interceptors = newValue } }
+ }
+
+ /// Creates a client for the org.apache.beam.model.pipeline.v1.TestStreamService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self._defaultCallOptions = defaultCallOptions
+ self._interceptors = interceptors
+ }
+}
+
+internal struct Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceNIOClient: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientInterceptorFactoryProtocol?
+
+ /// Creates a client for the org.apache.beam.model.pipeline.v1.TestStreamService service.
+ ///
+ /// - Parameters:
+ /// - channel: `GRPCChannel` to the service host.
+ /// - defaultCallOptions: Options to use for each service call if the user doesn't provide them.
+ /// - interceptors: A factory providing interceptors for each RPC.
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceAsyncClientProtocol: GRPCClient {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientInterceptorFactoryProtocol? { get }
+
+ func makeEventsCall(
+ _ request: Org_Apache_Beam_Model_Pipeline_V1_EventsRequest,
+ callOptions: CallOptions?
+ ) -> GRPCAsyncServerStreamingCall<Org_Apache_Beam_Model_Pipeline_V1_EventsRequest, Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event>
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceAsyncClientProtocol {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientMetadata.serviceDescriptor
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func makeEventsCall(
+ _ request: Org_Apache_Beam_Model_Pipeline_V1_EventsRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncServerStreamingCall<Org_Apache_Beam_Model_Pipeline_V1_EventsRequest, Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event> {
+ return self.makeAsyncServerStreamingCall(
+ path: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientMetadata.Methods.events.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeEventsInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceAsyncClientProtocol {
+ internal func events(
+ _ request: Org_Apache_Beam_Model_Pipeline_V1_EventsRequest,
+ callOptions: CallOptions? = nil
+ ) -> GRPCAsyncResponseStream<Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event> {
+ return self.performAsyncServerStreamingCall(
+ path: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientMetadata.Methods.events.path,
+ request: request,
+ callOptions: callOptions ?? self.defaultCallOptions,
+ interceptors: self.interceptors?.makeEventsInterceptors() ?? []
+ )
+ }
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal struct Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceAsyncClient: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceAsyncClientProtocol {
+ internal var channel: GRPCChannel
+ internal var defaultCallOptions: CallOptions
+ internal var interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientInterceptorFactoryProtocol?
+
+ internal init(
+ channel: GRPCChannel,
+ defaultCallOptions: CallOptions = CallOptions(),
+ interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientInterceptorFactoryProtocol? = nil
+ ) {
+ self.channel = channel
+ self.defaultCallOptions = defaultCallOptions
+ self.interceptors = interceptors
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when invoking 'events'.
+ func makeEventsInterceptors() -> [ClientInterceptor<Org_Apache_Beam_Model_Pipeline_V1_EventsRequest, Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event>]
+}
+
+internal enum Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "TestStreamService",
+ fullName: "org.apache.beam.model.pipeline.v1.TestStreamService",
+ methods: [
+ Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceClientMetadata.Methods.events,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let events = GRPCMethodDescriptor(
+ name: "Events",
+ path: "/org.apache.beam.model.pipeline.v1.TestStreamService/Events",
+ type: GRPCCallType.serverStreaming
+ )
+ }
+}
+
+/// To build a server, implement a class that conforms to this protocol.
+internal protocol Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceProvider: CallHandlerProvider {
+ var interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceServerInterceptorFactoryProtocol? { get }
+
+ /// A TestStream will request for events using this RPC.
+ func events(request: Org_Apache_Beam_Model_Pipeline_V1_EventsRequest, context: StreamingResponseCallContext<Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event>) -> EventLoopFuture<GRPCStatus>
+}
+
+extension Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceProvider {
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ /// Determines, calls and returns the appropriate request handler, depending on the request's method.
+ /// Returns nil for methods not handled by this service.
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "Events":
+ return ServerStreamingServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_Pipeline_V1_EventsRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event>(),
+ interceptors: self.interceptors?.makeEventsInterceptors() ?? [],
+ userFunction: self.events(request:context:)
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+/// To implement a server, implement an object which conforms to this protocol.
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+internal protocol Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceAsyncProvider: CallHandlerProvider, Sendable {
+ static var serviceDescriptor: GRPCServiceDescriptor { get }
+ var interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceServerInterceptorFactoryProtocol? { get }
+
+ /// A TestStream will request for events using this RPC.
+ func events(
+ request: Org_Apache_Beam_Model_Pipeline_V1_EventsRequest,
+ responseStream: GRPCAsyncResponseStreamWriter<Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event>,
+ context: GRPCAsyncServerCallContext
+ ) async throws
+}
+
+@available(macOS 10.15, iOS 13, tvOS 13, watchOS 6, *)
+extension Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceAsyncProvider {
+ internal static var serviceDescriptor: GRPCServiceDescriptor {
+ return Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceServerMetadata.serviceDescriptor
+ }
+
+ internal var serviceName: Substring {
+ return Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceServerMetadata.serviceDescriptor.fullName[...]
+ }
+
+ internal var interceptors: Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceServerInterceptorFactoryProtocol? {
+ return nil
+ }
+
+ internal func handle(
+ method name: Substring,
+ context: CallHandlerContext
+ ) -> GRPCServerHandlerProtocol? {
+ switch name {
+ case "Events":
+ return GRPCAsyncServerHandler(
+ context: context,
+ requestDeserializer: ProtobufDeserializer<Org_Apache_Beam_Model_Pipeline_V1_EventsRequest>(),
+ responseSerializer: ProtobufSerializer<Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event>(),
+ interceptors: self.interceptors?.makeEventsInterceptors() ?? [],
+ wrapping: { try await self.events(request: $0, responseStream: $1, context: $2) }
+ )
+
+ default:
+ return nil
+ }
+ }
+}
+
+internal protocol Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceServerInterceptorFactoryProtocol: Sendable {
+
+ /// - Returns: Interceptors to use when handling 'events'.
+ /// Defaults to calling `self.makeInterceptors()`.
+ func makeEventsInterceptors() -> [ServerInterceptor<Org_Apache_Beam_Model_Pipeline_V1_EventsRequest, Org_Apache_Beam_Model_Pipeline_V1_TestStreamPayload.Event>]
+}
+
+internal enum Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceServerMetadata {
+ internal static let serviceDescriptor = GRPCServiceDescriptor(
+ name: "TestStreamService",
+ fullName: "org.apache.beam.model.pipeline.v1.TestStreamService",
+ methods: [
+ Org_Apache_Beam_Model_Pipeline_V1_TestStreamServiceServerMetadata.Methods.events,
+ ]
+ )
+
+ internal enum Methods {
+ internal static let events = GRPCMethodDescriptor(
+ name: "Events",
+ path: "/org.apache.beam.model.pipeline.v1.TestStreamService/Events",
+ type: GRPCCallType.serverStreaming
+ )
+ }
+}
diff --git a/Sources/ApacheBeam/Generated/Model/org/apache/beam/model/fn_execution/v1/beam_fn_api.pb.swift b/Sources/ApacheBeam/Generated/Model/org/apache/beam/model/fn_execution/v1/beam_fn_api.pb.swift
new file mode 100644
index 0000000..328e944
--- /dev/null
+++ b/Sources/ApacheBeam/Generated/Model/org/apache/beam/model/fn_execution/v1/beam_fn_api.pb.swift
@@ -0,0 +1,5317 @@
+// DO NOT EDIT.
+// swift-format-ignore-file
+//
+// Generated by the Swift generator plugin for the protocol buffer compiler.
+// Source: org/apache/beam/model/fn_execution/v1/beam_fn_api.proto
+//
+// For information on using the generated types, please see the documentation:
+// https://github.com/apple/swift-protobuf/
+
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+//
+// Protocol Buffers describing the Fn API and boostrapping.
+//
+// TODO: Usage of plural names in lists looks awkward in Java
+// e.g. getOutputsMap, addCodersBuilder
+//
+// TODO: gRPC / proto field names conflict with generated code
+// e.g. "class" in java, "output" in python
+
+import Foundation
+import SwiftProtobuf
+
+// If the compiler emits an error on this type, it is because this file
+// was generated by a version of the `protoc` Swift plug-in that is
+// incompatible with the version of SwiftProtobuf to which you are linking.
+// Please ensure that you are building against the same version of the API
+// that was used to generate this file.
+fileprivate struct _GeneratedWithProtocGenSwiftVersion: SwiftProtobuf.ProtobufAPIVersionCheck {
+ struct _2: SwiftProtobuf.ProtobufAPIVersion_2 {}
+ typealias Version = _2
+}
+
+/// Describes transforms necessary to execute Beam over the FnAPI but are
+/// implementation details rather than part of the core model.
+struct Org_Apache_Beam_Model_FnExecution_V1_FnApiTransforms {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ enum Runner: SwiftProtobuf.Enum {
+ typealias RawValue = Int
+
+ /// DataSource is a Root Transform, and a source of data for downstream
+ /// transforms in the same ProcessBundleDescriptor.
+ /// It represents a stream of values coming in from an external source/over
+ /// a data channel, typically from the runner. It's not the PCollection itself
+ /// but a description of how to get the portion of the PCollection for a given
+ /// bundle.
+ ///
+ /// The DataSource transform is implemented in each SDK and not explicitly
+ /// provided during pipeline construction. A runner inserts the transform
+ /// in ProcessBundleDescriptors to indicate where the bundle
+ /// can retrieve data for an associated ProcessBundleRequest.
+ /// Data for the same request will be retrieved with the matching instruction ID,
+ /// and transform ID determined by the runner.
+ ///
+ /// The DataSource transform will take a stream of bytes from the remote
+ /// source for the matching instruction ID and decode them as windowed
+ /// values using the provided coder ID, which must be a windowed value coder.
+ ///
+ /// Payload: RemoteGrpcPort
+ case dataSource // = 0
+
+ /// DataSink is a transform that sends PCollection elements to a remote
+ /// port using the Data API.
+ ///
+ /// The DataSink transform is implemented in each SDK and not explicitly
+ /// provided during pipeline construction. A runner inserts the transform in
+ /// ProcessBundleDescriptors to indicate where the bundle can send
+ /// data for each associated ProcessBundleRequest. Data for the same
+ /// request will be sent with the matching instruction ID and transform ID.
+ /// Each PCollection that exits the ProcessBundleDescriptor subgraph will have
+ /// it's own DataSink, keyed by a transform ID determined by the runner.
+ ///
+ /// The DataSink will take in a stream of elements for a given instruction ID
+ /// and encode them for transmission to the remote sink. The coder ID must be
+ /// for a windowed value coder.
+ ///
+ /// Payload: RemoteGrpcPort
+ case dataSink // = 1
+ case UNRECOGNIZED(Int)
+
+ init() {
+ self = .dataSource
+ }
+
+ init?(rawValue: Int) {
+ switch rawValue {
+ case 0: self = .dataSource
+ case 1: self = .dataSink
+ default: self = .UNRECOGNIZED(rawValue)
+ }
+ }
+
+ var rawValue: Int {
+ switch self {
+ case .dataSource: return 0
+ case .dataSink: return 1
+ case .UNRECOGNIZED(let i): return i
+ }
+ }
+
+ }
+
+ init() {}
+}
+
+#if swift(>=4.2)
+
+extension Org_Apache_Beam_Model_FnExecution_V1_FnApiTransforms.Runner: CaseIterable {
+ // The compiler won't synthesize support with the UNRECOGNIZED case.
+ static var allCases: [Org_Apache_Beam_Model_FnExecution_V1_FnApiTransforms.Runner] = [
+ .dataSource,
+ .dataSink,
+ ]
+}
+
+#endif // swift(>=4.2)
+
+/// A descriptor for connecting to a remote port using the Beam Fn Data API.
+/// Allows for communication between two environments (for example between the
+/// runner and the SDK).
+/// Stable
+struct Org_Apache_Beam_Model_FnExecution_V1_RemoteGrpcPort {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) An API descriptor which describes where to
+ /// connect to including any authentication that is required.
+ var apiServiceDescriptor: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor {
+ get {return _apiServiceDescriptor ?? Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor()}
+ set {_apiServiceDescriptor = newValue}
+ }
+ /// Returns true if `apiServiceDescriptor` has been explicitly set.
+ var hasApiServiceDescriptor: Bool {return self._apiServiceDescriptor != nil}
+ /// Clears the value of `apiServiceDescriptor`. Subsequent reads from it will return its default value.
+ mutating func clearApiServiceDescriptor() {self._apiServiceDescriptor = nil}
+
+ /// (Required) The ID of the Coder that will be used to encode and decode data
+ /// sent over this port.
+ var coderID: String = String()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+
+ fileprivate var _apiServiceDescriptor: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor? = nil
+}
+
+/// Requests the ProcessBundleDescriptor with the given id.
+struct Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var processBundleDescriptorID: String = String()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// A request sent by a runner which the SDK is asked to fulfill.
+/// For any unsupported request type, an error should be returned with a
+/// matching instruction id.
+/// Stable
+struct Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) A unique identifier provided by the runner which represents
+ /// this requests execution. The InstructionResponse MUST have the matching id.
+ var instructionID: String = String()
+
+ /// (Required) A request that the SDK Harness needs to interpret.
+ var request: Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest.OneOf_Request? = nil
+
+ var processBundle: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest {
+ get {
+ if case .processBundle(let v)? = request {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest()
+ }
+ set {request = .processBundle(newValue)}
+ }
+
+ var processBundleProgress: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressRequest {
+ get {
+ if case .processBundleProgress(let v)? = request {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressRequest()
+ }
+ set {request = .processBundleProgress(newValue)}
+ }
+
+ var processBundleSplit: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest {
+ get {
+ if case .processBundleSplit(let v)? = request {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest()
+ }
+ set {request = .processBundleSplit(newValue)}
+ }
+
+ var finalizeBundle: Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleRequest {
+ get {
+ if case .finalizeBundle(let v)? = request {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleRequest()
+ }
+ set {request = .finalizeBundle(newValue)}
+ }
+
+ var monitoringInfos: Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataRequest {
+ get {
+ if case .monitoringInfos(let v)? = request {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataRequest()
+ }
+ set {request = .monitoringInfos(newValue)}
+ }
+
+ var harnessMonitoringInfos: Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosRequest {
+ get {
+ if case .harnessMonitoringInfos(let v)? = request {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosRequest()
+ }
+ set {request = .harnessMonitoringInfos(newValue)}
+ }
+
+ var sampleData: Org_Apache_Beam_Model_FnExecution_V1_SampleDataRequest {
+ get {
+ if case .sampleData(let v)? = request {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_SampleDataRequest()
+ }
+ set {request = .sampleData(newValue)}
+ }
+
+ /// DEPRECATED
+ var register: Org_Apache_Beam_Model_FnExecution_V1_RegisterRequest {
+ get {
+ if case .register(let v)? = request {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_RegisterRequest()
+ }
+ set {request = .register(newValue)}
+ }
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ /// (Required) A request that the SDK Harness needs to interpret.
+ enum OneOf_Request: Equatable {
+ case processBundle(Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest)
+ case processBundleProgress(Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressRequest)
+ case processBundleSplit(Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest)
+ case finalizeBundle(Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleRequest)
+ case monitoringInfos(Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataRequest)
+ case harnessMonitoringInfos(Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosRequest)
+ case sampleData(Org_Apache_Beam_Model_FnExecution_V1_SampleDataRequest)
+ /// DEPRECATED
+ case register(Org_Apache_Beam_Model_FnExecution_V1_RegisterRequest)
+
+ #if !swift(>=4.1)
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest.OneOf_Request, rhs: Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest.OneOf_Request) -> Bool {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch (lhs, rhs) {
+ case (.processBundle, .processBundle): return {
+ guard case .processBundle(let l) = lhs, case .processBundle(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.processBundleProgress, .processBundleProgress): return {
+ guard case .processBundleProgress(let l) = lhs, case .processBundleProgress(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.processBundleSplit, .processBundleSplit): return {
+ guard case .processBundleSplit(let l) = lhs, case .processBundleSplit(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.finalizeBundle, .finalizeBundle): return {
+ guard case .finalizeBundle(let l) = lhs, case .finalizeBundle(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.monitoringInfos, .monitoringInfos): return {
+ guard case .monitoringInfos(let l) = lhs, case .monitoringInfos(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.harnessMonitoringInfos, .harnessMonitoringInfos): return {
+ guard case .harnessMonitoringInfos(let l) = lhs, case .harnessMonitoringInfos(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.sampleData, .sampleData): return {
+ guard case .sampleData(let l) = lhs, case .sampleData(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.register, .register): return {
+ guard case .register(let l) = lhs, case .register(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ default: return false
+ }
+ }
+ #endif
+ }
+
+ init() {}
+}
+
+/// The response for an associated request the SDK had been asked to fulfill.
+/// Stable
+struct Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) A reference provided by the runner which represents a requests
+ /// execution. The InstructionResponse MUST have the matching id when
+ /// responding to the runner.
+ var instructionID: String = String()
+
+ /// If this is specified, then this instruction has failed.
+ /// A human readable string representing the reason as to why processing has
+ /// failed.
+ var error: String = String()
+
+ /// If the instruction did not fail, it is required to return an equivalent
+ /// response type depending on the request this matches.
+ var response: Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse.OneOf_Response? = nil
+
+ var processBundle: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleResponse {
+ get {
+ if case .processBundle(let v)? = response {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleResponse()
+ }
+ set {response = .processBundle(newValue)}
+ }
+
+ var processBundleProgress: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressResponse {
+ get {
+ if case .processBundleProgress(let v)? = response {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressResponse()
+ }
+ set {response = .processBundleProgress(newValue)}
+ }
+
+ var processBundleSplit: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse {
+ get {
+ if case .processBundleSplit(let v)? = response {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse()
+ }
+ set {response = .processBundleSplit(newValue)}
+ }
+
+ var finalizeBundle: Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleResponse {
+ get {
+ if case .finalizeBundle(let v)? = response {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleResponse()
+ }
+ set {response = .finalizeBundle(newValue)}
+ }
+
+ var monitoringInfos: Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataResponse {
+ get {
+ if case .monitoringInfos(let v)? = response {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataResponse()
+ }
+ set {response = .monitoringInfos(newValue)}
+ }
+
+ var harnessMonitoringInfos: Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosResponse {
+ get {
+ if case .harnessMonitoringInfos(let v)? = response {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosResponse()
+ }
+ set {response = .harnessMonitoringInfos(newValue)}
+ }
+
+ var sampleData: Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse {
+ get {
+ if case .sampleData(let v)? = response {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse()
+ }
+ set {response = .sampleData(newValue)}
+ }
+
+ /// DEPRECATED
+ var register: Org_Apache_Beam_Model_FnExecution_V1_RegisterResponse {
+ get {
+ if case .register(let v)? = response {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_RegisterResponse()
+ }
+ set {response = .register(newValue)}
+ }
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ /// If the instruction did not fail, it is required to return an equivalent
+ /// response type depending on the request this matches.
+ enum OneOf_Response: Equatable {
+ case processBundle(Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleResponse)
+ case processBundleProgress(Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressResponse)
+ case processBundleSplit(Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse)
+ case finalizeBundle(Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleResponse)
+ case monitoringInfos(Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataResponse)
+ case harnessMonitoringInfos(Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosResponse)
+ case sampleData(Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse)
+ /// DEPRECATED
+ case register(Org_Apache_Beam_Model_FnExecution_V1_RegisterResponse)
+
+ #if !swift(>=4.1)
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse.OneOf_Response, rhs: Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse.OneOf_Response) -> Bool {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch (lhs, rhs) {
+ case (.processBundle, .processBundle): return {
+ guard case .processBundle(let l) = lhs, case .processBundle(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.processBundleProgress, .processBundleProgress): return {
+ guard case .processBundleProgress(let l) = lhs, case .processBundleProgress(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.processBundleSplit, .processBundleSplit): return {
+ guard case .processBundleSplit(let l) = lhs, case .processBundleSplit(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.finalizeBundle, .finalizeBundle): return {
+ guard case .finalizeBundle(let l) = lhs, case .finalizeBundle(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.monitoringInfos, .monitoringInfos): return {
+ guard case .monitoringInfos(let l) = lhs, case .monitoringInfos(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.harnessMonitoringInfos, .harnessMonitoringInfos): return {
+ guard case .harnessMonitoringInfos(let l) = lhs, case .harnessMonitoringInfos(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.sampleData, .sampleData): return {
+ guard case .sampleData(let l) = lhs, case .sampleData(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.register, .register): return {
+ guard case .register(let l) = lhs, case .register(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ default: return false
+ }
+ }
+ #endif
+ }
+
+ init() {}
+}
+
+/// If supported, the `SampleDataRequest` will respond with a
+/// `SampleDataResponse`. The SDK being queried must have the
+/// "beam:protocol:data_sampling:v1" capability. Samples are taken only from the
+/// specified PCollection ids. An empty list will return everything.
+struct Org_Apache_Beam_Model_FnExecution_V1_SampleDataRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Optional) The PCollection ids to filter for.
+ var pcollectionIds: [String] = []
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// An element sampled when the SDK is processing a bundle. This is a proto
+/// message to allow for additional per-element metadata.
+struct Org_Apache_Beam_Model_FnExecution_V1_SampledElement {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) Sampled raw bytes for an element. This is a
+ /// single encoded element in the nested context.
+ var element: Data = Data()
+
+ /// (Required) Timestamp of when the sample was taken.
+ var sampleTimestamp: SwiftProtobuf.Google_Protobuf_Timestamp {
+ get {return _sampleTimestamp ?? SwiftProtobuf.Google_Protobuf_Timestamp()}
+ set {_sampleTimestamp = newValue}
+ }
+ /// Returns true if `sampleTimestamp` has been explicitly set.
+ var hasSampleTimestamp: Bool {return self._sampleTimestamp != nil}
+ /// Clears the value of `sampleTimestamp`. Subsequent reads from it will return its default value.
+ mutating func clearSampleTimestamp() {self._sampleTimestamp = nil}
+
+ /// (Optional) This will be set if this element was sampled because of a user
+ /// exception.
+ var exception: Org_Apache_Beam_Model_FnExecution_V1_SampledElement.Exception {
+ get {return _exception ?? Org_Apache_Beam_Model_FnExecution_V1_SampledElement.Exception()}
+ set {_exception = newValue}
+ }
+ /// Returns true if `exception` has been explicitly set.
+ var hasException: Bool {return self._exception != nil}
+ /// Clears the value of `exception`. Subsequent reads from it will return its default value.
+ mutating func clearException() {self._exception = nil}
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ struct Exception {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The instruction ID of the associated ProcessBundleRequest.
+ var instructionID: String = String()
+
+ /// (Required) The transform ID of the executing PTransform during the
+ /// exception.
+ var transformID: String = String()
+
+ /// (Required) The error message to be displayed to the user. Can use the
+ /// other fields to query for contextual logs.
+ var error: String = String()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ init() {}
+
+ fileprivate var _sampleTimestamp: SwiftProtobuf.Google_Protobuf_Timestamp? = nil
+ fileprivate var _exception: Org_Apache_Beam_Model_FnExecution_V1_SampledElement.Exception? = nil
+}
+
+/// If supported, the `SampleDataResponse` will contain samples from PCollections
+/// based upon the filters specified in the request.
+struct Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// Map from PCollection id to sampled elements.
+ var elementSamples: Dictionary<String,Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse.ElementList> = [:]
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ struct ElementList {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// Required. The individual elements sampled from a PCollection.
+ var elements: [Org_Apache_Beam_Model_FnExecution_V1_SampledElement] = []
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ init() {}
+}
+
+/// A request to provide full MonitoringInfo associated with the entire SDK
+/// harness process, not specific to a bundle.
+///
+/// An SDK can report metrics using an identifier that only contains the
+/// associated payload. A runner who wants to receive the full metrics
+/// information can request all the monitoring metadata via a
+/// MonitoringInfosMetadataRequest providing a list of ids as necessary.
+///
+/// The SDK is allowed to reuse the identifiers
+/// for the lifetime of the associated control connection as long
+/// as the MonitoringInfo could be reconstructed fully by overwriting its
+/// payload field with the bytes specified here.
+struct Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+struct Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// An identifier to MonitoringInfo.payload mapping containing
+ /// Metrics associated with the SDK harness, not a specific bundle.
+ ///
+ /// An SDK can report metrics using an identifier that only contains the
+ /// associated payload. A runner who wants to receive the full metrics
+ /// information can request all the monitoring metadata via a
+ /// MonitoringInfosMetadataRequest providing a list of ids as necessary.
+ ///
+ /// The SDK is allowed to reuse the identifiers
+ /// for the lifetime of the associated control connection as long
+ /// as the MonitoringInfo could be reconstructed fully by overwriting its
+ /// payload field with the bytes specified here.
+ var monitoringData: Dictionary<String,Data> = [:]
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// A list of objects which can be referred to by the runner in
+/// future requests.
+/// Stable
+struct Org_Apache_Beam_Model_FnExecution_V1_RegisterRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Optional) The set of descriptors used to process bundles.
+ var processBundleDescriptor: [Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor] = []
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// Stable
+struct Org_Apache_Beam_Model_FnExecution_V1_RegisterResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// Definitions that should be used to construct the bundle processing graph.
+struct Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) A pipeline level unique id which can be used as a reference to
+ /// refer to this.
+ var id: String = String()
+
+ /// (Required) A map from pipeline-scoped id to PTransform.
+ var transforms: Dictionary<String,Org_Apache_Beam_Model_Pipeline_V1_PTransform> = [:]
+
+ /// (Required) A map from pipeline-scoped id to PCollection.
+ var pcollections: Dictionary<String,Org_Apache_Beam_Model_Pipeline_V1_PCollection> = [:]
+
+ /// (Required) A map from pipeline-scoped id to WindowingStrategy.
+ var windowingStrategies: Dictionary<String,Org_Apache_Beam_Model_Pipeline_V1_WindowingStrategy> = [:]
+
+ /// (Required) A map from pipeline-scoped id to Coder.
+ var coders: Dictionary<String,Org_Apache_Beam_Model_Pipeline_V1_Coder> = [:]
+
+ /// (Required) A map from pipeline-scoped id to Environment.
+ var environments: Dictionary<String,Org_Apache_Beam_Model_Pipeline_V1_Environment> = [:]
+
+ /// A descriptor describing the end point to use for State API
+ /// calls. Required if the Runner intends to send remote references over the
+ /// data plane or if any of the transforms rely on user state or side inputs.
+ var stateApiServiceDescriptor: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor {
+ get {return _stateApiServiceDescriptor ?? Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor()}
+ set {_stateApiServiceDescriptor = newValue}
+ }
+ /// Returns true if `stateApiServiceDescriptor` has been explicitly set.
+ var hasStateApiServiceDescriptor: Bool {return self._stateApiServiceDescriptor != nil}
+ /// Clears the value of `stateApiServiceDescriptor`. Subsequent reads from it will return its default value.
+ mutating func clearStateApiServiceDescriptor() {self._stateApiServiceDescriptor = nil}
+
+ /// A descriptor describing the end point to use for Data API for user timers.
+ /// Required if the ProcessBundleDescriptor contains any transforms that have user timers.
+ var timerApiServiceDescriptor: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor {
+ get {return _timerApiServiceDescriptor ?? Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor()}
+ set {_timerApiServiceDescriptor = newValue}
+ }
+ /// Returns true if `timerApiServiceDescriptor` has been explicitly set.
+ var hasTimerApiServiceDescriptor: Bool {return self._timerApiServiceDescriptor != nil}
+ /// Clears the value of `timerApiServiceDescriptor`. Subsequent reads from it will return its default value.
+ mutating func clearTimerApiServiceDescriptor() {self._timerApiServiceDescriptor = nil}
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+
+ fileprivate var _stateApiServiceDescriptor: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor? = nil
+ fileprivate var _timerApiServiceDescriptor: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor? = nil
+}
+
+/// One of the applications specifying the scope of work for a bundle.
+/// See
+/// https://docs.google.com/document/d/1tUDb45sStdR8u7-jBkGdw3OGFK7aa2-V7eo86zYSE_4/edit#heading=h.9g3g5weg2u9
+/// for further details.
+struct Org_Apache_Beam_Model_FnExecution_V1_BundleApplication {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The transform to which to pass the element
+ var transformID: String = String()
+
+ /// (Required) Name of the transform's input to which to pass the element.
+ var inputID: String = String()
+
+ /// (Required) The encoded element to pass to the transform.
+ var element: Data = Data()
+
+ /// The map is keyed by the local output name of the PTransform. Each
+ /// value represents a lower bound on the timestamps of elements that
+ /// are produced by this PTransform into each of its output PCollections
+ /// when invoked with this application.
+ ///
+ /// If there is no watermark reported from RestrictionTracker, the runner will
+ /// use MIN_TIMESTAMP by default.
+ var outputWatermarks: Dictionary<String,SwiftProtobuf.Google_Protobuf_Timestamp> = [:]
+
+ /// Whether this application potentially produces an unbounded
+ /// amount of data. Note that this should only be set to BOUNDED if and
+ /// only if the application is known to produce a finite amount of output.
+ var isBounded: Org_Apache_Beam_Model_Pipeline_V1_IsBounded.Enum = .unspecified
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// An Application should be scheduled for execution after a delay.
+/// Either an absolute timestamp or a relative timestamp can represent a
+/// scheduled execution time.
+struct Org_Apache_Beam_Model_FnExecution_V1_DelayedBundleApplication {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The application that should be scheduled.
+ var application: Org_Apache_Beam_Model_FnExecution_V1_BundleApplication {
+ get {return _application ?? Org_Apache_Beam_Model_FnExecution_V1_BundleApplication()}
+ set {_application = newValue}
+ }
+ /// Returns true if `application` has been explicitly set.
+ var hasApplication: Bool {return self._application != nil}
+ /// Clears the value of `application`. Subsequent reads from it will return its default value.
+ mutating func clearApplication() {self._application = nil}
+
+ /// Recommended time delay at which the application should be scheduled to
+ /// execute by the runner. Time delay that equals 0 may be scheduled to execute
+ /// immediately. The unit of time delay should be microsecond.
+ var requestedTimeDelay: SwiftProtobuf.Google_Protobuf_Duration {
+ get {return _requestedTimeDelay ?? SwiftProtobuf.Google_Protobuf_Duration()}
+ set {_requestedTimeDelay = newValue}
+ }
+ /// Returns true if `requestedTimeDelay` has been explicitly set.
+ var hasRequestedTimeDelay: Bool {return self._requestedTimeDelay != nil}
+ /// Clears the value of `requestedTimeDelay`. Subsequent reads from it will return its default value.
+ mutating func clearRequestedTimeDelay() {self._requestedTimeDelay = nil}
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+
+ fileprivate var _application: Org_Apache_Beam_Model_FnExecution_V1_BundleApplication? = nil
+ fileprivate var _requestedTimeDelay: SwiftProtobuf.Google_Protobuf_Duration? = nil
+}
+
+/// A request to process a given bundle.
+/// Stable
+struct Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) A reference to the process bundle descriptor that must be
+ /// instantiated and executed by the SDK harness.
+ var processBundleDescriptorID: String = String()
+
+ /// (Optional) A list of cache tokens that can be used by an SDK to reuse
+ /// cached data returned by the State API across multiple bundles.
+ ///
+ /// Note that SDKs that can efficiently consume this field should declare
+ /// the beam:protocol:state_caching:v1 capability enabling runners to reduce
+ /// the amount of memory used.
+ ///
+ /// See https://s.apache.org/beam-fn-state-api-and-bundle-processing#heading=h.7ghoih5aig5m
+ /// for additional details on how to use the cache token with the State API
+ /// to cache data across bundle boundaries.
+ var cacheTokens: [Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken] = []
+
+ /// (Optional) Elements to be processed with the bundle. Either all or
+ /// none of the bundle elements should be included in the ProcessBundleRequest.
+ /// This embedding is to achieve better efficiency for bundles that contain
+ /// only small amounts of data and are cheap to be processed on the SDK harness
+ /// side. This field can be set only if the SDK declares that it supports the
+ /// beam:protocol:control_request_elements_embedding:v1 capability. See more
+ /// at https://s.apache.org/beam-fn-api-control-data-embedding.
+ var elements: Org_Apache_Beam_Model_FnExecution_V1_Elements {
+ get {return _elements ?? Org_Apache_Beam_Model_FnExecution_V1_Elements()}
+ set {_elements = newValue}
+ }
+ /// Returns true if `elements` has been explicitly set.
+ var hasElements: Bool {return self._elements != nil}
+ /// Clears the value of `elements`. Subsequent reads from it will return its default value.
+ mutating func clearElements() {self._elements = nil}
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ /// Contains the cache token and also defines the scope of what the token applies to.
+ ///
+ /// See https://s.apache.org/beam-fn-state-api-and-bundle-processing#heading=h.7ghoih5aig5m
+ /// for additional details on how to use the cache token with the State API
+ /// to cache data across bundle boundaries.
+ struct CacheToken {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// The scope of a cache token.
+ var type: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.OneOf_Type? = nil
+
+ var userState: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.UserState {
+ get {
+ if case .userState(let v)? = type {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.UserState()
+ }
+ set {type = .userState(newValue)}
+ }
+
+ var sideInput: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.SideInput {
+ get {
+ if case .sideInput(let v)? = type {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.SideInput()
+ }
+ set {type = .sideInput(newValue)}
+ }
+
+ /// An opaque token used with the StateKey to create a globally unique
+ /// identifier.
+ var token: Data = Data()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ /// The scope of a cache token.
+ enum OneOf_Type: Equatable {
+ case userState(Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.UserState)
+ case sideInput(Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.SideInput)
+
+ #if !swift(>=4.1)
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.OneOf_Type, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.OneOf_Type) -> Bool {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch (lhs, rhs) {
+ case (.userState, .userState): return {
+ guard case .userState(let l) = lhs, case .userState(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.sideInput, .sideInput): return {
+ guard case .sideInput(let l) = lhs, case .sideInput(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ default: return false
+ }
+ }
+ #endif
+ }
+
+ /// A flag to indicate a cache token is valid for all user state.
+ struct UserState {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ /// A flag to indicate a cache token is valid for a side input.
+ struct SideInput {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The id of the PTransform containing a side input.
+ var transformID: String = String()
+
+ /// (Required) The id of the side input.
+ var sideInputID: String = String()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ init() {}
+ }
+
+ init() {}
+
+ fileprivate var _elements: Org_Apache_Beam_Model_FnExecution_V1_Elements? = nil
+}
+
+struct Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Optional) Specifies that the bundle has not been completed and the
+ /// following applications need to be scheduled and executed in the future.
+ /// A runner that does not yet support residual roots MUST still check that
+ /// this is empty for correctness.
+ ///
+ /// Note that these residual roots must not have been returned as part of a
+ /// prior split for this bundle.
+ var residualRoots: [Org_Apache_Beam_Model_FnExecution_V1_DelayedBundleApplication] = []
+
+ /// DEPRECATED (Required) The list of metrics or other MonitoredState
+ /// collected while processing this bundle.
+ var monitoringInfos: [Org_Apache_Beam_Model_Pipeline_V1_MonitoringInfo] = []
+
+ /// (Optional) Specifies that the runner must callback to this worker
+ /// once the output of the bundle is committed. The Runner must send a
+ /// FinalizeBundleRequest with the instruction id of the ProcessBundleRequest
+ /// that is related to this ProcessBundleResponse.
+ var requiresFinalization: Bool = false
+
+ /// An identifier to MonitoringInfo.payload mapping.
+ ///
+ /// An SDK can report metrics using an identifier that only contains the
+ /// associated payload. A runner who wants to receive the full metrics
+ /// information can request all the monitoring metadata via a
+ /// MonitoringInfosMetadataRequest providing a list of ids as necessary.
+ ///
+ /// The SDK is allowed to reuse the identifiers across multiple bundles as long
+ /// as the MonitoringInfo could be reconstructed fully by overwriting its
+ /// payload field with the bytes specified here.
+ var monitoringData: Dictionary<String,Data> = [:]
+
+ /// (Optional) Output elements of the processed bundle. Either all or
+ /// none of the bundle elements should be included in the ProcessBundleResponse.
+ /// This embedding is to achieve better efficiency for bundles that only
+ /// contain small amounts of data. his field can be set only if the runner
+ /// declares that it supports the
+ /// beam:protocol:control_request_elements_embedding:v1 capability. See more at
+ /// https://s.apache.org/beam-fn-api-control-data-embedding.
+ var elements: Org_Apache_Beam_Model_FnExecution_V1_Elements {
+ get {return _elements ?? Org_Apache_Beam_Model_FnExecution_V1_Elements()}
+ set {_elements = newValue}
+ }
+ /// Returns true if `elements` has been explicitly set.
+ var hasElements: Bool {return self._elements != nil}
+ /// Clears the value of `elements`. Subsequent reads from it will return its default value.
+ mutating func clearElements() {self._elements = nil}
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+
+ fileprivate var _elements: Org_Apache_Beam_Model_FnExecution_V1_Elements? = nil
+}
+
+/// A request to report progress information for a given bundle.
+/// This is an optional request to be handled and is used to support advanced
+/// SDK features such as SplittableDoFn, user level metrics etc.
+struct Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) A reference to an active process bundle request with the given
+ /// instruction id.
+ var instructionID: String = String()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// A request to provide full MonitoringInfo for a set of provided ids.
+///
+/// An SDK can report metrics using an identifier that only contains the
+/// associated payload. A runner who wants to receive the full metrics
+/// information can request all the monitoring metadata via a
+/// MonitoringInfosMetadataRequest providing a list of ids as necessary.
+///
+/// The SDK is allowed to reuse the identifiers for the lifetime of the
+/// associated control connection as long as the MonitoringInfo could be
+/// reconstructed fully by overwriting its payload field with the bytes specified
+/// here.
+struct Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// A list of ids for which the full MonitoringInfo is requested for.
+ var monitoringInfoID: [String] = []
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+struct Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// DEPRECATED (Required) The list of metrics or other MonitoredState
+ /// collected while processing this bundle.
+ var monitoringInfos: [Org_Apache_Beam_Model_Pipeline_V1_MonitoringInfo] = []
+
+ /// An identifier to MonitoringInfo.payload mapping.
+ ///
+ /// An SDK can report metrics using an identifier that only contains the
+ /// associated payload. A runner who wants to receive the full metrics
+ /// information can request all the monitoring metadata via a
+ /// MonitoringInfosMetadataRequest providing a list of ids as necessary.
+ ///
+ /// The SDK is allowed to reuse the identifiers
+ /// for the lifetime of the associated control connection as long
+ /// as the MonitoringInfo could be reconstructed fully by overwriting its
+ /// payload field with the bytes specified here.
+ var monitoringData: Dictionary<String,Data> = [:]
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// A response that contains the full mapping information associated with
+/// a specified set of identifiers.
+///
+/// An SDK can report metrics using an identifier that only contains the
+/// associated payload. A runner who wants to receive the full metrics
+/// information can request all the monitoring metadata via a
+/// MonitoringInfosMetadataRequest providing a list of ids as necessary.
+///
+/// The SDK is allowed to reuse the identifiers
+/// for the lifetime of the associated control connection as long
+/// as the MonitoringInfo could be reconstructed fully by overwriting its
+/// payload field with the bytes specified here.
+struct Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// A mapping from an identifier to the full metrics information.
+ var monitoringInfo: Dictionary<String,Org_Apache_Beam_Model_Pipeline_V1_MonitoringInfo> = [:]
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// Represents a request to the SDK to split a currently active bundle.
+struct Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) A reference to an active process bundle request with the given
+ /// instruction id.
+ var instructionID: String = String()
+
+ /// (Required) Specifies the desired split for each transform.
+ ///
+ /// Currently only splits at gRPC read operations are supported.
+ /// This may, of course, limit the amount of work downstream operations
+ /// receive.
+ var desiredSplits: Dictionary<String,Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest.DesiredSplit> = [:]
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ /// A message specifying the desired split for a single transform.
+ struct DesiredSplit {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The fraction of known work remaining in this bundle
+ /// for this transform that should be kept by the SDK after this split.
+ ///
+ /// Set to 0 to "checkpoint" as soon as possible (keeping as little work as
+ /// possible and returning the remainder).
+ var fractionOfRemainder: Double = 0
+
+ /// (Optional) A set of allowed element indices where the SDK may split. When
+ /// this is empty, there are no constraints on where to split.
+ var allowedSplitPoints: [Int64] = []
+
+ /// (Required for gRPC Read operation transforms) Number of total elements
+ /// expected to be sent to this GrpcRead operation, required to correctly
+ /// account for unreceived data when determining where to split.
+ var estimatedInputElements: Int64 = 0
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ init() {}
+}
+
+/// Represents a partition of the bundle: a "primary" and a "residual", with the
+/// following properties:
+/// - The work in primary and residual doesn't overlap, and combined, adds up
+/// to the work in the current bundle if the split hadn't happened.
+/// - The current bundle, if it keeps executing, will have done exactly none of
+/// the work under residual_roots and none of the elements at and beyond the
+/// first_residual_element.
+/// - The current bundle, if no further splits happen, will have done exactly
+/// the work under primary_roots and all elements up to and including the
+/// channel split's last_primary_element.
+///
+/// This allows the SDK to relinquish ownership of and commit to not process some
+/// of the elements that it may have been sent (the residual) while retaining
+/// ownership and commitment to finish the other portion (the primary).
+///
+/// Example with three splits of a single bundle:
+/// Let's say the SDK is processing elements [A B C D E]. These elements make
+/// up the 0-indexed channel.
+///
+/// ** First Split **
+/// Channel Split = [ A B C D <> E ]
+/// Primary Roots = [] (No elements were split)
+/// Residual Roots = []
+///
+/// Say a split request comes in. The SDK could return a response with a channel
+/// split representing a last_primary_element of 3 (D) and
+/// first_residual_element of 4 (E). The SDK is now responsible for processing A
+/// B C D and the runner must process E in the future.
+///
+/// (A B C D) | (E)
+///
+/// ** Second Split **
+/// Channel Split = [ A < B C > D E ]
+/// Primary Roots = [B1 C1]
+/// Residual Roots = [B2 C2]
+///
+/// A future split request could have the SDK split the elements B into B1 and
+/// B2 and C into C1 and C2 representing their primary and residual roots. The
+///
+/// (A B1 C1) | (B2 C2 D)
+///
+/// SDK would return a response with a channel split representing a
+/// last_primary_element of 0 (A) and first_residual_element of 3 (D) with
+/// primary_roots (B1, C1) and residual_roots (B2, C2). The SDK is now
+/// responsible for processing A B1 C1 and the runner must process B2 C2 D (and
+/// E from the prior split) in the future.
+///
+/// ** Third Split **
+/// Channel Split = [ A < B C > D E ]
+/// Primary Roots = [B1a]
+/// Residual Roots [B1b C1]
+/// Yet another future split request could have the SDK could split B1 further
+/// into B1a and B1b primary and residuals and return C1 as a residual (assuming
+/// C1 was left unprocessed). The SDK would return a response with a channel
+/// split representing a last_primary_element of 0 (A) and
+/// first_residual_element of 3 (E) with primary_roots (B1a) and residual_roots
+/// (B1b, C1). The SDK is now responsible for processing A B1a the runner must
+/// process B1b C1 (in addition to C2, D, E from prior splits) in the future.
+///
+/// (A B1a) | (B1b C1)
+///
+/// For more rigorous definitions see https://s.apache.org/beam-breaking-fusion
+struct Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Optional) Root applications that should replace the current bundle.
+ ///
+ /// Note that primary roots can only be specified if a channel split's
+ /// last_primary_element + 1 < first_residual_element
+ ///
+ /// Note that there must be a corresponding residual root contained within
+ /// residual_roots representing the remainder of processing for the original
+ /// element this this primary root represents a fraction of.
+ var primaryRoots: [Org_Apache_Beam_Model_FnExecution_V1_BundleApplication] = []
+
+ /// (Optional) Root applications that have been removed from the current bundle and
+ /// have to be executed in a separate bundle (e.g. in parallel on a different
+ /// worker, or after the current bundle completes, etc.)
+ ///
+ /// Note that residual roots can only be specified if a channel split's
+ /// last_primary_element + 1 < first_residual_element
+ ///
+ /// Note that there must be a corresponding primary root contained within
+ /// primary_roots representing the remainder of processing for the original
+ /// element this this residual root represents a fraction of.
+ ///
+ /// Note that subsequent splits must not return prior residual roots.
+ var residualRoots: [Org_Apache_Beam_Model_FnExecution_V1_DelayedBundleApplication] = []
+
+ /// (Required) Partitions of input data channels into primary and residual
+ /// elements, if any. Must not include any elements represented in the bundle
+ /// applications roots above of the current split or any prior split of the
+ /// same bundle.
+ var channelSplits: [Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse.ChannelSplit] = []
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ /// Represents contiguous portions of the data channel that are either
+ /// entirely processed or entirely unprocessed and belong to the primary
+ /// or residual respectively.
+ ///
+ /// This affords both a more efficient representation over the FnAPI
+ /// (if the bundle is large) and often a more efficient representation
+ /// on the runner side (e.g. if the set of elements can be represented
+ /// as some range in an underlying dataset).
+ ///
+ /// Note that for a split the following properties must hold:
+ /// - last_primary_element < first_residual_element
+ /// - primary roots and residual roots can only be specified if the
+ /// last_primary_element + 1 < first_residual_element
+ /// (typically there is one primary and residual root per element in the
+ /// range (last_primary_element, first_residual_element))
+ /// - primary roots and residual roots must represent a disjoint but full
+ /// coverage of work represented by the elements between last_primary_element
+ /// and first_residual_element
+ ///
+ /// Note that subsequent splits of the same bundle must ensure that:
+ /// - the first_residual_element does not increase
+ /// - the first_residual_element does not decrease if there were residual
+ /// or primary roots returned in a prior split.
+ struct ChannelSplit {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The grpc read transform reading this channel.
+ var transformID: String = String()
+
+ /// (Required) The last element of the input channel that should be entirely
+ /// considered part of the primary, identified by its absolute zero-based
+ /// index in the (ordered) channel.
+ var lastPrimaryElement: Int64 = 0
+
+ /// (Required) The first element of the input channel that should be entirely
+ /// considered part of the residual, identified by its absolute zero-based
+ /// index in the (ordered) channel.
+ var firstResidualElement: Int64 = 0
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ init() {}
+}
+
+struct Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) A reference to a completed process bundle request with the given
+ /// instruction id.
+ var instructionID: String = String()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// Empty
+struct Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// Messages used to represent logical byte streams.
+/// Stable
+struct Org_Apache_Beam_Model_FnExecution_V1_Elements {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Optional) A list containing parts of logical byte streams.
+ var data: [Org_Apache_Beam_Model_FnExecution_V1_Elements.DataMessage] = []
+
+ /// (Optional) A list of timer byte streams.
+ var timers: [Org_Apache_Beam_Model_FnExecution_V1_Elements.Timers] = []
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ /// Represents multiple encoded elements in nested context for a given named
+ /// instruction and transform.
+ struct DataMessage {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) A reference to an active instruction request with the given
+ /// instruction id.
+ var instructionID: String = String()
+
+ /// (Required) A definition representing a consumer or producer of this data.
+ /// If received by a harness, this represents the consumer within that
+ /// harness that should consume these bytes. If sent by a harness, this
+ /// represents the producer of these bytes.
+ ///
+ /// Note that a single element may span multiple Data messages.
+ ///
+ /// Note that a sending/receiving pair should share the same identifier.
+ var transformID: String = String()
+
+ /// (Optional) Represents a part of a logical byte stream. Elements within
+ /// the logical byte stream are encoded in the nested context and
+ /// concatenated together.
+ var data: Data = Data()
+
+ /// (Optional) Set this bit to indicate the this is the last data block
+ /// for the given instruction and transform, ending the stream.
+ var isLast: Bool = false
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ /// Represent the encoded user timer for a given instruction, transform and
+ /// timer id.
+ struct Timers {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) A reference to an active instruction request with the given
+ /// instruction id.
+ var instructionID: String = String()
+
+ /// (Required) A definition representing a consumer or producer of this data.
+ /// If received by a harness, this represents the consumer within that
+ /// harness that should consume these timers. If sent by a harness, this
+ /// represents the producer of these timers.
+ var transformID: String = String()
+
+ /// (Required) The local timer family name used to identify the associated
+ /// timer family specification
+ var timerFamilyID: String = String()
+
+ /// (Optional) Represents a logical byte stream of timers. Encoded according
+ /// to the coder in the timer spec.
+ var timers: Data = Data()
+
+ /// (Optional) Set this bit to indicate the this is the last data block
+ /// for the given instruction and transform, ending the stream.
+ var isLast: Bool = false
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ init() {}
+}
+
+struct Org_Apache_Beam_Model_FnExecution_V1_StateRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) A unique identifier provided by the SDK which represents this
+ /// requests execution. The StateResponse corresponding with this request
+ /// will have the matching id.
+ var id: String = String()
+
+ /// (Required) The associated instruction id of the work that is currently
+ /// being processed. This allows for the runner to associate any modifications
+ /// to state to be committed with the appropriate work execution.
+ var instructionID: String = String()
+
+ /// (Required) The state key this request is for.
+ var stateKey: Org_Apache_Beam_Model_FnExecution_V1_StateKey {
+ get {return _stateKey ?? Org_Apache_Beam_Model_FnExecution_V1_StateKey()}
+ set {_stateKey = newValue}
+ }
+ /// Returns true if `stateKey` has been explicitly set.
+ var hasStateKey: Bool {return self._stateKey != nil}
+ /// Clears the value of `stateKey`. Subsequent reads from it will return its default value.
+ mutating func clearStateKey() {self._stateKey = nil}
+
+ /// (Required) The action to take on this request.
+ var request: Org_Apache_Beam_Model_FnExecution_V1_StateRequest.OneOf_Request? = nil
+
+ /// A request to get state.
+ var get: Org_Apache_Beam_Model_FnExecution_V1_StateGetRequest {
+ get {
+ if case .get(let v)? = request {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateGetRequest()
+ }
+ set {request = .get(newValue)}
+ }
+
+ /// A request to append to state.
+ var append: Org_Apache_Beam_Model_FnExecution_V1_StateAppendRequest {
+ get {
+ if case .append(let v)? = request {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateAppendRequest()
+ }
+ set {request = .append(newValue)}
+ }
+
+ /// A request to clear state.
+ var clear: Org_Apache_Beam_Model_FnExecution_V1_StateClearRequest {
+ get {
+ if case .clear(let v)? = request {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateClearRequest()
+ }
+ set {request = .clear(newValue)}
+ }
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ /// (Required) The action to take on this request.
+ enum OneOf_Request: Equatable {
+ /// A request to get state.
+ case get(Org_Apache_Beam_Model_FnExecution_V1_StateGetRequest)
+ /// A request to append to state.
+ case append(Org_Apache_Beam_Model_FnExecution_V1_StateAppendRequest)
+ /// A request to clear state.
+ case clear(Org_Apache_Beam_Model_FnExecution_V1_StateClearRequest)
+
+ #if !swift(>=4.1)
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_StateRequest.OneOf_Request, rhs: Org_Apache_Beam_Model_FnExecution_V1_StateRequest.OneOf_Request) -> Bool {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch (lhs, rhs) {
+ case (.get, .get): return {
+ guard case .get(let l) = lhs, case .get(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.append, .append): return {
+ guard case .append(let l) = lhs, case .append(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.clear, .clear): return {
+ guard case .clear(let l) = lhs, case .clear(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ default: return false
+ }
+ }
+ #endif
+ }
+
+ init() {}
+
+ fileprivate var _stateKey: Org_Apache_Beam_Model_FnExecution_V1_StateKey? = nil
+}
+
+struct Org_Apache_Beam_Model_FnExecution_V1_StateResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) A reference provided by the SDK which represents a requests
+ /// execution. The StateResponse must have the matching id when responding
+ /// to the SDK.
+ var id: String = String()
+
+ /// (Optional) If this is specified, then the state request has failed.
+ /// A human readable string representing the reason as to why the request
+ /// failed.
+ var error: String = String()
+
+ /// A corresponding response matching the request will be populated.
+ var response: Org_Apache_Beam_Model_FnExecution_V1_StateResponse.OneOf_Response? = nil
+
+ /// A response to getting state.
+ var get: Org_Apache_Beam_Model_FnExecution_V1_StateGetResponse {
+ get {
+ if case .get(let v)? = response {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateGetResponse()
+ }
+ set {response = .get(newValue)}
+ }
+
+ /// A response to appending to state.
+ var append: Org_Apache_Beam_Model_FnExecution_V1_StateAppendResponse {
+ get {
+ if case .append(let v)? = response {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateAppendResponse()
+ }
+ set {response = .append(newValue)}
+ }
+
+ /// A response to clearing state.
+ var clear: Org_Apache_Beam_Model_FnExecution_V1_StateClearResponse {
+ get {
+ if case .clear(let v)? = response {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateClearResponse()
+ }
+ set {response = .clear(newValue)}
+ }
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ /// A corresponding response matching the request will be populated.
+ enum OneOf_Response: Equatable {
+ /// A response to getting state.
+ case get(Org_Apache_Beam_Model_FnExecution_V1_StateGetResponse)
+ /// A response to appending to state.
+ case append(Org_Apache_Beam_Model_FnExecution_V1_StateAppendResponse)
+ /// A response to clearing state.
+ case clear(Org_Apache_Beam_Model_FnExecution_V1_StateClearResponse)
+
+ #if !swift(>=4.1)
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_StateResponse.OneOf_Response, rhs: Org_Apache_Beam_Model_FnExecution_V1_StateResponse.OneOf_Response) -> Bool {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch (lhs, rhs) {
+ case (.get, .get): return {
+ guard case .get(let l) = lhs, case .get(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.append, .append): return {
+ guard case .append(let l) = lhs, case .append(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.clear, .clear): return {
+ guard case .clear(let l) = lhs, case .clear(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ default: return false
+ }
+ }
+ #endif
+ }
+
+ init() {}
+}
+
+struct Org_Apache_Beam_Model_FnExecution_V1_StateKey {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) One of the following state keys must be set.
+ var type: Org_Apache_Beam_Model_FnExecution_V1_StateKey.OneOf_Type? = nil
+
+ var runner: Org_Apache_Beam_Model_FnExecution_V1_StateKey.Runner {
+ get {
+ if case .runner(let v)? = type {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateKey.Runner()
+ }
+ set {type = .runner(newValue)}
+ }
+
+ var multimapSideInput: Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapSideInput {
+ get {
+ if case .multimapSideInput(let v)? = type {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapSideInput()
+ }
+ set {type = .multimapSideInput(newValue)}
+ }
+
+ var bagUserState: Org_Apache_Beam_Model_FnExecution_V1_StateKey.BagUserState {
+ get {
+ if case .bagUserState(let v)? = type {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateKey.BagUserState()
+ }
+ set {type = .bagUserState(newValue)}
+ }
+
+ var iterableSideInput: Org_Apache_Beam_Model_FnExecution_V1_StateKey.IterableSideInput {
+ get {
+ if case .iterableSideInput(let v)? = type {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateKey.IterableSideInput()
+ }
+ set {type = .iterableSideInput(newValue)}
+ }
+
+ var multimapKeysSideInput: Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapKeysSideInput {
+ get {
+ if case .multimapKeysSideInput(let v)? = type {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapKeysSideInput()
+ }
+ set {type = .multimapKeysSideInput(newValue)}
+ }
+
+ var multimapKeysUserState: Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapKeysUserState {
+ get {
+ if case .multimapKeysUserState(let v)? = type {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapKeysUserState()
+ }
+ set {type = .multimapKeysUserState(newValue)}
+ }
+
+ var multimapUserState: Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapUserState {
+ get {
+ if case .multimapUserState(let v)? = type {return v}
+ return Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapUserState()
+ }
+ set {type = .multimapUserState(newValue)}
+ }
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ /// (Required) One of the following state keys must be set.
+ enum OneOf_Type: Equatable {
+ case runner(Org_Apache_Beam_Model_FnExecution_V1_StateKey.Runner)
+ case multimapSideInput(Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapSideInput)
+ case bagUserState(Org_Apache_Beam_Model_FnExecution_V1_StateKey.BagUserState)
+ case iterableSideInput(Org_Apache_Beam_Model_FnExecution_V1_StateKey.IterableSideInput)
+ case multimapKeysSideInput(Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapKeysSideInput)
+ case multimapKeysUserState(Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapKeysUserState)
+ case multimapUserState(Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapUserState)
+
+ #if !swift(>=4.1)
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_StateKey.OneOf_Type, rhs: Org_Apache_Beam_Model_FnExecution_V1_StateKey.OneOf_Type) -> Bool {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch (lhs, rhs) {
+ case (.runner, .runner): return {
+ guard case .runner(let l) = lhs, case .runner(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.multimapSideInput, .multimapSideInput): return {
+ guard case .multimapSideInput(let l) = lhs, case .multimapSideInput(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.bagUserState, .bagUserState): return {
+ guard case .bagUserState(let l) = lhs, case .bagUserState(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.iterableSideInput, .iterableSideInput): return {
+ guard case .iterableSideInput(let l) = lhs, case .iterableSideInput(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.multimapKeysSideInput, .multimapKeysSideInput): return {
+ guard case .multimapKeysSideInput(let l) = lhs, case .multimapKeysSideInput(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.multimapKeysUserState, .multimapKeysUserState): return {
+ guard case .multimapKeysUserState(let l) = lhs, case .multimapKeysUserState(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ case (.multimapUserState, .multimapUserState): return {
+ guard case .multimapUserState(let l) = lhs, case .multimapUserState(let r) = rhs else { preconditionFailure() }
+ return l == r
+ }()
+ default: return false
+ }
+ }
+ #endif
+ }
+
+ struct Runner {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) Opaque information supplied by the runner. Used to support
+ /// remote references.
+ /// https://s.apache.org/beam-fn-api-send-and-receive-data
+ ///
+ /// Used by state backed iterable. And in this use case, request type can
+ /// only be of type get. Details see:
+ /// https://s.apache.org/beam-fn-api-state-backed-iterables
+ var key: Data = Data()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ /// Represents a request for the values associated with a specified window
+ /// in a PCollection. See
+ /// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+ /// details.
+ ///
+ /// Can only be used to perform StateGetRequests on side inputs of the URN
+ /// beam:side_input:iterable:v1.
+ ///
+ /// For a PCollection<V>, the response data stream will be a concatenation
+ /// of all V's. See https://s.apache.org/beam-fn-api-send-and-receive-data
+ /// for further details.
+ struct IterableSideInput {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The id of the PTransform containing a side input.
+ var transformID: String = String()
+
+ /// (Required) The id of the side input.
+ var sideInputID: String = String()
+
+ /// (Required) The window (after mapping the currently executing elements
+ /// window into the side input windows domain) encoded in a nested context.
+ var window: Data = Data()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ /// Represents a request for the values associated with a specified user key
+ /// and window in a PCollection. See
+ /// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+ /// details.
+ ///
+ /// Can only be used to perform StateGetRequests on side inputs of the URN
+ /// beam:side_input:multimap:v1.
+ ///
+ /// For a PCollection<KV<K, V>>, the response data stream will be a
+ /// concatenation of all V's associated with the specified key K. See
+ /// https://s.apache.org/beam-fn-api-send-and-receive-data for further
+ /// details.
+ struct MultimapSideInput {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The id of the PTransform containing a side input.
+ var transformID: String = String()
+
+ /// (Required) The id of the side input.
+ var sideInputID: String = String()
+
+ /// (Required) The window (after mapping the currently executing elements
+ /// window into the side input windows domain) encoded in a nested context.
+ var window: Data = Data()
+
+ /// (Required) The key encoded in a nested context.
+ var key: Data = Data()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ /// Represents a request for the keys associated with a specified window in a PCollection. See
+ /// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+ /// details.
+ ///
+ /// Can only be used to perform StateGetRequests on side inputs of the URN
+ /// beam:side_input:multimap:v1.
+ ///
+ /// For a PCollection<KV<K, V>>, the response data stream will be a
+ /// concatenation of all K's associated with the specified window. See
+ /// https://s.apache.org/beam-fn-api-send-and-receive-data for further
+ /// details.
+ struct MultimapKeysSideInput {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The id of the PTransform containing a side input.
+ var transformID: String = String()
+
+ /// (Required) The id of the side input.
+ var sideInputID: String = String()
+
+ /// (Required) The window (after mapping the currently executing elements
+ /// window into the side input windows domain) encoded in a nested context.
+ var window: Data = Data()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ /// Represents a request for an unordered set of values associated with a
+ /// specified user key and window for a PTransform. See
+ /// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+ /// details.
+ ///
+ /// The response data stream will be a concatenation of all V's associated
+ /// with the specified user key and window.
+ /// See https://s.apache.org/beam-fn-api-send-and-receive-data for further
+ /// details.
+ struct BagUserState {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The id of the PTransform containing user state.
+ var transformID: String = String()
+
+ /// (Required) The id of the user state.
+ var userStateID: String = String()
+
+ /// (Required) The window encoded in a nested context.
+ var window: Data = Data()
+
+ /// (Required) The key of the currently executing element encoded in a
+ /// nested context.
+ var key: Data = Data()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ /// Represents a request for the keys of a multimap associated with a specified
+ /// user key and window for a PTransform. See
+ /// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+ /// details.
+ ///
+ /// Can only be used to perform StateGetRequests and StateClearRequests on the
+ /// user state.
+ ///
+ /// The response data stream will be a concatenation of all K's associated
+ /// with the specified user key and window.
+ /// See https://s.apache.org/beam-fn-api-send-and-receive-data for further
+ /// details.
+ struct MultimapKeysUserState {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The id of the PTransform containing user state.
+ var transformID: String = String()
+
+ /// (Required) The id of the user state.
+ var userStateID: String = String()
+
+ /// (Required) The window encoded in a nested context.
+ var window: Data = Data()
+
+ /// (Required) The key of the currently executing element encoded in a
+ /// nested context.
+ var key: Data = Data()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ /// Represents a request for the values of the map key associated with a
+ /// specified user key and window for a PTransform. See
+ /// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+ /// details.
+ ///
+ /// The response data stream will be a concatenation of all V's associated
+ /// with the specified map key, user key, and window.
+ /// See https://s.apache.org/beam-fn-api-send-and-receive-data for further
+ /// details.
+ struct MultimapUserState {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The id of the PTransform containing user state.
+ var transformID: String = String()
+
+ /// (Required) The id of the user state.
+ var userStateID: String = String()
+
+ /// (Required) The window encoded in a nested context.
+ var window: Data = Data()
+
+ /// (Required) The key of the currently executing element encoded in a
+ /// nested context.
+ var key: Data = Data()
+
+ /// (Required) The map key encoded in a nested context.
+ var mapKey: Data = Data()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ init() {}
+}
+
+/// A request to get state.
+struct Org_Apache_Beam_Model_FnExecution_V1_StateGetRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Optional) If specified, signals to the runner that the response
+ /// should resume from the following continuation token.
+ ///
+ /// If unspecified, signals to the runner that the response should start
+ /// from the beginning of the logical continuable stream.
+ var continuationToken: Data = Data()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// A response to get state representing a logical byte stream which can be
+/// continued using the state API.
+struct Org_Apache_Beam_Model_FnExecution_V1_StateGetResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Optional) If specified, represents a token which can be used with the
+ /// state API to get the next chunk of this logical byte stream. The end of
+ /// the logical byte stream is signalled by this field being unset.
+ var continuationToken: Data = Data()
+
+ /// Represents a part of a logical byte stream. Elements within
+ /// the logical byte stream are encoded in the nested context and
+ /// concatenated together.
+ var data: Data = Data()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// A request to append state.
+struct Org_Apache_Beam_Model_FnExecution_V1_StateAppendRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// Represents a part of a logical byte stream. Elements within
+ /// the logical byte stream are encoded in the nested context and
+ /// multiple append requests are concatenated together.
+ var data: Data = Data()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// A response to append state.
+struct Org_Apache_Beam_Model_FnExecution_V1_StateAppendResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// A request to clear state.
+struct Org_Apache_Beam_Model_FnExecution_V1_StateClearRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// A response to clear state.
+struct Org_Apache_Beam_Model_FnExecution_V1_StateClearResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// A log entry
+struct Org_Apache_Beam_Model_FnExecution_V1_LogEntry {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) The severity of the log statement.
+ var severity: Org_Apache_Beam_Model_FnExecution_V1_LogEntry.Severity.Enum = .unspecified
+
+ /// (Required) The time at which this log statement occurred.
+ var timestamp: SwiftProtobuf.Google_Protobuf_Timestamp {
+ get {return _timestamp ?? SwiftProtobuf.Google_Protobuf_Timestamp()}
+ set {_timestamp = newValue}
+ }
+ /// Returns true if `timestamp` has been explicitly set.
+ var hasTimestamp: Bool {return self._timestamp != nil}
+ /// Clears the value of `timestamp`. Subsequent reads from it will return its default value.
+ mutating func clearTimestamp() {self._timestamp = nil}
+
+ /// (Required) A human readable message.
+ var message: String = String()
+
+ /// (Optional) An optional trace of the functions involved. For example, in
+ /// Java this can include multiple causes and multiple suppressed exceptions.
+ var trace: String = String()
+
+ /// (Optional) A reference to the instruction this log statement is associated
+ /// with.
+ var instructionID: String = String()
+
+ /// (Optional) A reference to the transform this log statement is
+ /// associated with.
+ var transformID: String = String()
+
+ /// (Optional) Human-readable name of the function or method being invoked,
+ /// with optional context such as the class or package name. The format can
+ /// vary by language. For example:
+ /// qual.if.ied.Class.method (Java)
+ /// dir/package.func (Go)
+ /// module.function (Python)
+ /// file.cc:382 (C++)
+ var logLocation: String = String()
+
+ /// (Optional) The name of the thread this log statement is associated with.
+ var thread: String = String()
+
+ /// (Optional) Additional structured data to log.
+ /// Keys are limited to these characters: [a-zA-Z_-]
+ var customData: SwiftProtobuf.Google_Protobuf_Struct {
+ get {return _customData ?? SwiftProtobuf.Google_Protobuf_Struct()}
+ set {_customData = newValue}
+ }
+ /// Returns true if `customData` has been explicitly set.
+ var hasCustomData: Bool {return self._customData != nil}
+ /// Clears the value of `customData`. Subsequent reads from it will return its default value.
+ mutating func clearCustomData() {self._customData = nil}
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ /// A list of log entries, enables buffering and batching of multiple
+ /// log messages using the logging API.
+ struct List {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) One or or more log messages.
+ var logEntries: [Org_Apache_Beam_Model_FnExecution_V1_LogEntry] = []
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+ }
+
+ /// The severity of the event described in a log entry, expressed as one of the
+ /// severity levels listed below. For your reference, the levels are
+ /// assigned the listed numeric values. The effect of using numeric values
+ /// other than those listed is undefined.
+ ///
+ /// If you are writing log entries, you should map other severity encodings to
+ /// one of these standard levels. For example, you might map all of
+ /// Java's FINE, FINER, and FINEST levels to `Severity.DEBUG`.
+ ///
+ /// This list is intentionally not comprehensive; the intent is to provide a
+ /// common set of "good enough" severity levels so that logging front ends
+ /// can provide filtering and searching across log types. Users of the API are
+ /// free not to use all severity levels in their log messages.
+ struct Severity {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ enum Enum: SwiftProtobuf.Enum {
+ typealias RawValue = Int
+
+ /// Unspecified level information. Will be logged at the TRACE level.
+ case unspecified // = 0
+ case trace // = 1
+
+ /// Debugging information.
+ case debug // = 2
+
+ /// Normal events.
+ case info // = 3
+
+ /// Normal but significant events, such as start up, shut down, or
+ /// configuration.
+ case notice // = 4
+
+ /// Warning events might cause problems.
+ case warn // = 5
+
+ /// Error events are likely to cause problems.
+ case error // = 6
+
+ /// Critical events cause severe problems or brief outages and may
+ /// indicate that a person must take action.
+ case critical // = 7
+ case UNRECOGNIZED(Int)
+
+ init() {
+ self = .unspecified
+ }
+
+ init?(rawValue: Int) {
+ switch rawValue {
+ case 0: self = .unspecified
+ case 1: self = .trace
+ case 2: self = .debug
+ case 3: self = .info
+ case 4: self = .notice
+ case 5: self = .warn
+ case 6: self = .error
+ case 7: self = .critical
+ default: self = .UNRECOGNIZED(rawValue)
+ }
+ }
+
+ var rawValue: Int {
+ switch self {
+ case .unspecified: return 0
+ case .trace: return 1
+ case .debug: return 2
+ case .info: return 3
+ case .notice: return 4
+ case .warn: return 5
+ case .error: return 6
+ case .critical: return 7
+ case .UNRECOGNIZED(let i): return i
+ }
+ }
+
+ }
+
+ init() {}
+ }
+
+ init() {}
+
+ fileprivate var _timestamp: SwiftProtobuf.Google_Protobuf_Timestamp? = nil
+ fileprivate var _customData: SwiftProtobuf.Google_Protobuf_Struct? = nil
+}
+
+#if swift(>=4.2)
+
+extension Org_Apache_Beam_Model_FnExecution_V1_LogEntry.Severity.Enum: CaseIterable {
+ // The compiler won't synthesize support with the UNRECOGNIZED case.
+ static var allCases: [Org_Apache_Beam_Model_FnExecution_V1_LogEntry.Severity.Enum] = [
+ .unspecified,
+ .trace,
+ .debug,
+ .info,
+ .notice,
+ .warn,
+ .error,
+ .critical,
+ ]
+}
+
+#endif // swift(>=4.2)
+
+struct Org_Apache_Beam_Model_FnExecution_V1_LogControl {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+struct Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var workerID: String = String()
+
+ var controlEndpoint: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor {
+ get {return _controlEndpoint ?? Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor()}
+ set {_controlEndpoint = newValue}
+ }
+ /// Returns true if `controlEndpoint` has been explicitly set.
+ var hasControlEndpoint: Bool {return self._controlEndpoint != nil}
+ /// Clears the value of `controlEndpoint`. Subsequent reads from it will return its default value.
+ mutating func clearControlEndpoint() {self._controlEndpoint = nil}
+
+ var loggingEndpoint: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor {
+ get {return _loggingEndpoint ?? Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor()}
+ set {_loggingEndpoint = newValue}
+ }
+ /// Returns true if `loggingEndpoint` has been explicitly set.
+ var hasLoggingEndpoint: Bool {return self._loggingEndpoint != nil}
+ /// Clears the value of `loggingEndpoint`. Subsequent reads from it will return its default value.
+ mutating func clearLoggingEndpoint() {self._loggingEndpoint = nil}
+
+ var artifactEndpoint: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor {
+ get {return _artifactEndpoint ?? Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor()}
+ set {_artifactEndpoint = newValue}
+ }
+ /// Returns true if `artifactEndpoint` has been explicitly set.
+ var hasArtifactEndpoint: Bool {return self._artifactEndpoint != nil}
+ /// Clears the value of `artifactEndpoint`. Subsequent reads from it will return its default value.
+ mutating func clearArtifactEndpoint() {self._artifactEndpoint = nil}
+
+ var provisionEndpoint: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor {
+ get {return _provisionEndpoint ?? Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor()}
+ set {_provisionEndpoint = newValue}
+ }
+ /// Returns true if `provisionEndpoint` has been explicitly set.
+ var hasProvisionEndpoint: Bool {return self._provisionEndpoint != nil}
+ /// Clears the value of `provisionEndpoint`. Subsequent reads from it will return its default value.
+ mutating func clearProvisionEndpoint() {self._provisionEndpoint = nil}
+
+ var params: Dictionary<String,String> = [:]
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+
+ fileprivate var _controlEndpoint: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor? = nil
+ fileprivate var _loggingEndpoint: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor? = nil
+ fileprivate var _artifactEndpoint: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor? = nil
+ fileprivate var _provisionEndpoint: Org_Apache_Beam_Model_Pipeline_V1_ApiServiceDescriptor? = nil
+}
+
+struct Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var error: String = String()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+struct Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var workerID: String = String()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+struct Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ var error: String = String()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// Request from runner to SDK Harness asking for its status. For more details see
+/// https://s.apache.org/beam-fn-api-harness-status
+struct Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) Unique ID identifying this request.
+ var id: String = String()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+/// Response from SDK Harness to runner containing the debug related status info.
+struct Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse {
+ // SwiftProtobuf.Message conformance is added in an extension below. See the
+ // `Message` and `Message+*Additions` files in the SwiftProtobuf library for
+ // methods supported on all messages.
+
+ /// (Required) Unique ID from the original request.
+ var id: String = String()
+
+ /// (Optional) Error message if exception encountered generating the status response.
+ var error: String = String()
+
+ /// (Optional) Status debugging info reported by SDK harness worker. Content and
+ /// format is not strongly enforced but should be print-friendly and
+ /// appropriate as an HTTP response body for end user. For details of the preferred
+ /// info to include in the message see
+ /// https://s.apache.org/beam-fn-api-harness-status
+ var statusInfo: String = String()
+
+ var unknownFields = SwiftProtobuf.UnknownStorage()
+
+ init() {}
+}
+
+#if swift(>=5.5) && canImport(_Concurrency)
+extension Org_Apache_Beam_Model_FnExecution_V1_FnApiTransforms: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_FnApiTransforms.Runner: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_RemoteGrpcPort: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest.OneOf_Request: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse.OneOf_Response: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_SampleDataRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_SampledElement: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_SampledElement.Exception: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse.ElementList: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_RegisterRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_RegisterResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_BundleApplication: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_DelayedBundleApplication: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.OneOf_Type: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.UserState: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.SideInput: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest.DesiredSplit: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse.ChannelSplit: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_Elements: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_Elements.DataMessage: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_Elements.Timers: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateRequest.OneOf_Request: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateResponse.OneOf_Response: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateKey: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateKey.OneOf_Type: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateKey.Runner: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateKey.IterableSideInput: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapSideInput: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapKeysSideInput: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateKey.BagUserState: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapKeysUserState: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapUserState: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateGetRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateGetResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateAppendRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateAppendResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateClearRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StateClearResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_LogEntry: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_LogEntry.List: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_LogEntry.Severity: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_LogEntry.Severity.Enum: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_LogControl: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StartWorkerRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StartWorkerResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StopWorkerRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_StopWorkerResponse: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusRequest: @unchecked Sendable {}
+extension Org_Apache_Beam_Model_FnExecution_V1_WorkerStatusResponse: @unchecked Sendable {}
+#endif // swift(>=5.5) && canImport(_Concurrency)
+
+// MARK: - Code below here is support for the SwiftProtobuf runtime.
+
+fileprivate let _protobuf_package = "org.apache.beam.model.fn_execution.v1"
+
+extension Org_Apache_Beam_Model_FnExecution_V1_FnApiTransforms: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".FnApiTransforms"
+ static let _protobuf_nameMap = SwiftProtobuf._NameMap()
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let _ = try decoder.nextFieldNumber() {
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_FnApiTransforms, rhs: Org_Apache_Beam_Model_FnExecution_V1_FnApiTransforms) -> Bool {
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_FnApiTransforms.Runner: SwiftProtobuf._ProtoNameProviding {
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 0: .same(proto: "DATA_SOURCE"),
+ 1: .same(proto: "DATA_SINK"),
+ ]
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_RemoteGrpcPort: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".RemoteGrpcPort"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "api_service_descriptor"),
+ 2: .standard(proto: "coder_id"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularMessageField(value: &self._apiServiceDescriptor) }()
+ case 2: try { try decoder.decodeSingularStringField(value: &self.coderID) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every if/case branch local when no optimizations
+ // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and
+ // https://github.com/apple/swift-protobuf/issues/1182
+ try { if let v = self._apiServiceDescriptor {
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1)
+ } }()
+ if !self.coderID.isEmpty {
+ try visitor.visitSingularStringField(value: self.coderID, fieldNumber: 2)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_RemoteGrpcPort, rhs: Org_Apache_Beam_Model_FnExecution_V1_RemoteGrpcPort) -> Bool {
+ if lhs._apiServiceDescriptor != rhs._apiServiceDescriptor {return false}
+ if lhs.coderID != rhs.coderID {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".GetProcessBundleDescriptorRequest"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "process_bundle_descriptor_id"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.processBundleDescriptorID) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.processBundleDescriptorID.isEmpty {
+ try visitor.visitSingularStringField(value: self.processBundleDescriptorID, fieldNumber: 1)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest, rhs: Org_Apache_Beam_Model_FnExecution_V1_GetProcessBundleDescriptorRequest) -> Bool {
+ if lhs.processBundleDescriptorID != rhs.processBundleDescriptorID {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".InstructionRequest"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "instruction_id"),
+ 1001: .standard(proto: "process_bundle"),
+ 1002: .standard(proto: "process_bundle_progress"),
+ 1003: .standard(proto: "process_bundle_split"),
+ 1004: .standard(proto: "finalize_bundle"),
+ 1005: .standard(proto: "monitoring_infos"),
+ 1006: .standard(proto: "harness_monitoring_infos"),
+ 1007: .standard(proto: "sample_data"),
+ 1000: .same(proto: "register"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.instructionID) }()
+ case 1000: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_RegisterRequest?
+ var hadOneofValue = false
+ if let current = self.request {
+ hadOneofValue = true
+ if case .register(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.request = .register(v)
+ }
+ }()
+ case 1001: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest?
+ var hadOneofValue = false
+ if let current = self.request {
+ hadOneofValue = true
+ if case .processBundle(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.request = .processBundle(v)
+ }
+ }()
+ case 1002: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressRequest?
+ var hadOneofValue = false
+ if let current = self.request {
+ hadOneofValue = true
+ if case .processBundleProgress(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.request = .processBundleProgress(v)
+ }
+ }()
+ case 1003: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest?
+ var hadOneofValue = false
+ if let current = self.request {
+ hadOneofValue = true
+ if case .processBundleSplit(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.request = .processBundleSplit(v)
+ }
+ }()
+ case 1004: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleRequest?
+ var hadOneofValue = false
+ if let current = self.request {
+ hadOneofValue = true
+ if case .finalizeBundle(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.request = .finalizeBundle(v)
+ }
+ }()
+ case 1005: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataRequest?
+ var hadOneofValue = false
+ if let current = self.request {
+ hadOneofValue = true
+ if case .monitoringInfos(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.request = .monitoringInfos(v)
+ }
+ }()
+ case 1006: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosRequest?
+ var hadOneofValue = false
+ if let current = self.request {
+ hadOneofValue = true
+ if case .harnessMonitoringInfos(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.request = .harnessMonitoringInfos(v)
+ }
+ }()
+ case 1007: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_SampleDataRequest?
+ var hadOneofValue = false
+ if let current = self.request {
+ hadOneofValue = true
+ if case .sampleData(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.request = .sampleData(v)
+ }
+ }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every if/case branch local when no optimizations
+ // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and
+ // https://github.com/apple/swift-protobuf/issues/1182
+ if !self.instructionID.isEmpty {
+ try visitor.visitSingularStringField(value: self.instructionID, fieldNumber: 1)
+ }
+ switch self.request {
+ case .register?: try {
+ guard case .register(let v)? = self.request else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1000)
+ }()
+ case .processBundle?: try {
+ guard case .processBundle(let v)? = self.request else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1001)
+ }()
+ case .processBundleProgress?: try {
+ guard case .processBundleProgress(let v)? = self.request else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1002)
+ }()
+ case .processBundleSplit?: try {
+ guard case .processBundleSplit(let v)? = self.request else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1003)
+ }()
+ case .finalizeBundle?: try {
+ guard case .finalizeBundle(let v)? = self.request else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1004)
+ }()
+ case .monitoringInfos?: try {
+ guard case .monitoringInfos(let v)? = self.request else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1005)
+ }()
+ case .harnessMonitoringInfos?: try {
+ guard case .harnessMonitoringInfos(let v)? = self.request else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1006)
+ }()
+ case .sampleData?: try {
+ guard case .sampleData(let v)? = self.request else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1007)
+ }()
+ case nil: break
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest, rhs: Org_Apache_Beam_Model_FnExecution_V1_InstructionRequest) -> Bool {
+ if lhs.instructionID != rhs.instructionID {return false}
+ if lhs.request != rhs.request {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".InstructionResponse"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "instruction_id"),
+ 2: .same(proto: "error"),
+ 1001: .standard(proto: "process_bundle"),
+ 1002: .standard(proto: "process_bundle_progress"),
+ 1003: .standard(proto: "process_bundle_split"),
+ 1004: .standard(proto: "finalize_bundle"),
+ 1005: .standard(proto: "monitoring_infos"),
+ 1006: .standard(proto: "harness_monitoring_infos"),
+ 1007: .standard(proto: "sample_data"),
+ 1000: .same(proto: "register"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.instructionID) }()
+ case 2: try { try decoder.decodeSingularStringField(value: &self.error) }()
+ case 1000: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_RegisterResponse?
+ var hadOneofValue = false
+ if let current = self.response {
+ hadOneofValue = true
+ if case .register(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.response = .register(v)
+ }
+ }()
+ case 1001: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleResponse?
+ var hadOneofValue = false
+ if let current = self.response {
+ hadOneofValue = true
+ if case .processBundle(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.response = .processBundle(v)
+ }
+ }()
+ case 1002: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressResponse?
+ var hadOneofValue = false
+ if let current = self.response {
+ hadOneofValue = true
+ if case .processBundleProgress(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.response = .processBundleProgress(v)
+ }
+ }()
+ case 1003: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse?
+ var hadOneofValue = false
+ if let current = self.response {
+ hadOneofValue = true
+ if case .processBundleSplit(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.response = .processBundleSplit(v)
+ }
+ }()
+ case 1004: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleResponse?
+ var hadOneofValue = false
+ if let current = self.response {
+ hadOneofValue = true
+ if case .finalizeBundle(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.response = .finalizeBundle(v)
+ }
+ }()
+ case 1005: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataResponse?
+ var hadOneofValue = false
+ if let current = self.response {
+ hadOneofValue = true
+ if case .monitoringInfos(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.response = .monitoringInfos(v)
+ }
+ }()
+ case 1006: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosResponse?
+ var hadOneofValue = false
+ if let current = self.response {
+ hadOneofValue = true
+ if case .harnessMonitoringInfos(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.response = .harnessMonitoringInfos(v)
+ }
+ }()
+ case 1007: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse?
+ var hadOneofValue = false
+ if let current = self.response {
+ hadOneofValue = true
+ if case .sampleData(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.response = .sampleData(v)
+ }
+ }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every if/case branch local when no optimizations
+ // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and
+ // https://github.com/apple/swift-protobuf/issues/1182
+ if !self.instructionID.isEmpty {
+ try visitor.visitSingularStringField(value: self.instructionID, fieldNumber: 1)
+ }
+ if !self.error.isEmpty {
+ try visitor.visitSingularStringField(value: self.error, fieldNumber: 2)
+ }
+ switch self.response {
+ case .register?: try {
+ guard case .register(let v)? = self.response else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1000)
+ }()
+ case .processBundle?: try {
+ guard case .processBundle(let v)? = self.response else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1001)
+ }()
+ case .processBundleProgress?: try {
+ guard case .processBundleProgress(let v)? = self.response else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1002)
+ }()
+ case .processBundleSplit?: try {
+ guard case .processBundleSplit(let v)? = self.response else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1003)
+ }()
+ case .finalizeBundle?: try {
+ guard case .finalizeBundle(let v)? = self.response else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1004)
+ }()
+ case .monitoringInfos?: try {
+ guard case .monitoringInfos(let v)? = self.response else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1005)
+ }()
+ case .harnessMonitoringInfos?: try {
+ guard case .harnessMonitoringInfos(let v)? = self.response else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1006)
+ }()
+ case .sampleData?: try {
+ guard case .sampleData(let v)? = self.response else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1007)
+ }()
+ case nil: break
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse, rhs: Org_Apache_Beam_Model_FnExecution_V1_InstructionResponse) -> Bool {
+ if lhs.instructionID != rhs.instructionID {return false}
+ if lhs.error != rhs.error {return false}
+ if lhs.response != rhs.response {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_SampleDataRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".SampleDataRequest"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "pcollection_ids"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeRepeatedStringField(value: &self.pcollectionIds) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.pcollectionIds.isEmpty {
+ try visitor.visitRepeatedStringField(value: self.pcollectionIds, fieldNumber: 1)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_SampleDataRequest, rhs: Org_Apache_Beam_Model_FnExecution_V1_SampleDataRequest) -> Bool {
+ if lhs.pcollectionIds != rhs.pcollectionIds {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_SampledElement: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".SampledElement"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .same(proto: "element"),
+ 2: .standard(proto: "sample_timestamp"),
+ 3: .same(proto: "exception"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularBytesField(value: &self.element) }()
+ case 2: try { try decoder.decodeSingularMessageField(value: &self._sampleTimestamp) }()
+ case 3: try { try decoder.decodeSingularMessageField(value: &self._exception) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every if/case branch local when no optimizations
+ // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and
+ // https://github.com/apple/swift-protobuf/issues/1182
+ if !self.element.isEmpty {
+ try visitor.visitSingularBytesField(value: self.element, fieldNumber: 1)
+ }
+ try { if let v = self._sampleTimestamp {
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 2)
+ } }()
+ try { if let v = self._exception {
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 3)
+ } }()
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_SampledElement, rhs: Org_Apache_Beam_Model_FnExecution_V1_SampledElement) -> Bool {
+ if lhs.element != rhs.element {return false}
+ if lhs._sampleTimestamp != rhs._sampleTimestamp {return false}
+ if lhs._exception != rhs._exception {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_SampledElement.Exception: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = Org_Apache_Beam_Model_FnExecution_V1_SampledElement.protoMessageName + ".Exception"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "instruction_id"),
+ 2: .standard(proto: "transform_id"),
+ 3: .same(proto: "error"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.instructionID) }()
+ case 2: try { try decoder.decodeSingularStringField(value: &self.transformID) }()
+ case 3: try { try decoder.decodeSingularStringField(value: &self.error) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.instructionID.isEmpty {
+ try visitor.visitSingularStringField(value: self.instructionID, fieldNumber: 1)
+ }
+ if !self.transformID.isEmpty {
+ try visitor.visitSingularStringField(value: self.transformID, fieldNumber: 2)
+ }
+ if !self.error.isEmpty {
+ try visitor.visitSingularStringField(value: self.error, fieldNumber: 3)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_SampledElement.Exception, rhs: Org_Apache_Beam_Model_FnExecution_V1_SampledElement.Exception) -> Bool {
+ if lhs.instructionID != rhs.instructionID {return false}
+ if lhs.transformID != rhs.transformID {return false}
+ if lhs.error != rhs.error {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".SampleDataResponse"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "element_samples"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse.ElementList>.self, value: &self.elementSamples) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.elementSamples.isEmpty {
+ try visitor.visitMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse.ElementList>.self, value: self.elementSamples, fieldNumber: 1)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse, rhs: Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse) -> Bool {
+ if lhs.elementSamples != rhs.elementSamples {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse.ElementList: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse.protoMessageName + ".ElementList"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .same(proto: "elements"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeRepeatedMessageField(value: &self.elements) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.elements.isEmpty {
+ try visitor.visitRepeatedMessageField(value: self.elements, fieldNumber: 1)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse.ElementList, rhs: Org_Apache_Beam_Model_FnExecution_V1_SampleDataResponse.ElementList) -> Bool {
+ if lhs.elements != rhs.elements {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".HarnessMonitoringInfosRequest"
+ static let _protobuf_nameMap = SwiftProtobuf._NameMap()
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let _ = try decoder.nextFieldNumber() {
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosRequest, rhs: Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosRequest) -> Bool {
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosResponse: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".HarnessMonitoringInfosResponse"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "monitoring_data"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeMapField(fieldType: SwiftProtobuf._ProtobufMap<SwiftProtobuf.ProtobufString,SwiftProtobuf.ProtobufBytes>.self, value: &self.monitoringData) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.monitoringData.isEmpty {
+ try visitor.visitMapField(fieldType: SwiftProtobuf._ProtobufMap<SwiftProtobuf.ProtobufString,SwiftProtobuf.ProtobufBytes>.self, value: self.monitoringData, fieldNumber: 1)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosResponse, rhs: Org_Apache_Beam_Model_FnExecution_V1_HarnessMonitoringInfosResponse) -> Bool {
+ if lhs.monitoringData != rhs.monitoringData {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_RegisterRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".RegisterRequest"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "process_bundle_descriptor"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeRepeatedMessageField(value: &self.processBundleDescriptor) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.processBundleDescriptor.isEmpty {
+ try visitor.visitRepeatedMessageField(value: self.processBundleDescriptor, fieldNumber: 1)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_RegisterRequest, rhs: Org_Apache_Beam_Model_FnExecution_V1_RegisterRequest) -> Bool {
+ if lhs.processBundleDescriptor != rhs.processBundleDescriptor {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_RegisterResponse: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".RegisterResponse"
+ static let _protobuf_nameMap = SwiftProtobuf._NameMap()
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let _ = try decoder.nextFieldNumber() {
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_RegisterResponse, rhs: Org_Apache_Beam_Model_FnExecution_V1_RegisterResponse) -> Bool {
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".ProcessBundleDescriptor"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .same(proto: "id"),
+ 2: .same(proto: "transforms"),
+ 3: .same(proto: "pcollections"),
+ 4: .standard(proto: "windowing_strategies"),
+ 5: .same(proto: "coders"),
+ 6: .same(proto: "environments"),
+ 7: .standard(proto: "state_api_service_descriptor"),
+ 8: .standard(proto: "timer_api_service_descriptor"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.id) }()
+ case 2: try { try decoder.decodeMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_Pipeline_V1_PTransform>.self, value: &self.transforms) }()
+ case 3: try { try decoder.decodeMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_Pipeline_V1_PCollection>.self, value: &self.pcollections) }()
+ case 4: try { try decoder.decodeMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_Pipeline_V1_WindowingStrategy>.self, value: &self.windowingStrategies) }()
+ case 5: try { try decoder.decodeMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_Pipeline_V1_Coder>.self, value: &self.coders) }()
+ case 6: try { try decoder.decodeMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_Pipeline_V1_Environment>.self, value: &self.environments) }()
+ case 7: try { try decoder.decodeSingularMessageField(value: &self._stateApiServiceDescriptor) }()
+ case 8: try { try decoder.decodeSingularMessageField(value: &self._timerApiServiceDescriptor) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every if/case branch local when no optimizations
+ // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and
+ // https://github.com/apple/swift-protobuf/issues/1182
+ if !self.id.isEmpty {
+ try visitor.visitSingularStringField(value: self.id, fieldNumber: 1)
+ }
+ if !self.transforms.isEmpty {
+ try visitor.visitMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_Pipeline_V1_PTransform>.self, value: self.transforms, fieldNumber: 2)
+ }
+ if !self.pcollections.isEmpty {
+ try visitor.visitMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_Pipeline_V1_PCollection>.self, value: self.pcollections, fieldNumber: 3)
+ }
+ if !self.windowingStrategies.isEmpty {
+ try visitor.visitMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_Pipeline_V1_WindowingStrategy>.self, value: self.windowingStrategies, fieldNumber: 4)
+ }
+ if !self.coders.isEmpty {
+ try visitor.visitMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_Pipeline_V1_Coder>.self, value: self.coders, fieldNumber: 5)
+ }
+ if !self.environments.isEmpty {
+ try visitor.visitMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_Pipeline_V1_Environment>.self, value: self.environments, fieldNumber: 6)
+ }
+ try { if let v = self._stateApiServiceDescriptor {
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 7)
+ } }()
+ try { if let v = self._timerApiServiceDescriptor {
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 8)
+ } }()
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleDescriptor) -> Bool {
+ if lhs.id != rhs.id {return false}
+ if lhs.transforms != rhs.transforms {return false}
+ if lhs.pcollections != rhs.pcollections {return false}
+ if lhs.windowingStrategies != rhs.windowingStrategies {return false}
+ if lhs.coders != rhs.coders {return false}
+ if lhs.environments != rhs.environments {return false}
+ if lhs._stateApiServiceDescriptor != rhs._stateApiServiceDescriptor {return false}
+ if lhs._timerApiServiceDescriptor != rhs._timerApiServiceDescriptor {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_BundleApplication: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".BundleApplication"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "transform_id"),
+ 2: .standard(proto: "input_id"),
+ 3: .same(proto: "element"),
+ 4: .standard(proto: "output_watermarks"),
+ 5: .standard(proto: "is_bounded"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.transformID) }()
+ case 2: try { try decoder.decodeSingularStringField(value: &self.inputID) }()
+ case 3: try { try decoder.decodeSingularBytesField(value: &self.element) }()
+ case 4: try { try decoder.decodeMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,SwiftProtobuf.Google_Protobuf_Timestamp>.self, value: &self.outputWatermarks) }()
+ case 5: try { try decoder.decodeSingularEnumField(value: &self.isBounded) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.transformID.isEmpty {
+ try visitor.visitSingularStringField(value: self.transformID, fieldNumber: 1)
+ }
+ if !self.inputID.isEmpty {
+ try visitor.visitSingularStringField(value: self.inputID, fieldNumber: 2)
+ }
+ if !self.element.isEmpty {
+ try visitor.visitSingularBytesField(value: self.element, fieldNumber: 3)
+ }
+ if !self.outputWatermarks.isEmpty {
+ try visitor.visitMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,SwiftProtobuf.Google_Protobuf_Timestamp>.self, value: self.outputWatermarks, fieldNumber: 4)
+ }
+ if self.isBounded != .unspecified {
+ try visitor.visitSingularEnumField(value: self.isBounded, fieldNumber: 5)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_BundleApplication, rhs: Org_Apache_Beam_Model_FnExecution_V1_BundleApplication) -> Bool {
+ if lhs.transformID != rhs.transformID {return false}
+ if lhs.inputID != rhs.inputID {return false}
+ if lhs.element != rhs.element {return false}
+ if lhs.outputWatermarks != rhs.outputWatermarks {return false}
+ if lhs.isBounded != rhs.isBounded {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_DelayedBundleApplication: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".DelayedBundleApplication"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .same(proto: "application"),
+ 2: .standard(proto: "requested_time_delay"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularMessageField(value: &self._application) }()
+ case 2: try { try decoder.decodeSingularMessageField(value: &self._requestedTimeDelay) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every if/case branch local when no optimizations
+ // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and
+ // https://github.com/apple/swift-protobuf/issues/1182
+ try { if let v = self._application {
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1)
+ } }()
+ try { if let v = self._requestedTimeDelay {
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 2)
+ } }()
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_DelayedBundleApplication, rhs: Org_Apache_Beam_Model_FnExecution_V1_DelayedBundleApplication) -> Bool {
+ if lhs._application != rhs._application {return false}
+ if lhs._requestedTimeDelay != rhs._requestedTimeDelay {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".ProcessBundleRequest"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "process_bundle_descriptor_id"),
+ 2: .standard(proto: "cache_tokens"),
+ 3: .same(proto: "elements"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.processBundleDescriptorID) }()
+ case 2: try { try decoder.decodeRepeatedMessageField(value: &self.cacheTokens) }()
+ case 3: try { try decoder.decodeSingularMessageField(value: &self._elements) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every if/case branch local when no optimizations
+ // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and
+ // https://github.com/apple/swift-protobuf/issues/1182
+ if !self.processBundleDescriptorID.isEmpty {
+ try visitor.visitSingularStringField(value: self.processBundleDescriptorID, fieldNumber: 1)
+ }
+ if !self.cacheTokens.isEmpty {
+ try visitor.visitRepeatedMessageField(value: self.cacheTokens, fieldNumber: 2)
+ }
+ try { if let v = self._elements {
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 3)
+ } }()
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest) -> Bool {
+ if lhs.processBundleDescriptorID != rhs.processBundleDescriptorID {return false}
+ if lhs.cacheTokens != rhs.cacheTokens {return false}
+ if lhs._elements != rhs._elements {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.protoMessageName + ".CacheToken"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "user_state"),
+ 2: .standard(proto: "side_input"),
+ 10: .same(proto: "token"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.UserState?
+ var hadOneofValue = false
+ if let current = self.type {
+ hadOneofValue = true
+ if case .userState(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.type = .userState(v)
+ }
+ }()
+ case 2: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.SideInput?
+ var hadOneofValue = false
+ if let current = self.type {
+ hadOneofValue = true
+ if case .sideInput(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.type = .sideInput(v)
+ }
+ }()
+ case 10: try { try decoder.decodeSingularBytesField(value: &self.token) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every if/case branch local when no optimizations
+ // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and
+ // https://github.com/apple/swift-protobuf/issues/1182
+ switch self.type {
+ case .userState?: try {
+ guard case .userState(let v)? = self.type else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1)
+ }()
+ case .sideInput?: try {
+ guard case .sideInput(let v)? = self.type else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 2)
+ }()
+ case nil: break
+ }
+ if !self.token.isEmpty {
+ try visitor.visitSingularBytesField(value: self.token, fieldNumber: 10)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken) -> Bool {
+ if lhs.type != rhs.type {return false}
+ if lhs.token != rhs.token {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.UserState: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.protoMessageName + ".UserState"
+ static let _protobuf_nameMap = SwiftProtobuf._NameMap()
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let _ = try decoder.nextFieldNumber() {
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.UserState, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.UserState) -> Bool {
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.SideInput: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.protoMessageName + ".SideInput"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "transform_id"),
+ 2: .standard(proto: "side_input_id"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.transformID) }()
+ case 2: try { try decoder.decodeSingularStringField(value: &self.sideInputID) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.transformID.isEmpty {
+ try visitor.visitSingularStringField(value: self.transformID, fieldNumber: 1)
+ }
+ if !self.sideInputID.isEmpty {
+ try visitor.visitSingularStringField(value: self.sideInputID, fieldNumber: 2)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.SideInput, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleRequest.CacheToken.SideInput) -> Bool {
+ if lhs.transformID != rhs.transformID {return false}
+ if lhs.sideInputID != rhs.sideInputID {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleResponse: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".ProcessBundleResponse"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 2: .standard(proto: "residual_roots"),
+ 3: .standard(proto: "monitoring_infos"),
+ 4: .standard(proto: "requires_finalization"),
+ 5: .standard(proto: "monitoring_data"),
+ 6: .same(proto: "elements"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 2: try { try decoder.decodeRepeatedMessageField(value: &self.residualRoots) }()
+ case 3: try { try decoder.decodeRepeatedMessageField(value: &self.monitoringInfos) }()
+ case 4: try { try decoder.decodeSingularBoolField(value: &self.requiresFinalization) }()
+ case 5: try { try decoder.decodeMapField(fieldType: SwiftProtobuf._ProtobufMap<SwiftProtobuf.ProtobufString,SwiftProtobuf.ProtobufBytes>.self, value: &self.monitoringData) }()
+ case 6: try { try decoder.decodeSingularMessageField(value: &self._elements) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every if/case branch local when no optimizations
+ // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and
+ // https://github.com/apple/swift-protobuf/issues/1182
+ if !self.residualRoots.isEmpty {
+ try visitor.visitRepeatedMessageField(value: self.residualRoots, fieldNumber: 2)
+ }
+ if !self.monitoringInfos.isEmpty {
+ try visitor.visitRepeatedMessageField(value: self.monitoringInfos, fieldNumber: 3)
+ }
+ if self.requiresFinalization != false {
+ try visitor.visitSingularBoolField(value: self.requiresFinalization, fieldNumber: 4)
+ }
+ if !self.monitoringData.isEmpty {
+ try visitor.visitMapField(fieldType: SwiftProtobuf._ProtobufMap<SwiftProtobuf.ProtobufString,SwiftProtobuf.ProtobufBytes>.self, value: self.monitoringData, fieldNumber: 5)
+ }
+ try { if let v = self._elements {
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 6)
+ } }()
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleResponse, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleResponse) -> Bool {
+ if lhs.residualRoots != rhs.residualRoots {return false}
+ if lhs.monitoringInfos != rhs.monitoringInfos {return false}
+ if lhs.requiresFinalization != rhs.requiresFinalization {return false}
+ if lhs.monitoringData != rhs.monitoringData {return false}
+ if lhs._elements != rhs._elements {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".ProcessBundleProgressRequest"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "instruction_id"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.instructionID) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.instructionID.isEmpty {
+ try visitor.visitSingularStringField(value: self.instructionID, fieldNumber: 1)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressRequest, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressRequest) -> Bool {
+ if lhs.instructionID != rhs.instructionID {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".MonitoringInfosMetadataRequest"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "monitoring_info_id"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeRepeatedStringField(value: &self.monitoringInfoID) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.monitoringInfoID.isEmpty {
+ try visitor.visitRepeatedStringField(value: self.monitoringInfoID, fieldNumber: 1)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataRequest, rhs: Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataRequest) -> Bool {
+ if lhs.monitoringInfoID != rhs.monitoringInfoID {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressResponse: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".ProcessBundleProgressResponse"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 3: .standard(proto: "monitoring_infos"),
+ 5: .standard(proto: "monitoring_data"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 3: try { try decoder.decodeRepeatedMessageField(value: &self.monitoringInfos) }()
+ case 5: try { try decoder.decodeMapField(fieldType: SwiftProtobuf._ProtobufMap<SwiftProtobuf.ProtobufString,SwiftProtobuf.ProtobufBytes>.self, value: &self.monitoringData) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.monitoringInfos.isEmpty {
+ try visitor.visitRepeatedMessageField(value: self.monitoringInfos, fieldNumber: 3)
+ }
+ if !self.monitoringData.isEmpty {
+ try visitor.visitMapField(fieldType: SwiftProtobuf._ProtobufMap<SwiftProtobuf.ProtobufString,SwiftProtobuf.ProtobufBytes>.self, value: self.monitoringData, fieldNumber: 5)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressResponse, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleProgressResponse) -> Bool {
+ if lhs.monitoringInfos != rhs.monitoringInfos {return false}
+ if lhs.monitoringData != rhs.monitoringData {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataResponse: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".MonitoringInfosMetadataResponse"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "monitoring_info"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_Pipeline_V1_MonitoringInfo>.self, value: &self.monitoringInfo) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.monitoringInfo.isEmpty {
+ try visitor.visitMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_Pipeline_V1_MonitoringInfo>.self, value: self.monitoringInfo, fieldNumber: 1)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataResponse, rhs: Org_Apache_Beam_Model_FnExecution_V1_MonitoringInfosMetadataResponse) -> Bool {
+ if lhs.monitoringInfo != rhs.monitoringInfo {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".ProcessBundleSplitRequest"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "instruction_id"),
+ 3: .standard(proto: "desired_splits"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.instructionID) }()
+ case 3: try { try decoder.decodeMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest.DesiredSplit>.self, value: &self.desiredSplits) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.instructionID.isEmpty {
+ try visitor.visitSingularStringField(value: self.instructionID, fieldNumber: 1)
+ }
+ if !self.desiredSplits.isEmpty {
+ try visitor.visitMapField(fieldType: SwiftProtobuf._ProtobufMessageMap<SwiftProtobuf.ProtobufString,Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest.DesiredSplit>.self, value: self.desiredSplits, fieldNumber: 3)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest) -> Bool {
+ if lhs.instructionID != rhs.instructionID {return false}
+ if lhs.desiredSplits != rhs.desiredSplits {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest.DesiredSplit: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest.protoMessageName + ".DesiredSplit"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "fraction_of_remainder"),
+ 3: .standard(proto: "allowed_split_points"),
+ 2: .standard(proto: "estimated_input_elements"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularDoubleField(value: &self.fractionOfRemainder) }()
+ case 2: try { try decoder.decodeSingularInt64Field(value: &self.estimatedInputElements) }()
+ case 3: try { try decoder.decodeRepeatedInt64Field(value: &self.allowedSplitPoints) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if self.fractionOfRemainder != 0 {
+ try visitor.visitSingularDoubleField(value: self.fractionOfRemainder, fieldNumber: 1)
+ }
+ if self.estimatedInputElements != 0 {
+ try visitor.visitSingularInt64Field(value: self.estimatedInputElements, fieldNumber: 2)
+ }
+ if !self.allowedSplitPoints.isEmpty {
+ try visitor.visitPackedInt64Field(value: self.allowedSplitPoints, fieldNumber: 3)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest.DesiredSplit, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitRequest.DesiredSplit) -> Bool {
+ if lhs.fractionOfRemainder != rhs.fractionOfRemainder {return false}
+ if lhs.allowedSplitPoints != rhs.allowedSplitPoints {return false}
+ if lhs.estimatedInputElements != rhs.estimatedInputElements {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".ProcessBundleSplitResponse"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "primary_roots"),
+ 2: .standard(proto: "residual_roots"),
+ 3: .standard(proto: "channel_splits"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeRepeatedMessageField(value: &self.primaryRoots) }()
+ case 2: try { try decoder.decodeRepeatedMessageField(value: &self.residualRoots) }()
+ case 3: try { try decoder.decodeRepeatedMessageField(value: &self.channelSplits) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.primaryRoots.isEmpty {
+ try visitor.visitRepeatedMessageField(value: self.primaryRoots, fieldNumber: 1)
+ }
+ if !self.residualRoots.isEmpty {
+ try visitor.visitRepeatedMessageField(value: self.residualRoots, fieldNumber: 2)
+ }
+ if !self.channelSplits.isEmpty {
+ try visitor.visitRepeatedMessageField(value: self.channelSplits, fieldNumber: 3)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse) -> Bool {
+ if lhs.primaryRoots != rhs.primaryRoots {return false}
+ if lhs.residualRoots != rhs.residualRoots {return false}
+ if lhs.channelSplits != rhs.channelSplits {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse.ChannelSplit: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse.protoMessageName + ".ChannelSplit"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "transform_id"),
+ 2: .standard(proto: "last_primary_element"),
+ 3: .standard(proto: "first_residual_element"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.transformID) }()
+ case 2: try { try decoder.decodeSingularInt64Field(value: &self.lastPrimaryElement) }()
+ case 3: try { try decoder.decodeSingularInt64Field(value: &self.firstResidualElement) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.transformID.isEmpty {
+ try visitor.visitSingularStringField(value: self.transformID, fieldNumber: 1)
+ }
+ if self.lastPrimaryElement != 0 {
+ try visitor.visitSingularInt64Field(value: self.lastPrimaryElement, fieldNumber: 2)
+ }
+ if self.firstResidualElement != 0 {
+ try visitor.visitSingularInt64Field(value: self.firstResidualElement, fieldNumber: 3)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse.ChannelSplit, rhs: Org_Apache_Beam_Model_FnExecution_V1_ProcessBundleSplitResponse.ChannelSplit) -> Bool {
+ if lhs.transformID != rhs.transformID {return false}
+ if lhs.lastPrimaryElement != rhs.lastPrimaryElement {return false}
+ if lhs.firstResidualElement != rhs.firstResidualElement {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".FinalizeBundleRequest"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "instruction_id"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.instructionID) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.instructionID.isEmpty {
+ try visitor.visitSingularStringField(value: self.instructionID, fieldNumber: 1)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleRequest, rhs: Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleRequest) -> Bool {
+ if lhs.instructionID != rhs.instructionID {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleResponse: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".FinalizeBundleResponse"
+ static let _protobuf_nameMap = SwiftProtobuf._NameMap()
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let _ = try decoder.nextFieldNumber() {
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleResponse, rhs: Org_Apache_Beam_Model_FnExecution_V1_FinalizeBundleResponse) -> Bool {
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_Elements: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".Elements"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .same(proto: "data"),
+ 2: .same(proto: "timers"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeRepeatedMessageField(value: &self.data) }()
+ case 2: try { try decoder.decodeRepeatedMessageField(value: &self.timers) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.data.isEmpty {
+ try visitor.visitRepeatedMessageField(value: self.data, fieldNumber: 1)
+ }
+ if !self.timers.isEmpty {
+ try visitor.visitRepeatedMessageField(value: self.timers, fieldNumber: 2)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_Elements, rhs: Org_Apache_Beam_Model_FnExecution_V1_Elements) -> Bool {
+ if lhs.data != rhs.data {return false}
+ if lhs.timers != rhs.timers {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_Elements.DataMessage: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = Org_Apache_Beam_Model_FnExecution_V1_Elements.protoMessageName + ".Data"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "instruction_id"),
+ 2: .standard(proto: "transform_id"),
+ 3: .same(proto: "data"),
+ 4: .standard(proto: "is_last"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.instructionID) }()
+ case 2: try { try decoder.decodeSingularStringField(value: &self.transformID) }()
+ case 3: try { try decoder.decodeSingularBytesField(value: &self.data) }()
+ case 4: try { try decoder.decodeSingularBoolField(value: &self.isLast) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.instructionID.isEmpty {
+ try visitor.visitSingularStringField(value: self.instructionID, fieldNumber: 1)
+ }
+ if !self.transformID.isEmpty {
+ try visitor.visitSingularStringField(value: self.transformID, fieldNumber: 2)
+ }
+ if !self.data.isEmpty {
+ try visitor.visitSingularBytesField(value: self.data, fieldNumber: 3)
+ }
+ if self.isLast != false {
+ try visitor.visitSingularBoolField(value: self.isLast, fieldNumber: 4)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_Elements.DataMessage, rhs: Org_Apache_Beam_Model_FnExecution_V1_Elements.DataMessage) -> Bool {
+ if lhs.instructionID != rhs.instructionID {return false}
+ if lhs.transformID != rhs.transformID {return false}
+ if lhs.data != rhs.data {return false}
+ if lhs.isLast != rhs.isLast {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_Elements.Timers: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = Org_Apache_Beam_Model_FnExecution_V1_Elements.protoMessageName + ".Timers"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .standard(proto: "instruction_id"),
+ 2: .standard(proto: "transform_id"),
+ 3: .standard(proto: "timer_family_id"),
+ 4: .same(proto: "timers"),
+ 5: .standard(proto: "is_last"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.instructionID) }()
+ case 2: try { try decoder.decodeSingularStringField(value: &self.transformID) }()
+ case 3: try { try decoder.decodeSingularStringField(value: &self.timerFamilyID) }()
+ case 4: try { try decoder.decodeSingularBytesField(value: &self.timers) }()
+ case 5: try { try decoder.decodeSingularBoolField(value: &self.isLast) }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ if !self.instructionID.isEmpty {
+ try visitor.visitSingularStringField(value: self.instructionID, fieldNumber: 1)
+ }
+ if !self.transformID.isEmpty {
+ try visitor.visitSingularStringField(value: self.transformID, fieldNumber: 2)
+ }
+ if !self.timerFamilyID.isEmpty {
+ try visitor.visitSingularStringField(value: self.timerFamilyID, fieldNumber: 3)
+ }
+ if !self.timers.isEmpty {
+ try visitor.visitSingularBytesField(value: self.timers, fieldNumber: 4)
+ }
+ if self.isLast != false {
+ try visitor.visitSingularBoolField(value: self.isLast, fieldNumber: 5)
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_Elements.Timers, rhs: Org_Apache_Beam_Model_FnExecution_V1_Elements.Timers) -> Bool {
+ if lhs.instructionID != rhs.instructionID {return false}
+ if lhs.transformID != rhs.transformID {return false}
+ if lhs.timerFamilyID != rhs.timerFamilyID {return false}
+ if lhs.timers != rhs.timers {return false}
+ if lhs.isLast != rhs.isLast {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_StateRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".StateRequest"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .same(proto: "id"),
+ 2: .standard(proto: "instruction_id"),
+ 3: .standard(proto: "state_key"),
+ 1000: .same(proto: "get"),
+ 1001: .same(proto: "append"),
+ 1002: .same(proto: "clear"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.id) }()
+ case 2: try { try decoder.decodeSingularStringField(value: &self.instructionID) }()
+ case 3: try { try decoder.decodeSingularMessageField(value: &self._stateKey) }()
+ case 1000: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateGetRequest?
+ var hadOneofValue = false
+ if let current = self.request {
+ hadOneofValue = true
+ if case .get(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.request = .get(v)
+ }
+ }()
+ case 1001: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateAppendRequest?
+ var hadOneofValue = false
+ if let current = self.request {
+ hadOneofValue = true
+ if case .append(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.request = .append(v)
+ }
+ }()
+ case 1002: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateClearRequest?
+ var hadOneofValue = false
+ if let current = self.request {
+ hadOneofValue = true
+ if case .clear(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.request = .clear(v)
+ }
+ }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every if/case branch local when no optimizations
+ // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and
+ // https://github.com/apple/swift-protobuf/issues/1182
+ if !self.id.isEmpty {
+ try visitor.visitSingularStringField(value: self.id, fieldNumber: 1)
+ }
+ if !self.instructionID.isEmpty {
+ try visitor.visitSingularStringField(value: self.instructionID, fieldNumber: 2)
+ }
+ try { if let v = self._stateKey {
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 3)
+ } }()
+ switch self.request {
+ case .get?: try {
+ guard case .get(let v)? = self.request else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1000)
+ }()
+ case .append?: try {
+ guard case .append(let v)? = self.request else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1001)
+ }()
+ case .clear?: try {
+ guard case .clear(let v)? = self.request else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1002)
+ }()
+ case nil: break
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_StateRequest, rhs: Org_Apache_Beam_Model_FnExecution_V1_StateRequest) -> Bool {
+ if lhs.id != rhs.id {return false}
+ if lhs.instructionID != rhs.instructionID {return false}
+ if lhs._stateKey != rhs._stateKey {return false}
+ if lhs.request != rhs.request {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_StateResponse: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".StateResponse"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .same(proto: "id"),
+ 2: .same(proto: "error"),
+ 1000: .same(proto: "get"),
+ 1001: .same(proto: "append"),
+ 1002: .same(proto: "clear"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try { try decoder.decodeSingularStringField(value: &self.id) }()
+ case 2: try { try decoder.decodeSingularStringField(value: &self.error) }()
+ case 1000: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateGetResponse?
+ var hadOneofValue = false
+ if let current = self.response {
+ hadOneofValue = true
+ if case .get(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.response = .get(v)
+ }
+ }()
+ case 1001: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateAppendResponse?
+ var hadOneofValue = false
+ if let current = self.response {
+ hadOneofValue = true
+ if case .append(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.response = .append(v)
+ }
+ }()
+ case 1002: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateClearResponse?
+ var hadOneofValue = false
+ if let current = self.response {
+ hadOneofValue = true
+ if case .clear(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.response = .clear(v)
+ }
+ }()
+ default: break
+ }
+ }
+ }
+
+ func traverse<V: SwiftProtobuf.Visitor>(visitor: inout V) throws {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every if/case branch local when no optimizations
+ // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and
+ // https://github.com/apple/swift-protobuf/issues/1182
+ if !self.id.isEmpty {
+ try visitor.visitSingularStringField(value: self.id, fieldNumber: 1)
+ }
+ if !self.error.isEmpty {
+ try visitor.visitSingularStringField(value: self.error, fieldNumber: 2)
+ }
+ switch self.response {
+ case .get?: try {
+ guard case .get(let v)? = self.response else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1000)
+ }()
+ case .append?: try {
+ guard case .append(let v)? = self.response else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1001)
+ }()
+ case .clear?: try {
+ guard case .clear(let v)? = self.response else { preconditionFailure() }
+ try visitor.visitSingularMessageField(value: v, fieldNumber: 1002)
+ }()
+ case nil: break
+ }
+ try unknownFields.traverse(visitor: &visitor)
+ }
+
+ static func ==(lhs: Org_Apache_Beam_Model_FnExecution_V1_StateResponse, rhs: Org_Apache_Beam_Model_FnExecution_V1_StateResponse) -> Bool {
+ if lhs.id != rhs.id {return false}
+ if lhs.error != rhs.error {return false}
+ if lhs.response != rhs.response {return false}
+ if lhs.unknownFields != rhs.unknownFields {return false}
+ return true
+ }
+}
+
+extension Org_Apache_Beam_Model_FnExecution_V1_StateKey: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding {
+ static let protoMessageName: String = _protobuf_package + ".StateKey"
+ static let _protobuf_nameMap: SwiftProtobuf._NameMap = [
+ 1: .same(proto: "runner"),
+ 2: .standard(proto: "multimap_side_input"),
+ 3: .standard(proto: "bag_user_state"),
+ 4: .standard(proto: "iterable_side_input"),
+ 5: .standard(proto: "multimap_keys_side_input"),
+ 6: .standard(proto: "multimap_keys_user_state"),
+ 7: .standard(proto: "multimap_user_state"),
+ ]
+
+ mutating func decodeMessage<D: SwiftProtobuf.Decoder>(decoder: inout D) throws {
+ while let fieldNumber = try decoder.nextFieldNumber() {
+ // The use of inline closures is to circumvent an issue where the compiler
+ // allocates stack space for every case branch when no optimizations are
+ // enabled. https://github.com/apple/swift-protobuf/issues/1034
+ switch fieldNumber {
+ case 1: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateKey.Runner?
+ var hadOneofValue = false
+ if let current = self.type {
+ hadOneofValue = true
+ if case .runner(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.type = .runner(v)
+ }
+ }()
+ case 2: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapSideInput?
+ var hadOneofValue = false
+ if let current = self.type {
+ hadOneofValue = true
+ if case .multimapSideInput(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.type = .multimapSideInput(v)
+ }
+ }()
+ case 3: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateKey.BagUserState?
+ var hadOneofValue = false
+ if let current = self.type {
+ hadOneofValue = true
+ if case .bagUserState(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.type = .bagUserState(v)
+ }
+ }()
+ case 4: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateKey.IterableSideInput?
+ var hadOneofValue = false
+ if let current = self.type {
+ hadOneofValue = true
+ if case .iterableSideInput(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.type = .iterableSideInput(v)
+ }
+ }()
+ case 5: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapKeysSideInput?
+ var hadOneofValue = false
+ if let current = self.type {
+ hadOneofValue = true
+ if case .multimapKeysSideInput(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.type = .multimapKeysSideInput(v)
+ }
+ }()
+ case 6: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapKeysUserState?
+ var hadOneofValue = false
+ if let current = self.type {
+ hadOneofValue = true
+ if case .multimapKeysUserState(let m) = current {v = m}
+ }
+ try decoder.decodeSingularMessageField(value: &v)
+ if let v = v {
+ if hadOneofValue {try decoder.handleConflictingOneOf()}
+ self.type = .multimapKeysUserState(v)
+ }
+ }()
+ case 7: try {
+ var v: Org_Apache_Beam_Model_FnExecution_V1_StateKey.MultimapUserState?
+ var hadOneofValue = false
+ if let current = self.type {
+ hadOneofValue = true
+ if case .multimapUserState(let m) = current {v = m}
+ }
+