yuxiqian commented on code in PR #3801:
URL: https://github.com/apache/flink-cdc/pull/3801#discussion_r1891245022
##########
flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposer.java:
##########
@@ -94,76 +97,124 @@ public PipelineExecution compose(PipelineDef pipelineDef) {
int parallelism =
pipelineDefConfig.get(PipelineOptions.PIPELINE_PARALLELISM);
env.getConfig().setParallelism(parallelism);
+ translate(env, pipelineDef);
+
+ // Add framework JARs
+ addFrameworkJars();
+
+ return new FlinkPipelineExecution(
+ env, pipelineDefConfig.get(PipelineOptions.PIPELINE_NAME),
isBlocking);
+ }
+
+ private void translate(StreamExecutionEnvironment env, PipelineDef
pipelineDef) {
+ Configuration pipelineDefConfig = pipelineDef.getConfig();
+ int parallelism =
pipelineDefConfig.get(PipelineOptions.PIPELINE_PARALLELISM);
SchemaChangeBehavior schemaChangeBehavior =
pipelineDefConfig.get(PipelineOptions.PIPELINE_SCHEMA_CHANGE_BEHAVIOR);
- // Build Source Operator
+ // Initialize translators
DataSourceTranslator sourceTranslator = new DataSourceTranslator();
- DataStream<Event> stream =
- sourceTranslator.translate(
- pipelineDef.getSource(), env, pipelineDefConfig,
parallelism);
-
- // Build PreTransformOperator for processing Schema Event
TransformTranslator transformTranslator = new TransformTranslator();
- stream =
- transformTranslator.translatePreTransform(
- stream,
- pipelineDef.getTransforms(),
- pipelineDef.getUdfs(),
- pipelineDef.getModels());
-
- // Schema operator
+ PartitioningTranslator partitioningTranslator = new
PartitioningTranslator();
SchemaOperatorTranslator schemaOperatorTranslator =
new SchemaOperatorTranslator(
schemaChangeBehavior,
pipelineDefConfig.get(PipelineOptions.PIPELINE_SCHEMA_OPERATOR_UID),
pipelineDefConfig.get(PipelineOptions.PIPELINE_SCHEMA_OPERATOR_RPC_TIMEOUT),
pipelineDefConfig.get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE));
+ DistributedSchemaOperatorTranslator
distributedSchemaOperatorTranslator =
+ new DistributedSchemaOperatorTranslator(
+ schemaChangeBehavior,
+
pipelineDefConfig.get(PipelineOptions.PIPELINE_SCHEMA_OPERATOR_UID),
+
pipelineDefConfig.get(PipelineOptions.PIPELINE_SCHEMA_OPERATOR_RPC_TIMEOUT),
+
pipelineDefConfig.get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE));
+ DataSinkTranslator sinkTranslator = new DataSinkTranslator();
+
+ // And required constructors
OperatorIDGenerator schemaOperatorIDGenerator =
new
OperatorIDGenerator(schemaOperatorTranslator.getSchemaOperatorUid());
-
- // Build PostTransformOperator for processing Data Event
- stream =
- transformTranslator.translatePostTransform(
- stream,
- pipelineDef.getTransforms(),
-
pipelineDef.getConfig().get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE),
- pipelineDef.getUdfs(),
- pipelineDef.getModels());
-
- // Build DataSink in advance as schema operator requires
MetadataApplier
- DataSinkTranslator sinkTranslator = new DataSinkTranslator();
+ DataSource dataSource =
+ sourceTranslator.createDataSource(pipelineDef.getSource(),
pipelineDefConfig, env);
DataSink dataSink =
sinkTranslator.createDataSink(pipelineDef.getSink(),
pipelineDefConfig, env);
+ boolean canContainDistributedTables =
dataSource.canContainDistributedTables();
+
+ // O ---> Source
+ DataStream<Event> stream =
+ sourceTranslator.translate(
+ pipelineDef.getSource(), dataSource, env,
pipelineDefConfig, parallelism);
+
+ // Source ---> PreTransform
stream =
- schemaOperatorTranslator.translate(
+ transformTranslator.translatePreTransform(
stream,
- parallelism,
- dataSink.getMetadataApplier()
- .setAcceptedSchemaEvolutionTypes(
-
pipelineDef.getSink().getIncludedSchemaEvolutionTypes()),
- pipelineDef.getRoute());
+ pipelineDef.getTransforms(),
+ pipelineDef.getUdfs(),
+ pipelineDef.getModels(),
+ canContainDistributedTables);
- // Build Partitioner used to shuffle Event
- PartitioningTranslator partitioningTranslator = new
PartitioningTranslator();
+ // PreTransform ---> PostTransform
stream =
- partitioningTranslator.translate(
+ transformTranslator.translatePostTransform(
stream,
- parallelism,
- parallelism,
- schemaOperatorIDGenerator.generate(),
-
dataSink.getDataChangeEventHashFunctionProvider(parallelism));
-
- // Build Sink Operator
- sinkTranslator.translate(
- pipelineDef.getSink(), stream, dataSink,
schemaOperatorIDGenerator.generate());
-
- // Add framework JARs
- addFrameworkJars();
+ pipelineDef.getTransforms(),
+
pipelineDef.getConfig().get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE),
+ pipelineDef.getUdfs(),
+ pipelineDef.getModels());
- return new FlinkPipelineExecution(
- env, pipelineDefConfig.get(PipelineOptions.PIPELINE_NAME),
isBlocking);
+ if (canContainDistributedTables) {
+ // Translate a distributed topology for sources with distributed
tables
+ // PostTransform -> Partitioning
+ DataStream<PartitioningEvent> partitionedStream =
+ partitioningTranslator.translateDistributed(
+ stream,
+ parallelism,
+ parallelism,
+
dataSink.getDataChangeEventHashFunctionProvider(parallelism));
+
+ // Partitioning -> Schema Operator
+ stream =
+ distributedSchemaOperatorTranslator.translate(
+ partitionedStream,
+ parallelism,
+ dataSink.getMetadataApplier()
+ .setAcceptedSchemaEvolutionTypes(
+ pipelineDef
+ .getSink()
+
.getIncludedSchemaEvolutionTypes()),
+ pipelineDef.getRoute());
+
+ // Schema Operator -> Sink
+ sinkTranslator.translate(
+ pipelineDef.getSink(), stream, dataSink,
schemaOperatorIDGenerator.generate());
+ } else {
+ // Translate a regular topology for sources without distributed
tables
+ // PostTransform ---> Schema Operator
+ stream =
+ schemaOperatorTranslator.translate(
+ stream,
+ parallelism,
+ dataSink.getMetadataApplier()
+ .setAcceptedSchemaEvolutionTypes(
+ pipelineDef
+ .getSink()
+
.getIncludedSchemaEvolutionTypes()),
+ pipelineDef.getRoute());
+
+ // Schema Operator ---(shuffled)---> Partitioning
+ stream =
+ partitioningTranslator.translateRegular(
+ stream,
+ parallelism,
+ parallelism,
+ schemaOperatorIDGenerator.generate(),
+
dataSink.getDataChangeEventHashFunctionProvider(parallelism));
+
+ // Partitioning ---> Sink ---> X
+ sinkTranslator.translate(
+ pipelineDef.getSink(), stream, dataSink,
schemaOperatorIDGenerator.generate());
Review Comment:
Good point, extracted common operator parts.
--
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]