bvaradar commented on a change in pull request #2082:
URL: https://github.com/apache/hudi/pull/2082#discussion_r496273433



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java
##########
@@ -119,6 +137,11 @@ protected void 
addPendingCompactionOperations(Stream<Pair<String, CompactionOper
     );
   }
 
+  @Override
+  protected void addPendingClusteringOperations(Stream<Pair<String, 
ClusteringOperation>> operations) {
+    // TODO

Review comment:
       This TODO and others are needed for incremental filesystem view syncing. 

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java
##########
@@ -159,6 +163,14 @@ public String getFileName() {
       } else {
         return HoodieTimeline.makeCommitFileName(timestamp);
       }
+    } else if (HoodieTimeline.CLUSTERING_ACTION.equals(action)) {

Review comment:
       I am assuming this would be changed to REPLACE_ACTION from the other PR. 

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/table/action/clustering/updates/RejectUpdateStrategy.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.clustering.updates;
+
+import org.apache.hudi.avro.model.HoodieClusteringOperation;
+import org.apache.hudi.avro.model.HoodieClusteringPlan;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieUpdateRejectException;
+import org.apache.hudi.table.WorkloadProfile;
+import org.apache.hudi.table.WorkloadStat;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class RejectUpdateStrategy implements UpdateStrategy {
+  private static final Logger LOG = 
LogManager.getLogger(RejectUpdateStrategy.class);
+
+  @Override
+  public void apply(HoodieTableMetaClient client, WorkloadProfile 
workloadProfile) {
+    List<Pair<HoodieInstant, HoodieClusteringPlan>> plans = 
ClusteringUtils.getAllPendingClusteringPlans(client);
+    if (plans == null || plans.size() == 0) {
+      return;
+    }
+    List<Pair<String, String>> partitionFileIdPairs = plans.stream().map(entry 
-> {
+      HoodieClusteringPlan plan = entry.getValue();
+      List<HoodieClusteringOperation> operations = plan.getOperations();
+      List<Pair<String, String>> partitionFileIdPair =
+              operations.stream()
+                      .flatMap(operation -> 
operation.getBaseFilePaths().stream().map(filePath -> 
Pair.of(operation.getPartitionPath(), FSUtils.getFileId(filePath))))
+                      .collect(Collectors.toList());
+      return partitionFileIdPair;
+    }).collect(Collectors.toList()).stream().flatMap(list -> 
list.stream()).collect(Collectors.toList());
+
+    if (partitionFileIdPairs.size() == 0) {
+      return;
+    }
+
+    Set<Map.Entry<String, WorkloadStat>> partitionStatEntries = 
workloadProfile.getPartitionPathStatMap().entrySet();
+    for (Map.Entry<String, WorkloadStat> partitionStat : partitionStatEntries) 
{
+      for (Map.Entry<String, Pair<String, Long>> updateLocEntry :
+              partitionStat.getValue().getUpdateLocationToCount().entrySet()) {
+        String partitionPath = partitionStat.getKey();
+        String fileId = updateLocEntry.getKey();
+        if (partitionFileIdPairs.contains(Pair.of(partitionPath, fileId))) {
+          LOG.error("Not allowed to update the clustering files, partition: " 
+ partitionPath + ", fileID " + fileId + ", please use other strategy.");

Review comment:
       "please use other strategy" => Reword ? FOr pending clustering 
operations, we are not going to support update for now right ?

##########
File path: hudi-common/src/main/avro/HoodieClusteringPlan.avsc
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ */
+{

Review comment:
       Are we going to reuse Replace metadata or this one ?

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java
##########
@@ -125,6 +128,16 @@ public HoodieWriteMetadata 
bulkInsertPrepped(JavaSparkContext jsc, String instan
         this, instantTime, preppedRecords, bulkInsertPartitioner).execute();
   }
 
+  @Override
+  public Option<HoodieClusteringPlan> scheduleClustering(JavaSparkContext jsc, 
String instantTime, Option<Map<String, String>> extraMetadata) {
+    return new ScheduleClusteringActionExecutor(jsc, config, this, 
instantTime, extraMetadata).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata clustering(JavaSparkContext jsc, String 
compactionInstantTime) {
+    return new RunClusteringActionExecutor(jsc, config, this, 
compactionInstantTime).execute();
+  }
+

Review comment:
       Need to ensure small file handling doesn't toe-step clustering.  In 
other words, we should employ update rejection policy for small file handling 
too.

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/table/action/clustering/strategy/BaseFileSizeBasedClusteringStrategy.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.clustering.strategy;
+
+import org.apache.hudi.avro.model.HoodieClusteringOperation;
+import org.apache.hudi.avro.model.HoodieClusteringPlan;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.config.HoodieWriteConfig;
+import 
org.apache.hudi.table.action.compact.strategy.BoundedIOCompactionStrategy;
+import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * LogFileSizeBasedClusteringStrategy orders the compactions based on the 
total log files size and limits the
+ * clusterings within a configured IO bound.
+ *
+ * @see BoundedIOCompactionStrategy
+ * @see CompactionStrategy
+ */
+public class BaseFileSizeBasedClusteringStrategy extends 
BoundedIOClusteringStrategy
+    implements Comparator<HoodieClusteringOperation> {
+
+  private static final String TOTAL_BASE_FILE_SIZE = "TOTAL_BASE_FILE_SIZE";
+
+  @Override
+  public Map<String, Double> captureMetrics(HoodieWriteConfig config, 
List<HoodieBaseFile> dataFile,
+      String partitionPath) {
+    Map<String, Double> metrics = super.captureMetrics(config, dataFile, 
partitionPath);
+
+    // Total size of all the data files
+    Long totalBaseFileSize = 
dataFile.stream().map(HoodieBaseFile::getFileSize).filter(size -> size >= 0)
+        .reduce(Long::sum).orElse(0L);
+    // save the metrics needed during the order
+    metrics.put(TOTAL_BASE_FILE_SIZE, totalBaseFileSize.doubleValue());
+    return metrics;
+  }
+
+  @Override
+  public List<HoodieClusteringOperation> orderAndFilter(HoodieWriteConfig 
writeConfig,
+      List<HoodieClusteringOperation> operations, List<HoodieClusteringPlan> 
pendingCompactionPlans) {
+    // Order the operations based on the reverse size of the logs and limit 
them by the IO
+    return super.orderAndFilter(writeConfig, 
operations.stream().sorted(this).collect(Collectors.toList()),
+        pendingCompactionPlans);
+  }
+
+  @Override
+  public int compare(HoodieClusteringOperation op1, HoodieClusteringOperation 
op2) {

Review comment:
       This can be moved to separate comparator class ?

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/table/action/clustering/strategy/BoundedIOClusteringStrategy.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.clustering.strategy;
+
+import org.apache.hudi.avro.model.HoodieClusteringOperation;
+import org.apache.hudi.avro.model.HoodieClusteringPlan;
+import org.apache.hudi.config.HoodieWriteConfig;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * CompactionStrategy which looks at total IO to be done for the compaction 
(read + write) and limits the list of

Review comment:
       compaction -> clustering. 
   
   

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java
##########
@@ -110,6 +114,11 @@ protected void resetViewState() {
     return fileIdToPendingCompaction;
   }
 
+  protected Map<HoodieFileGroupId, Pair<String, ClusteringOperation>> 
createFileIdToPendingClusteringMap(

Review comment:
       It would be a good idea to rebase this with 
https://github.com/apache/hudi/pull/2048 as it is almost ready.

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
##########
@@ -65,7 +65,8 @@
       COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, 
DELTA_COMMIT_EXTENSION,
       INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, 
SAVEPOINT_EXTENSION,
       INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, 
REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION,
-      INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, 
INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION));
+      INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, 
INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION,

Review comment:
       Are we going to reuse REPLACE action here ?

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/table/action/clustering/HoodieCopyOnWriteTableCluster.java
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.clustering;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.avro.model.HoodieClusteringOperation;
+import org.apache.hudi.avro.model.HoodieClusteringPlan;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.ClusteringOperation;
+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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.view.SyncableFileSystemView;
+import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
+import org.apache.hudi.table.HoodieCopyOnWriteTable;
+import org.apache.hudi.table.HoodieTable;
+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 org.apache.spark.api.java.function.Function;
+import org.apache.spark.util.AccumulatorV2;
+import org.apache.spark.util.LongAccumulator;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+
+import static java.util.stream.Collectors.toList;
+
+public class HoodieCopyOnWriteTableCluster implements HoodieCluster {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieCopyOnWriteTableCluster.class);
+  // Accumulator to keep track of total file slices for a table
+  private AccumulatorV2<Long, Long> totalFileSlices;
+
+  public static class BaseFileIterator implements Iterator<HoodieRecord<? 
extends HoodieRecordPayload>> {
+    List<HoodieFileReader> readers;
+    Iterator<GenericRecord> currentReader;
+    Schema schema;
+
+    public BaseFileIterator(List<HoodieFileReader> readers, Schema schema) {
+      this.readers = readers;
+      this.schema = schema;
+      if (readers.size() > 0) {
+        try {
+          currentReader = readers.remove(0).getRecordIterator(schema);
+        } catch (Exception e) {
+          throw new HoodieException(e);
+        }
+      }
+    }
+
+    @Override
+    public boolean hasNext() {
+      if (currentReader == null) {
+        return false;
+      } else if (currentReader.hasNext()) {
+        return true;
+      } else if (readers.size() > 0) {
+        try {
+          currentReader = readers.remove(0).getRecordIterator(schema);
+          return currentReader.hasNext();
+        } catch (Exception e) {
+          throw new HoodieException("unable to initialize read with base file 
", e);
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public HoodieRecord<? extends HoodieRecordPayload> next() {
+      //GenericRecord record = currentReader.next();
+      return transform(currentReader.next());
+    }
+
+    private HoodieRecord<? extends HoodieRecordPayload> 
transform(GenericRecord record) {
+      OverwriteWithLatestAvroPayload payload = new 
OverwriteWithLatestAvroPayload(Option.of(record));
+      String key = 
record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+      String partition = 
record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
+
+      HoodieKey hoodieKey = new HoodieKey(key, partition);
+      HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, payload);
+      return hoodieRecord;
+    }
+  }
+
+  @Override
+  public JavaRDD<WriteStatus> clustering(JavaSparkContext jsc, 
HoodieClusteringPlan clusteringPlan,
+      HoodieTable hoodieTable, HoodieWriteConfig config, String 
clusteringInstantTime) throws IOException {
+    if (clusteringPlan == null || (clusteringPlan.getOperations() == null)
+        || (clusteringPlan.getOperations().isEmpty())) {
+      return jsc.emptyRDD();
+    }
+    HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
+    // Compacting is very similar to applying updates to existing file
+    HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, 
jsc.hadoopConfiguration(), metaClient);
+    List<ClusteringOperation> operations = 
clusteringPlan.getOperations().stream()
+        
.map(ClusteringOperation::convertFromAvroRecordInstance).collect(toList());
+    LOG.info("Cluster clustering " + operations + " files");
+
+    return jsc.parallelize(operations, operations.size())
+        .map(s -> clustering(table, metaClient, config, s, 
clusteringInstantTime)).flatMap(List::iterator);
+  }
+
+  private List<WriteStatus> clustering(HoodieCopyOnWriteTable 
hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
+      HoodieWriteConfig config, ClusteringOperation operation, String 
instantTime) {
+    Schema readerSchema = HoodieAvroUtils.addMetadataFields(new 
Schema.Parser().parse(config.getSchema()));
+    LOG.info("Clustering partitionPath " + operation.getPartitionPath() + " 
with base data files " + operation.getBaseFilePaths()
+        + " for commit " + instantTime);
+
+    List<String> baseFiles = operation.getBaseFilePaths().stream().map(
+        p -> new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), 
operation.getPartitionPath()), p).toString())
+        .collect(toList());
+    List<HoodieFileReader> list = baseFiles.stream().map(s -> {
+      try {
+        return 
HoodieFileReaderFactory.getFileReader(hoodieCopyOnWriteTable.getHadoopConf(), 
new Path(s));
+      } catch (IOException e) {
+        throw new HoodieException(e);
+      }
+    }).collect(toList());
+
+    BaseFileIterator baseFileIterator = new BaseFileIterator(list, 
readerSchema);
+    Iterator<List<WriteStatus>> result = 
hoodieCopyOnWriteTable.handleInsert(instantTime, operation.getPartitionPath(), 
FSUtils.createNewFileIdPfx(),
+            baseFileIterator);
+
+    Iterable<List<WriteStatus>> resultIterable = () -> result;
+    return StreamSupport.stream(resultIterable.spliterator(), 
false).flatMap(Collection::stream).peek(s -> {
+      s.getStat().setPartitionPath(operation.getPartitionPath());
+      RuntimeStats runtimeStats = new RuntimeStats();
+      s.getStat().setRuntimeStats(runtimeStats);
+    }).collect(toList());
+  }
+
+  @Override
+  public HoodieClusteringPlan generateClusteringPlan(JavaSparkContext jsc, 
HoodieTable hoodieTable,
+                                                     HoodieWriteConfig config, 
String clusteringCommitTime, Set<HoodieFileGroupId> fgIdsInPendingClusterings)
+      throws IOException {
+
+
+    totalFileSlices = new LongAccumulator();
+    jsc.sc().register(totalFileSlices);
+
+    HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
+    LOG.info("Clustering " + metaClient.getBasePath() + " with commit " + 
clusteringCommitTime);
+    List<String> partitionPaths = 
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
+        config.shouldAssumeDatePartitioning());
+
+    // filter the partition paths if needed to reduce list status
+    partitionPaths = 
config.getClusteringStrategy().filterPartitionPaths(config, partitionPaths);
+
+    if (partitionPaths.isEmpty()) {
+      // In case no partitions could be picked, return no compaction plan
+      return null;
+    }
+
+    SliceView fileSystemView = hoodieTable.getSliceView();
+    LOG.info("Clustering looking for files to cluster in " + partitionPaths + 
" partitions");
+    Set<HoodieFileGroupId> fgIdsPendingCompactions = ((SyncableFileSystemView) 
fileSystemView).getPendingCompactionOperations()
+            .map(instantTimeOpPair -> 
instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+    List<HoodieClusteringOperation> operations = 
jsc.parallelize(partitionPaths, partitionPaths.size()).map((Function<String, 
ClusteringOperation>) partitionPath -> {
+      Stream<FileSlice> fileSliceStream = 
fileSystemView.getLatestFileSlices(partitionPath);
+      List<HoodieBaseFile> baseFiles = fileSliceStream.filter(slice -> 
(!fgIdsInPendingClusterings.contains(slice.getFileGroupId()) && 
!fgIdsPendingCompactions.contains(slice.getFileGroupId())))

Review comment:
       For COW table, pending compactions would not be present.

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/table/action/clustering/updates/UpdateStrategy.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.clustering.updates;
+
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.table.WorkloadProfile;
+
+public interface UpdateStrategy {

Review comment:
       Need description on the purpose of this interface.

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java
##########
@@ -110,6 +114,11 @@ protected void resetViewState() {
     return fileIdToPendingCompaction;
   }
 
+  protected Map<HoodieFileGroupId, Pair<String, ClusteringOperation>> 
createFileIdToPendingClusteringMap(

Review comment:
       Assuming we are going to reuse REPLACE, many of these changes should go 
away ?

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/table/action/clustering/HoodieCopyOnWriteTableCluster.java
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.clustering;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.avro.model.HoodieClusteringOperation;
+import org.apache.hudi.avro.model.HoodieClusteringPlan;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.ClusteringOperation;
+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.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.view.SyncableFileSystemView;
+import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
+import org.apache.hudi.table.HoodieCopyOnWriteTable;
+import org.apache.hudi.table.HoodieTable;
+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 org.apache.spark.api.java.function.Function;
+import org.apache.spark.util.AccumulatorV2;
+import org.apache.spark.util.LongAccumulator;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+
+import static java.util.stream.Collectors.toList;
+
+public class HoodieCopyOnWriteTableCluster implements HoodieCluster {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieCopyOnWriteTableCluster.class);
+  // Accumulator to keep track of total file slices for a table
+  private AccumulatorV2<Long, Long> totalFileSlices;
+
+  public static class BaseFileIterator implements Iterator<HoodieRecord<? 
extends HoodieRecordPayload>> {
+    List<HoodieFileReader> readers;
+    Iterator<GenericRecord> currentReader;
+    Schema schema;
+
+    public BaseFileIterator(List<HoodieFileReader> readers, Schema schema) {
+      this.readers = readers;
+      this.schema = schema;
+      if (readers.size() > 0) {
+        try {
+          currentReader = readers.remove(0).getRecordIterator(schema);
+        } catch (Exception e) {
+          throw new HoodieException(e);
+        }
+      }
+    }
+
+    @Override
+    public boolean hasNext() {
+      if (currentReader == null) {
+        return false;
+      } else if (currentReader.hasNext()) {
+        return true;
+      } else if (readers.size() > 0) {
+        try {
+          currentReader = readers.remove(0).getRecordIterator(schema);
+          return currentReader.hasNext();
+        } catch (Exception e) {
+          throw new HoodieException("unable to initialize read with base file 
", e);
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public HoodieRecord<? extends HoodieRecordPayload> next() {
+      //GenericRecord record = currentReader.next();
+      return transform(currentReader.next());
+    }
+
+    private HoodieRecord<? extends HoodieRecordPayload> 
transform(GenericRecord record) {
+      OverwriteWithLatestAvroPayload payload = new 
OverwriteWithLatestAvroPayload(Option.of(record));
+      String key = 
record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+      String partition = 
record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
+
+      HoodieKey hoodieKey = new HoodieKey(key, partition);
+      HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, payload);
+      return hoodieRecord;
+    }
+  }
+
+  @Override
+  public JavaRDD<WriteStatus> clustering(JavaSparkContext jsc, 
HoodieClusteringPlan clusteringPlan,
+      HoodieTable hoodieTable, HoodieWriteConfig config, String 
clusteringInstantTime) throws IOException {
+    if (clusteringPlan == null || (clusteringPlan.getOperations() == null)
+        || (clusteringPlan.getOperations().isEmpty())) {
+      return jsc.emptyRDD();
+    }
+    HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
+    // Compacting is very similar to applying updates to existing file
+    HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, 
jsc.hadoopConfiguration(), metaClient);
+    List<ClusteringOperation> operations = 
clusteringPlan.getOperations().stream()
+        
.map(ClusteringOperation::convertFromAvroRecordInstance).collect(toList());
+    LOG.info("Cluster clustering " + operations + " files");
+
+    return jsc.parallelize(operations, operations.size())
+        .map(s -> clustering(table, metaClient, config, s, 
clusteringInstantTime)).flatMap(List::iterator);
+  }
+
+  private List<WriteStatus> clustering(HoodieCopyOnWriteTable 
hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
+      HoodieWriteConfig config, ClusteringOperation operation, String 
instantTime) {
+    Schema readerSchema = HoodieAvroUtils.addMetadataFields(new 
Schema.Parser().parse(config.getSchema()));
+    LOG.info("Clustering partitionPath " + operation.getPartitionPath() + " 
with base data files " + operation.getBaseFilePaths()
+        + " for commit " + instantTime);
+
+    List<String> baseFiles = operation.getBaseFilePaths().stream().map(
+        p -> new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), 
operation.getPartitionPath()), p).toString())
+        .collect(toList());
+    List<HoodieFileReader> list = baseFiles.stream().map(s -> {
+      try {
+        return 
HoodieFileReaderFactory.getFileReader(hoodieCopyOnWriteTable.getHadoopConf(), 
new Path(s));
+      } catch (IOException e) {
+        throw new HoodieException(e);
+      }
+    }).collect(toList());
+
+    BaseFileIterator baseFileIterator = new BaseFileIterator(list, 
readerSchema);
+    Iterator<List<WriteStatus>> result = 
hoodieCopyOnWriteTable.handleInsert(instantTime, operation.getPartitionPath(), 
FSUtils.createNewFileIdPfx(),
+            baseFileIterator);
+
+    Iterable<List<WriteStatus>> resultIterable = () -> result;
+    return StreamSupport.stream(resultIterable.spliterator(), 
false).flatMap(Collection::stream).peek(s -> {
+      s.getStat().setPartitionPath(operation.getPartitionPath());
+      RuntimeStats runtimeStats = new RuntimeStats();
+      s.getStat().setRuntimeStats(runtimeStats);
+    }).collect(toList());
+  }
+
+  @Override
+  public HoodieClusteringPlan generateClusteringPlan(JavaSparkContext jsc, 
HoodieTable hoodieTable,
+                                                     HoodieWriteConfig config, 
String clusteringCommitTime, Set<HoodieFileGroupId> fgIdsInPendingClusterings)
+      throws IOException {
+
+
+    totalFileSlices = new LongAccumulator();
+    jsc.sc().register(totalFileSlices);
+
+    HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
+    LOG.info("Clustering " + metaClient.getBasePath() + " with commit " + 
clusteringCommitTime);
+    List<String> partitionPaths = 
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
+        config.shouldAssumeDatePartitioning());
+
+    // filter the partition paths if needed to reduce list status
+    partitionPaths = 
config.getClusteringStrategy().filterPartitionPaths(config, partitionPaths);
+
+    if (partitionPaths.isEmpty()) {
+      // In case no partitions could be picked, return no compaction plan
+      return null;
+    }
+
+    SliceView fileSystemView = hoodieTable.getSliceView();
+    LOG.info("Clustering looking for files to cluster in " + partitionPaths + 
" partitions");
+    Set<HoodieFileGroupId> fgIdsPendingCompactions = ((SyncableFileSystemView) 
fileSystemView).getPendingCompactionOperations()
+            .map(instantTimeOpPair -> 
instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+    List<HoodieClusteringOperation> operations = 
jsc.parallelize(partitionPaths, partitionPaths.size()).map((Function<String, 
ClusteringOperation>) partitionPath -> {
+      Stream<FileSlice> fileSliceStream = 
fileSystemView.getLatestFileSlices(partitionPath);
+      List<HoodieBaseFile> baseFiles = fileSliceStream.filter(slice -> 
(!fgIdsInPendingClusterings.contains(slice.getFileGroupId()) && 
!fgIdsPendingCompactions.contains(slice.getFileGroupId())))

Review comment:
       Some of the code seems common with Compaction handling. Any chance of 
moving to common helper functions ?




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to