You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
FlinkStreamingPortablePipelineTranslator, used for portable streaming pipelines. It has a lot of "overlap" (copy/pasted code) with FlinkStreamingTransformTranslators but supports a slightly different set of transforms. Based on Flink DataStream API too.
FlinkBatchPortablePipelineTranslator -> Used for batch portable pipelines. Based on the deprecated DataSet API.
FlinkBatchTransformTranslators -> Used for batch "native" java pipelines. . Based on the deprecated DataSet API.
FlinkBatchPortablePipelineTranslator and FlinkBatchTransformTranslators should both be deprecated since the are implemented using the deprecated DataSet API which Flink will eventually remove.
FlinkBatchTransformTranslators can be replaced by FlinkStreamingTransformTranslators (#28614).
Given the similarities between the classes, I thinkFlinkStreamingTransformTranslators and FlinkStreamingPortablePipelineTranslator could be merged to support both portable and non portable pipelines with a unique translation layer.
Since we can easily convert instances of org.apache.beam.sdk.Pipeline to RunnerAPI.Pipeline, it should be possible to support all types of pipelines (native streaming, portable streaming, native batch, portable batch) with the same translation implementation.
The goal of this task is to introduce this new unique implementation and to eventually remove all the other alternatives.
Assuming this proposal is of interest, my colleagues and myself could implement it.
Issue Priority
Priority: 3 (nice-to-have improvement)
Issue Components
Component: Python SDK
Component: Java SDK
Component: Go SDK
Component: Typescript SDK
Component: IO connector
Component: Beam examples
Component: Beam playground
Component: Beam katas
Component: Website
Component: Spark Runner
Component: Flink Runner
Component: Samza Runner
Component: Twister2 Runner
Component: Hazelcast Jet Runner
Component: Google Cloud Dataflow Runner
The text was updated successfully, but these errors were encountered:
What needs to happen?
Currently the Flink runner has 4 alternative implementations to translate a pipeline.
FlinkStreamingTransformTranslators
Based on
DataStream
API. Only support streaming workflows. It can also support batch workflows once [Flink Runner] Add UseDataStreamForBatch option to Flink runner to enable batch execution on DataStream API #28614 is merged. It uses the "native"org.apache.beam.sdk.Pipeline
class and the implementation is based on Flink'sDataStream
API.FlinkStreamingPortablePipelineTranslator
, used for portable streaming pipelines. It has a lot of "overlap" (copy/pasted code) withFlinkStreamingTransformTranslators
but supports a slightly different set of transforms. Based on FlinkDataStream
API too.FlinkBatchPortablePipelineTranslator
-> Used for batch portable pipelines. Based on the deprecatedDataSet
API.FlinkBatchTransformTranslators
-> Used for batch "native" java pipelines. . Based on the deprecatedDataSet
API.FlinkBatchPortablePipelineTranslator
andFlinkBatchTransformTranslators
should both be deprecated since the are implemented using the deprecatedDataSet
API which Flink will eventually remove.FlinkBatchTransformTranslators
can be replaced byFlinkStreamingTransformTranslators
(#28614).Given the similarities between the classes, I think
FlinkStreamingTransformTranslators
andFlinkStreamingPortablePipelineTranslator
could be merged to support both portable and non portable pipelines with a unique translation layer.Since we can easily convert instances of
org.apache.beam.sdk.Pipeline
toRunnerAPI.Pipeline
, it should be possible to support all types of pipelines (native streaming, portable streaming, native batch, portable batch) with the same translation implementation.The goal of this task is to introduce this new unique implementation and to eventually remove all the other alternatives.
Assuming this proposal is of interest, my colleagues and myself could implement it.
Issue Priority
Priority: 3 (nice-to-have improvement)
Issue Components
The text was updated successfully, but these errors were encountered: