You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/07/23 07:50:40 UTC

[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5328: [HUDI-3883] Fix Bulk Insert to repartition the dataset based on Partition Path

alexeykudinkin commented on code in PR #5328:
URL: https://github.com/apache/hudi/pull/5328#discussion_r928092517


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java:
##########
@@ -27,16 +28,18 @@
  */
 public abstract class BulkInsertInternalPartitionerFactory {
 
-  public static BulkInsertPartitioner get(BulkInsertSortMode sortMode) {
-    switch (sortMode) {
+  public static BulkInsertPartitioner get(BulkInsertSortMode bulkInsertMode, HoodieTableConfig tableConfig) {

Review Comment:
   There are modes which have nothing to do with sorting (re-partitioning, for ex), so long-term we should strip the sort part of it



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RepartitioningBulkInsertPartitionerBase.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.execution.bulkinsert;
+
+import org.apache.hudi.common.function.SerializableFunctionUnchecked;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.spark.Partitioner;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+/**
+ * Base class for any {@link BulkInsertPartitioner} implementation that does re-partitioning,
+ * to better align "logical" (query-engine's partitioning of the incoming dataset) w/ the table's
+ * "physical" partitioning
+ */
+public abstract class RepartitioningBulkInsertPartitionerBase<I> implements BulkInsertPartitioner<I> {
+
+  protected final boolean isPartitionedTable;
+
+  public RepartitioningBulkInsertPartitionerBase(HoodieTableConfig tableConfig) {

Review Comment:
   Good call. I thought about it initially, but then decided that it's better to abstract this handling w/in the partitioner rather than pushing it onto the caller. LMK what you think.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RepartitioningBulkInsertPartitionerBase.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.execution.bulkinsert;
+
+import org.apache.hudi.common.function.SerializableFunctionUnchecked;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.spark.Partitioner;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+/**
+ * Base class for any {@link BulkInsertPartitioner} implementation that does re-partitioning,
+ * to better align "logical" (query-engine's partitioning of the incoming dataset) w/ the table's
+ * "physical" partitioning
+ */
+public abstract class RepartitioningBulkInsertPartitionerBase<I> implements BulkInsertPartitioner<I> {
+
+  protected final boolean isPartitionedTable;
+
+  public RepartitioningBulkInsertPartitionerBase(HoodieTableConfig tableConfig) {
+    this.isPartitionedTable = tableConfig.getPartitionFields().map(pfs -> pfs.length > 0).orElse(false);
+  }
+
+  protected static class PartitionPathRDDPartitioner extends Partitioner implements Serializable {
+    private final SerializableFunctionUnchecked<Object, String> partitionPathExtractor;
+    private final int numPartitions;
+
+    PartitionPathRDDPartitioner(SerializableFunctionUnchecked<Object, String> partitionPathExtractor, int numPartitions) {
+      this.partitionPathExtractor = partitionPathExtractor;
+      this.numPartitions = numPartitions;
+    }
+
+    @Override
+    public int numPartitions() {
+      return numPartitions;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public int getPartition(Object o) {
+      return Math.abs(Objects.hash(partitionPathExtractor.apply(o))) % numPartitions;

Review Comment:
   Not sure i follow your train of thought: that's the whole idea of such partitioners (parittion-sort and partitioner-no-sort) to be able to partition the data to be better aligned with physical partitioning, right? 
   
   In case data is heavily skewed into most recent partition, it shouldn't be handled with this partitioner.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java:
##########
@@ -19,42 +19,69 @@
 package org.apache.hudi.execution.bulkinsert;
 
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.spark.sql.Column;
 import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.Row;
+import scala.collection.JavaConverters;
 
 import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.util.ValidationUtils.checkState;
+import static org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner.getOrderByColumnNames;
 
 /**
- * A partitioner that does sorting based on specified column values for each spark partitions.
+ * A partitioner that does local sorting for each RDD partition based on the tuple of
+ * values of the columns configured for ordering.
  */
-public class RowCustomColumnsSortPartitioner implements BulkInsertPartitioner<Dataset<Row>> {
+public class RowCustomColumnsSortPartitioner extends RepartitioningBulkInsertPartitionerBase<Dataset<Row>> {
+
+  private final String[] orderByColumnNames;
 
-  private final String[] sortColumnNames;
+  public RowCustomColumnsSortPartitioner(HoodieWriteConfig config, HoodieTableConfig tableConfig) {
+    super(tableConfig);
+    this.orderByColumnNames = getOrderByColumnNames(config);
 
-  public RowCustomColumnsSortPartitioner(HoodieWriteConfig config) {
-    this.sortColumnNames = getSortColumnName(config);
+    checkState(orderByColumnNames.length > 0);
   }
 
-  public RowCustomColumnsSortPartitioner(String[] columnNames) {
-    this.sortColumnNames = columnNames;
+  public RowCustomColumnsSortPartitioner(String[] columnNames, HoodieTableConfig tableConfig) {
+    super(tableConfig);
+    this.orderByColumnNames = columnNames;
+
+    checkState(orderByColumnNames.length > 0);
   }
 
   @Override
-  public Dataset<Row> repartitionRecords(Dataset<Row> records, int outputSparkPartitions) {
-    final String[] sortColumns = this.sortColumnNames;
-    return records.coalesce(outputSparkPartitions)
-        .sortWithinPartitions(HoodieRecord.PARTITION_PATH_METADATA_FIELD, sortColumns);
+  public Dataset<Row> repartitionRecords(Dataset<Row> dataset, int outputSparkPartitions) {
+    Dataset<Row> repartitionedDataset;
+
+    // NOTE: In case of partitioned table even "global" ordering (across all RDD partitions) could
+    //       not change table's partitioning and therefore there's no point in doing global sorting
+    //       across "physical" partitions, and instead we can reduce total amount of data being
+    //       shuffled by doing do "local" sorting:
+    //          - First, re-partitioning dataset such that "logical" partitions are aligned w/
+    //          "physical" ones
+    //          - Sorting locally w/in RDD ("logical") partitions
+    //
+    //       Non-partitioned tables will be globally sorted.
+    if (isPartitionedTable) {
+      repartitionedDataset = dataset.repartition(outputSparkPartitions, new Column(HoodieRecord.PARTITION_PATH_METADATA_FIELD));

Review Comment:
   It will if the dataset wasn't partitioned before, but it brings the benefits of the properly sized files (b/c whole physical partition will be written by a single executor)



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org