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}
+        }
+