ahmedabu98 opened a new issue, #32365: URL: https://github.com/apache/beam/issues/32365
### What needs to happen? Dynamic destinations is a feature (implemented in some IOs) that allows users to write to a multitude of destinations using one IO instance. At runtime, the IO determines a destination for each incoming record based on a function. Currently, this feature is only supported with native use-cases (e.g. Python uses [callables](https://beam.apache.org/releases/pydoc/2.54.0/apache_beam.io.fileio.html#dynamic-destinations); Java uses [SerializableFunction](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/FileIO.html#:~:text=as%20per%20above.-,Dynamic%20destinations,-If%20the%20elements) or [DynamicDestinations](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.Write.html#to-org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinations-)). Of course, we can’t simply pass a whole function from one language to another, but we can extract the important bits and articulate them in a way that is [compatible](https://github.com/apache/beam/blob/e067ed216bbd2868ebb22c360600f3dd3e489897/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema.proto#L194-L204) with Beam SDK languages. There have been [discussions](https://lists.apache.org/thread/khq2pv6469l1or6yd40b9bqkr26bjjm6) and a design doc: https://s.apache.org/portable-dynamic-destinations The agreed upon approach (proposal (1) in the doc) looks something like this: <img width="711" alt="image" src="https://github.com/user-attachments/assets/9f159bec-1393-4537-b2fb-a3e5badb7b96"> PR #31807 is a proof of concept that includes the necessary utilities and a real implementation for portable dynamic destinations in IcebergIO. This PR can be broken down into multiple PRs though to decouple concepts and make it easier to review. ### Issue Priority Priority: 2 (default / most normal work should be filed as P2) ### Issue Components - [ ] Component: Python SDK - [X] Component: Java SDK - [ ] Component: Go SDK - [ ] Component: Typescript SDK - [X] Component: IO connector - [ ] Component: Beam YAML - [ ] Component: Beam examples - [ ] Component: Beam playground - [ ] Component: Beam katas - [ ] Component: Website - [ ] Component: Infrastructure - [ ] Component: Spark Runner - [ ] Component: Flink Runner - [ ] Component: Samza Runner - [ ] Component: Twister2 Runner - [ ] Component: Hazelcast Jet Runner - [ ] Component: Google Cloud Dataflow Runner -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
