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



##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ScheduleClusteringStrategy.java
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.strategy;
+
+import org.apache.hudi.avro.model.HoodieClusteringPlan;
+import org.apache.hudi.avro.model.HoodieSliceInfo;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.BaseFile;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.view.SyncableFileSystemView;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Pluggable implementation for scheduling clustering and creating 
ClusteringPlan.
+ */
+public abstract class ScheduleClusteringStrategy<T extends 
HoodieRecordPayload,I,K,O> implements Serializable {

Review comment:
       I feel like inheritance is wrong pattern here. These look similar now, 
but i think metrics will change differently over time. So having tight 
integration may make it difficult for these two evolve differently. Let me know 
if you have strong opinion. I can change it.

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java
##########
@@ -27,8 +27,20 @@
 
 public abstract class AbstractBulkInsertHelper<T extends HoodieRecordPayload, 
I, K, O, R> {
 
+  /**
+   * Mark instant as inflight, write input records, updateIndex and return 
result.

Review comment:
       Fixed

##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java
##########
@@ -296,6 +298,57 @@ protected void completeCompaction(HoodieCommitMetadata 
metadata, JavaRDD<WriteSt
     return statuses;
   }
 
+  @Override
+  protected HoodieWriteMetadata<JavaRDD<WriteStatus>> cluster(String 
clusteringInstant, boolean shouldComplete) {
+    HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
+    HoodieTimeline pendingClusteringTimeline = 
table.getActiveTimeline().filterPendingReplaceTimeline();
+    HoodieInstant inflightInstant = 
HoodieTimeline.getReplaceCommitInflightInstant(clusteringInstant);
+    if (pendingClusteringTimeline.containsInstant(inflightInstant)) {
+      rollbackInflightClustering(inflightInstant, table);
+      table.getMetaClient().reloadActiveTimeline();
+    }
+    clusteringTimer = metrics.getClusteringCtx();
+    LOG.info("Starting clustering at " + clusteringInstant);
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> clusteringMetadata = 
table.clustering(context, clusteringInstant);
+    JavaRDD<WriteStatus> statuses = clusteringMetadata.getWriteStatuses();
+    if (shouldComplete && clusteringMetadata.getCommitMetadata().isPresent()) {
+      completeClustering((HoodieReplaceCommitMetadata) 
clusteringMetadata.getCommitMetadata().get(), statuses, table, 
clusteringInstant);
+    }
+    return clusteringMetadata;
+  }
+
+  protected void completeClustering(HoodieReplaceCommitMetadata metadata, 
JavaRDD<WriteStatus> writeStatuses,
+                                    HoodieTable<T, JavaRDD<HoodieRecord<T>>, 
JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                    String clusteringCommitTime) {
+
+    List<HoodieWriteStat> writeStats = 
writeStatuses.map(WriteStatus::getStat).collect();
+    if (!writeStatuses.filter(WriteStatus::hasErrors).isEmpty()) {
+      throw new HoodieClusteringException("Clustering failed to write to 
files:"
+          + 
writeStatuses.filter(WriteStatus::hasErrors).map(WriteStatus::getFileId).collect());
+    }
+    finalizeWrite(table, clusteringCommitTime, writeStats);
+    try {
+      LOG.info("Committing Clustering " + clusteringCommitTime + ". Finished 
with result " + metadata);
+      table.getActiveTimeline().transitionReplaceInflightToComplete(
+          HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime),
+          Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
+    } catch (IOException e) {
+      throw new HoodieClusteringException("unable to transition clustering 
inflight to complete: " + clusteringCommitTime,  e);
+    }
+
+    if (clusteringTimer != null) {
+      long durationInMs = metrics.getDurationInMs(clusteringTimer.stop());
+      try {
+        
metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(clusteringCommitTime).getTime(),
+            durationInMs, metadata, HoodieActiveTimeline.COMPACTION_ACTION);

Review comment:
       Good catch, fixed.

##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/SparkBulkInsertBasedRunClusteringStrategy.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.client.clustering.run;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+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.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
+import org.apache.hudi.table.action.cluster.strategy.RunClusteringStrategy;
+import org.apache.hudi.table.action.commit.SparkBulkInsertHelper;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.util.Map;
+import java.util.Properties;
+
+import static 
org.apache.hudi.config.HoodieClusteringConfig.SORT_COLUMNS_PROPERTY;
+
+/**
+ * Clustering Strategy based on following.
+ * 1) Spark execution engine.
+ * 2) Uses bulk_insert to write data into new files.
+ */
+public class SparkBulkInsertBasedRunClusteringStrategy<T extends 
HoodieRecordPayload<T>>

Review comment:
       Moved.

##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/schedule/SparkBoundedDayBasedScheduleClusteringStrategy.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.client.clustering.schedule;
+
+import org.apache.hudi.avro.model.HoodieClusteringGroup;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.model.FileSlice;
+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.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
+import 
org.apache.hudi.table.action.cluster.strategy.PartitionAwareScheduleClusteringStrategy;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.hudi.config.HoodieClusteringConfig.SORT_COLUMNS_PROPERTY;
+
+/**
+ * Clustering Strategy based on following.
+ * 1) Spark execution engine.
+ * 2) Limits amount of data per clustering operation.
+ */
+public class SparkBoundedDayBasedScheduleClusteringStrategy<T extends 
HoodieRecordPayload<T>>

Review comment:
       Moved




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