codope commented on a change in pull request #3857:
URL: https://github.com/apache/hudi/pull/3857#discussion_r751809311



##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java
##########
@@ -430,9 +446,37 @@ public Builder withDataOptimizeDataSkippingEnable(boolean 
dataSkipping) {
     }
 
     public HoodieClusteringConfig build() {
+      clusteringConfig.setDefaultValue(
+          PLAN_STRATEGY_CLASS_NAME, 
getDefaultPlanStrategyClassName(engineType));
+      clusteringConfig.setDefaultValue(
+          EXECUTION_STRATEGY_CLASS_NAME, 
getDefaultExecutionStrategyClassName(engineType));
       clusteringConfig.setDefaults(HoodieClusteringConfig.class.getName());
       return clusteringConfig;
     }
+
+    private String getDefaultPlanStrategyClassName(EngineType engineType) {
+      switch (engineType) {
+        case SPARK:
+          return SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY;
+        case FLINK:
+        case JAVA:
+          return JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY;

Review comment:
       For `FLINK` also, is it going to be java based strategy?

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java
##########
@@ -59,15 +69,15 @@
 
   public static final ConfigProperty<String> PLAN_STRATEGY_CLASS_NAME = 
ConfigProperty
       .key("hoodie.clustering.plan.strategy.class")
-      
.defaultValue("org.apache.hudi.client.clustering.plan.strategy.SparkSizeBasedClusteringPlanStrategy")
+      .noDefaultValue()

Review comment:
       This will affect documentation. With this change, it would appear in the 
docs that Hudi is not setting any default value but in fact it is in the 
`build()` method. Either, we make sure that docs change is also done. 
   Or, another way is to localize the change of defaults. Can it be done by 
overriding the `getWriteConfig()` method in 
`ClusteringPlanStrategy/ClusteringExecutionStrategy`? That way, we may not need 
`engineType` setting as well.

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##########
@@ -99,6 +99,12 @@
       .withDocumentation("Controls how compaction scheduling is triggered, by 
time or num delta commits or combination of both. "
           + "Valid options: " + 
Arrays.stream(CompactionTriggerStrategy.values()).map(Enum::name).collect(Collectors.joining(",")));
 
+  public static final ConfigProperty<String> ASYNC_COMPACT_ENABLE = 
ConfigProperty

Review comment:
       We should check how this new config affects hthe default behavior of 
async compaction in `HoodieDeltaStreamer`  and also whether it is in sync with 
`DataSourceWriteOptions`. 

##########
File path: 
hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.strategy;
+
+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.WriteStatus;
+import org.apache.hudi.client.common.JavaTaskContextSupplier;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.ClusteringOperation;
+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.RewriteAvroPayload;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+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.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.KeyGenUtils;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import 
org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import static 
org.apache.hudi.common.table.log.HoodieFileSliceReader.getFileSliceReader;
+
+/**
+ * Clustering strategy for Java engine.
+ */
+public abstract class JavaExecutionStrategy<T extends HoodieRecordPayload<T>>
+    extends ClusteringExecutionStrategy<T, List<HoodieRecord<T>>, 
List<HoodieKey>, List<WriteStatus>> {
+
+  private static final Logger LOG = 
LogManager.getLogger(JavaExecutionStrategy.class);
+
+  public JavaExecutionStrategy(
+      HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig 
writeConfig) {
+    super(table, engineContext, writeConfig);
+  }
+
+  @Override
+  public HoodieWriteMetadata<List<WriteStatus>> performClustering(
+      HoodieClusteringPlan clusteringPlan, Schema schema, String instantTime) {
+    // execute clustering for each group and collect WriteStatus
+    List<WriteStatus> writeStatusList = new ArrayList<>();
+    clusteringPlan.getInputGroups().forEach(
+        inputGroup -> writeStatusList.addAll(runClusteringForGroup(
+            inputGroup, clusteringPlan.getStrategy().getStrategyParams(),
+            
Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false),
+            instantTime)));
+    HoodieWriteMetadata<List<WriteStatus>> writeMetadata = new 
HoodieWriteMetadata<>();
+    writeMetadata.setWriteStatuses(writeStatusList);
+    return writeMetadata;
+  }
+
+  /**
+   * Execute clustering to write inputRecords into new files as defined by 
rules in strategy parameters.
+   * The number of new file groups created is bounded by numOutputGroups.
+   * Note that commit is not done as part of strategy. commit is callers 
responsibility.
+   *
+   * @param inputRecords           List of {@link HoodieRecord}.
+   * @param numOutputGroups        Number of output file groups.
+   * @param instantTime            Clustering (replace commit) instant time.
+   * @param strategyParams         Strategy parameters containing columns to 
sort the data by when clustering.
+   * @param schema                 Schema of the data including metadata 
fields.
+   * @param fileGroupIdList        File group id corresponding to each out 
group.
+   * @param preserveHoodieMetadata Whether to preserve commit metadata while 
clustering.
+   * @return List of {@link WriteStatus}.
+   */
+  public abstract List<WriteStatus> performClusteringWithRecordList(
+      final List<HoodieRecord<T>> inputRecords, final int numOutputGroups, 
final String instantTime,
+      final Map<String, String> strategyParams, final Schema schema,
+      final List<HoodieFileGroupId> fileGroupIdList, final boolean 
preserveHoodieMetadata);
+
+  /**
+   * Create {@link BulkInsertPartitioner} based on strategy params.
+   *
+   * @param strategyParams Strategy parameters containing columns to sort the 
data by when clustering.
+   * @param schema         Schema of the data including metadata fields.
+   * @return empty for now.
+   */
+  protected Option<BulkInsertPartitioner<T>> getPartitioner(Map<String, 
String> strategyParams, Schema schema) {
+    return Option.empty();

Review comment:
       `LAYOUT_OPTIMIZE_ENABLE` is for z-order clustering. We need to add a new 
partitioner for java client. I think we can take it up as a follow-up task to 
extend z-order clustering in java. However, we can still add a custom java sort 
partitioner based on columns. We already have global sort partitioner: 
`JavaGlobalSortPartitioner`. Something on similar lines but incorporating the 
logic of `RDDCustomColumnsSortPartitioner#getRecordSortColumnValues`. 




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

To unsubscribe, e-mail: [email protected]

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


Reply via email to