Skip to content
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

[HUDI-4397] Flink Inline Cluster and Compact plan distribute strategy… #6106

Merged
merged 1 commit into from
Jul 15, 2022
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.hudi.sink.utils;

import org.apache.hudi.common.model.ClusteringOperation;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.configuration.FlinkOptions;
Expand Down Expand Up @@ -65,6 +66,7 @@

import java.util.HashMap;
import java.util.Map;
import java.util.stream.Collectors;

/**
* Utilities to generate all kinds of sub-pipelines.
Expand Down Expand Up @@ -357,7 +359,7 @@ public static DataStream<Object> hoodieStreamWrite(Configuration conf, int defau
*
* <pre>
* /=== | task1 | ===\
* | plan generation | ===> re-balance | commit |
* | plan generation | ===> hash | commit |
* \=== | task2 | ===/
*
* Note: both the compaction plan generation task and commission task are singleton.
Expand All @@ -372,7 +374,7 @@ public static DataStreamSink<CompactionCommitEvent> compact(Configuration conf,
TypeInformation.of(CompactionPlanEvent.class),
new CompactionPlanOperator(conf))
.setParallelism(1) // plan generate must be singleton
.rebalance()
.keyBy(plan -> plan.getOperation().getFileGroupId().getFileId())
.transform("compact_task",
TypeInformation.of(CompactionCommitEvent.class),
new ProcessOperator<>(new CompactFunction(conf)))
Expand All @@ -392,7 +394,7 @@ public static DataStreamSink<CompactionCommitEvent> compact(Configuration conf,
*
* <pre>
* /=== | task1 | ===\
* | plan generation | ===> re-balance | commit |
* | plan generation | ===> hash | commit |
* \=== | task2 | ===/
*
* Note: both the clustering plan generation task and commission task are singleton.
Expand All @@ -408,7 +410,9 @@ public static DataStreamSink<ClusteringCommitEvent> cluster(Configuration conf,
TypeInformation.of(ClusteringPlanEvent.class),
new ClusteringPlanOperator(conf))
.setParallelism(1) // plan generate must be singleton
.rebalance()
.keyBy(plan -> plan.getClusteringGroupInfo().getOperations()
.stream().map(ClusteringOperation::getFileId)
.collect(Collectors.joining()))
.transform("clustering_task",
TypeInformation.of(ClusteringCommitEvent.class),
new ClusteringOperator(conf, rowType))
Expand Down