-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-7090]Set the maxParallelism for singleton operator #10090
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -410,10 +410,11 @@ public static DataStream<Object> hoodieStreamWrite(Configuration conf, DataStrea | |
| * @return the compaction pipeline | ||
| */ | ||
| public static DataStreamSink<CompactionCommitEvent> compact(Configuration conf, DataStream<Object> dataStream) { | ||
| return dataStream.transform("compact_plan_generate", | ||
| DataStreamSink<CompactionCommitEvent> compactionCommitEventDataStream = dataStream.transform("compact_plan_generate", | ||
| TypeInformation.of(CompactionPlanEvent.class), | ||
| new CompactionPlanOperator(conf)) | ||
| .setParallelism(1) // plan generate must be singleton | ||
| .setMaxParallelism(1) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this line compatible with flink release before 1.18?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. compatible. Only the DataStreamSink Class doesn't have setMaxParallelism method before 1.18.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The SingleOutputStreamOperator Class was returned when call Datastream.transform method. For example : flink 1.14 |
||
| // make the distribution strategy deterministic to avoid concurrent modifications | ||
| // on the same bucket files | ||
| .keyBy(plan -> plan.getOperation().getFileGroupId().getFileId()) | ||
|
|
@@ -424,6 +425,8 @@ public static DataStreamSink<CompactionCommitEvent> compact(Configuration conf, | |
| .addSink(new CompactionCommitSink(conf)) | ||
| .name("compact_commit") | ||
| .setParallelism(1); // compaction commit should be singleton | ||
| compactionCommitEventDataStream.getTransformation().setMaxParallelism(1); | ||
| return compactionCommitEventDataStream; | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -452,6 +455,7 @@ public static DataStreamSink<ClusteringCommitEvent> cluster(Configuration conf, | |
| TypeInformation.of(ClusteringPlanEvent.class), | ||
| new ClusteringPlanOperator(conf)) | ||
| .setParallelism(1) // plan generate must be singleton | ||
| .setMaxParallelism(1) // plan generate must be singleton | ||
| .keyBy(plan -> | ||
| // make the distribution strategy deterministic to avoid concurrent modifications | ||
| // on the same bucket files | ||
|
|
@@ -465,15 +469,19 @@ public static DataStreamSink<ClusteringCommitEvent> cluster(Configuration conf, | |
| ExecNodeUtil.setManagedMemoryWeight(clusteringStream.getTransformation(), | ||
| conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L); | ||
| } | ||
| return clusteringStream.addSink(new ClusteringCommitSink(conf)) | ||
| DataStreamSink<ClusteringCommitEvent> clusteringCommitEventDataStream = clusteringStream.addSink(new ClusteringCommitSink(conf)) | ||
| .name("clustering_commit") | ||
| .setParallelism(1); // clustering commit should be singleton | ||
| clusteringCommitEventDataStream.getTransformation().setMaxParallelism(1); | ||
| return clusteringCommitEventDataStream; | ||
| } | ||
|
|
||
| public static DataStreamSink<Object> clean(Configuration conf, DataStream<Object> dataStream) { | ||
| return dataStream.addSink(new CleanFunction<>(conf)) | ||
| DataStreamSink<Object> cleanCommitDataStream = dataStream.addSink(new CleanFunction<>(conf)) | ||
| .setParallelism(1) | ||
| .name("clean_commits"); | ||
| cleanCommitDataStream.getTransformation().setMaxParallelism(1); | ||
| return cleanCommitDataStream; | ||
| } | ||
|
|
||
| public static DataStreamSink<Object> dummySink(DataStream<Object> dataStream) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -207,6 +207,7 @@ public DataStream<RowData> produceDataStream(StreamExecutionEnvironment execEnv) | |
| SingleOutputStreamOperator<RowData> source = execEnv.addSource(monitoringFunction, getSourceOperatorName("split_monitor")) | ||
| .uid(Pipelines.opUID("split_monitor", conf)) | ||
| .setParallelism(1) | ||
| .setMaxParallelism(1) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is the
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this line compatible with flink release before 1.18? |
||
| .keyBy(MergeOnReadInputSplit::getFileId) | ||
| .transform("split_reader", typeInfo, factory) | ||
| .uid(Pipelines.opUID("split_reader", conf)) | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice catch, just need to fix the compatibility issues: https://github.com/apache/hudi/actions/runs/6864369897/job/18665977391?pr=10090
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Seems that only Flink 1.16+ has this
setParallelismAPI ?Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe 1.18+, add setMaxParallelism method on 2023-04.
https://issues.apache.org/jira/browse/FLINK-31873