1
0

[HUDI-2901] Fixed the bug clustering jobs cannot running in parallel (#4178)

This commit is contained in:
xiarixiaoyao
2021-12-10 14:39:35 +08:00
committed by GitHub
parent ea154bcb5d
commit 456d74ce4e
2 changed files with 52 additions and 5 deletions

View File

@@ -35,6 +35,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.RewriteAvroPayload;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.common.util.FutureUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.collection.Pair;
@@ -88,16 +89,17 @@ public abstract class MultipleSparkJobExecutionStrategy<T extends HoodieRecordPa
@Override
public HoodieWriteMetadata<JavaRDD<WriteStatus>> performClustering(final HoodieClusteringPlan clusteringPlan, final Schema schema, final String instantTime) {
// execute clustering for each group async and collect WriteStatus
JavaSparkContext engineContext = HoodieSparkEngineContext.getSparkContext(getEngineContext());
// execute clustering for each group async and collect WriteStatus
Stream<JavaRDD<WriteStatus>> writeStatusRDDStream = clusteringPlan.getInputGroups().stream()
Stream<JavaRDD<WriteStatus>> writeStatusRDDStream = FutureUtils.allOf(
clusteringPlan.getInputGroups().stream()
.map(inputGroup -> runClusteringForGroupAsync(inputGroup,
clusteringPlan.getStrategy().getStrategyParams(),
Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false),
instantTime))
.map(CompletableFuture::join);
.collect(Collectors.toList()))
.join()
.stream();
JavaRDD<WriteStatus>[] writeStatuses = convertStreamToArray(writeStatusRDDStream);
JavaRDD<WriteStatus> writeStatusRDD = engineContext.union(writeStatuses);
@@ -145,7 +147,6 @@ public abstract class MultipleSparkJobExecutionStrategy<T extends HoodieRecordPa
}
}
/**
* Submit job to execute clustering for the group.
*/