n3nash commented on a change in pull request #2263:
URL: https://github.com/apache/hudi/pull/2263#discussion_r535652862



##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkRunClusteringCommitActionExecutor.java
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.action.cluster;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.avro.model.HoodieClusteringGroup;
+import org.apache.hudi.avro.model.HoodieClusteringPlan;
+import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.log.HoodieFileSliceReader;
+import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.CommitUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieClusteringException;
+import org.apache.hudi.io.IOUtils;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.action.cluster.strategy.RunClusteringStrategy;
+import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+public class SparkRunClusteringCommitActionExecutor<T extends 
HoodieRecordPayload<T>>
+    extends BaseSparkCommitActionExecutor<T> {
+
+  private static final Logger LOG = 
LogManager.getLogger(SparkRunClusteringCommitActionExecutor.class);
+  private final HoodieClusteringPlan clusteringPlan;
+
+  public SparkRunClusteringCommitActionExecutor(HoodieEngineContext context,
+                                                HoodieWriteConfig config, 
HoodieTable table,
+                                                String instantTime) {
+    super(context, config, table, instantTime, WriteOperationType.CLUSTER);
+    this.clusteringPlan = 
ClusteringUtils.getClusteringPlan(table.getMetaClient(), 
HoodieTimeline.getReplaceCommitRequestedInstant(instantTime))
+      .map(Pair::getRight).orElseThrow(() -> new 
HoodieClusteringException("Unable to read clustering plan for instant: " + 
instantTime));
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
+    HoodieInstant instant = 
HoodieTimeline.getReplaceCommitRequestedInstant(instantTime);
+    // Mark instant as clustering inflight
+    table.getActiveTimeline().transitionReplaceRequestedToInflight(instant, 
Option.empty());
+    table.getMetaClient().reloadActiveTimeline();
+
+    JavaSparkContext engineContext = 
HoodieSparkEngineContext.getSparkContext(context);
+    // run clustering for each group async and collect WriteStatus
+    JavaRDD<WriteStatus> writeStatusRDD = 
clusteringPlan.getInputGroups().stream()
+        .map(inputGroup -> runClusteringForGroupAsync(inputGroup, 
clusteringPlan.getStrategy().getStrategyParams()))
+        .map(CompletableFuture::join)
+        .reduce((rdd1, rdd2) -> 
rdd1.union(rdd2)).orElse(engineContext.emptyRDD());
+    if (writeStatusRDD.isEmpty()) {
+      throw new HoodieClusteringException("Clustering plan produced 0 
WriteStatus for " + instantTime + " #groups: " + 
clusteringPlan.getInputGroups().size());
+    }
+
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> writeMetadata = 
buildWriteMetadata(writeStatusRDD);
+    updateIndexAndCommitIfNeeded(writeStatusRDD, writeMetadata);
+    if (!writeMetadata.getCommitMetadata().isPresent()) {
+      HoodieCommitMetadata commitMetadata = 
CommitUtils.buildMetadata(writeStatusRDD.map(WriteStatus::getStat).collect(), 
writeMetadata.getPartitionToReplaceFileIds(),
+          extraMetadata, operationType, getSchemaToStoreInCommit(), 
getCommitActionType());
+      writeMetadata.setCommitMetadata(Option.of(commitMetadata));
+    }
+    return writeMetadata;
+  }
+
+  private CompletableFuture<JavaRDD<WriteStatus>> 
runClusteringForGroupAsync(HoodieClusteringGroup clusteringGroup, Map<String, 
String> strategyParams) {
+    CompletableFuture<JavaRDD<WriteStatus>> writeStatusesFuture = 
CompletableFuture.supplyAsync(() -> {
+      JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
+      JavaRDD<HoodieRecord<? extends HoodieRecordPayload>> inputRecords = 
jsc.parallelize(clusteringGroup.getSlices(), 
clusteringGroup.getSlices().size()).map(sliceInfo -> {
+        long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new 
SparkTaskContextSupplier(), config.getProps());
+        LOG.info("MaxMemoryPerCompaction run as part of clustering => " + 
maxMemoryPerCompaction);
+        try {
+          Schema readerSchema = HoodieAvroUtils.addMetadataFields(new 
Schema.Parser().parse(config.getSchema()));
+          HoodieFileReader<? extends IndexedRecord> baseFileReader = 
HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new 
Path(sliceInfo.getDataFilePath()));
+          HoodieMergedLogRecordScanner scanner = new 
HoodieMergedLogRecordScanner(table.getMetaClient().getFs(),

Review comment:
       @satishkotha I see the new one looks messy, if the runtime/difference in 
performance is the same for MergedScanner and no memory overhead of FileSlices 
without any log files, let's keep your other implementation with MergedScanner 
for both. Added 1 comment on the FileSliceReader




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to