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 2020/07/21 22:05:46 UTC

[GitHub] [hudi] yihua commented on a change in pull request #1149: [HUDI-472] Introduce configurations and new modes of sorting for bulk_insert

yihua commented on a change in pull request #1149:
URL: https://github.com/apache/hudi/pull/1149#discussion_r458381933



##########
File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -245,6 +250,16 @@ public int getMaxConsistencyCheckIntervalMs() {
     return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
   }
 
+  public BulkInsertSortMode getBulkInsertSortMode() {
+    String sortMode = props.getProperty(BULKINSERT_SORT_MODE);
+    try {
+      return BulkInsertSortMode.valueOf(sortMode.toUpperCase());
+    } catch (IllegalArgumentException e) {

Review comment:
       Makes sense.

##########
File path: hudi-client/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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;
+
+import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.execution.LazyInsertIterable.HoodieInsertValueGenResult;
+import org.apache.hudi.io.HoodieWriteHandle;
+import org.apache.hudi.io.WriteHandleFactory;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Consumes stream of hoodie records from in-memory queue and writes to one or more create-handles.
+ */
+public class CopyOnWriteInsertHandler<T extends HoodieRecordPayload>
+    extends

Review comment:
       Fixed.  It's due to my IDE's hard wrap at 100 chars.  

##########
File path: hudi-client/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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;
+
+import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.execution.LazyInsertIterable.HoodieInsertValueGenResult;
+import org.apache.hudi.io.HoodieWriteHandle;
+import org.apache.hudi.io.WriteHandleFactory;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Consumes stream of hoodie records from in-memory queue and writes to one or more create-handles.
+ */
+public class CopyOnWriteInsertHandler<T extends HoodieRecordPayload>
+    extends
+    BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> {
+
+  protected HoodieWriteConfig config;
+  protected String instantTime;
+  protected HoodieTable<T> hoodieTable;
+  protected String idPrefix;
+  protected int numFilesWritten;
+  protected SparkTaskContextSupplier sparkTaskContextSupplier;
+  protected WriteHandleFactory<T> writeHandleFactory;
+
+  protected final List<WriteStatus> statuses = new ArrayList<>();
+  protected Map<String, HoodieWriteHandle> handles = new HashMap<>();
+
+  public CopyOnWriteInsertHandler(
+      HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable, String idPrefix,
+      SparkTaskContextSupplier sparkTaskContextSupplier, WriteHandleFactory<T> writeHandleFactory) {
+    this.config = config;
+    this.instantTime = instantTime;
+    this.hoodieTable = hoodieTable;
+    this.idPrefix = idPrefix;
+    this.numFilesWritten = 0;
+    this.sparkTaskContextSupplier = sparkTaskContextSupplier;
+    this.writeHandleFactory = writeHandleFactory;
+  }
+
+  @Override
+  public void consumeOneRecord(HoodieInsertValueGenResult<HoodieRecord> payload) {
+    final HoodieRecord insertPayload = payload.record;
+    String partitionPath = insertPayload.getPartitionPath();
+    HoodieWriteHandle handle = handles.get(partitionPath);
+    // lazily initialize the handle, for the first time
+    if (handle == null) {
+      handle = writeHandleFactory.create(
+          config, instantTime, hoodieTable, insertPayload.getPartitionPath(),
+          idPrefix, sparkTaskContextSupplier);
+      handles.put(partitionPath, handle);
+    }
+
+    if (handle.canWrite(payload.record)) {
+      // write the payload, if the handle has capacity
+      handle.write(insertPayload, payload.insertValue, payload.exception);
+    } else {
+      // handle is full.
+      statuses.add(handle.close());
+      // Need to handle the rejected payload & open new handle
+      handle = writeHandleFactory.create(
+          config, instantTime, hoodieTable, insertPayload.getPartitionPath(),
+          idPrefix, sparkTaskContextSupplier);
+      handles.put(partitionPath, handle);
+      handle.write(insertPayload, payload.insertValue,
+          payload.exception); // we should be able to write 1 payload.
+    }
+  }
+
+  @Override
+  public void finish() {
+    for (HoodieWriteHandle handle : handles.values()) {

Review comment:
       So the `HoodieWriteHandle` instances stored in the `handles` mapping are all open.  If you look at the L84-92, after one handle is closed, it is added to the `statuses` and removed from the mapping by replacing it with the newly opened handle.
   
   And I assume that we should return all `WriteStatus` instances from all handles in this case, whether it is closed in the middle or not.

##########
File path: hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunctionForNonSortedRecords.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.execution.CopyOnWriteInsertHandler;
+import org.apache.hudi.execution.LazyInsertIterable.HoodieInsertValueGenResult;
+import org.apache.hudi.io.CreateHandleFactory;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+public class BulkInsertMapFunctionForNonSortedRecords<T extends HoodieRecordPayload>
+    extends BulkInsertMapFunction<T> {
+
+  Map<String, CopyOnWriteInsertHandler> parallelWritersMap;

Review comment:
       Fixed.

##########
File path: hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionRangePartitioner.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+
+import org.apache.spark.RangePartitioner;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+import scala.Tuple2;
+import scala.math.Ordering;
+import scala.math.Ordering$;
+import scala.reflect.ClassTag;
+import scala.reflect.ClassTag$;
+
+public class RDDPartitionRangePartitioner<T extends HoodieRecordPayload>
+    extends BulkInsertInternalPartitioner<T> implements Serializable {
+  @Override
+  public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
+      int outputSparkPartitions) {
+    JavaPairRDD<String, HoodieRecord<T>> pairRDD = records.mapToPair(record ->
+        new Tuple2(
+            new StringBuilder()
+                .append(record.getPartitionPath())
+                .append("+")
+                .append(record.getRecordKey())
+                .toString(), record));
+    Ordering<String> ordering = Ordering$.MODULE$

Review comment:
       `RangePartitioner` is a scala class so if we have to use it, we have to import it.  In this case, shall we avoid this mode of sorting?

##########
File path: hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertHelper.java
##########
@@ -56,31 +59,50 @@
 
     final JavaRDD<HoodieRecord<T>> repartitionedRecords;
     final int parallelism = config.getBulkInsertShuffleParallelism();
+    boolean arePartitionRecordsSorted = true;
     if (bulkInsertPartitioner.isPresent()) {
-      repartitionedRecords = bulkInsertPartitioner.get().repartitionRecords(dedupedRecords, parallelism);
+      repartitionedRecords = bulkInsertPartitioner.get()
+          .repartitionRecords(dedupedRecords, parallelism);
+      arePartitionRecordsSorted = bulkInsertPartitioner.get().arePartitionRecordsSorted();
     } else {
-      // Now, sort the records and line them up nicely for loading.
-      repartitionedRecords = dedupedRecords.sortBy(record -> {
-        // Let's use "partitionPath + key" as the sort key. Spark, will ensure
-        // the records split evenly across RDD partitions, such that small partitions fit
-        // into 1 RDD partition, while big ones spread evenly across multiple RDD partitions
-        return String.format("%s+%s", record.getPartitionPath(), record.getRecordKey());
-      }, true, parallelism);
+      BulkInsertInternalPartitioner partitioner =
+          BulkInsertInternalPartitioner.get(config.getBulkInsertSortMode());
+      repartitionedRecords = partitioner.repartitionRecords(dedupedRecords, parallelism);
+      arePartitionRecordsSorted = partitioner.arePartitionRecordsSorted();

Review comment:
       Right, simplified the logic to make it clear.

##########
File path: hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionRangePartitioner.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+
+import org.apache.spark.RangePartitioner;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+import scala.Tuple2;
+import scala.math.Ordering;
+import scala.math.Ordering$;
+import scala.reflect.ClassTag;
+import scala.reflect.ClassTag$;
+
+public class RDDPartitionRangePartitioner<T extends HoodieRecordPayload>
+    extends BulkInsertInternalPartitioner<T> implements Serializable {
+  @Override
+  public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
+      int outputSparkPartitions) {
+    JavaPairRDD<String, HoodieRecord<T>> pairRDD = records.mapToPair(record ->
+        new Tuple2(
+            new StringBuilder()
+                .append(record.getPartitionPath())
+                .append("+")
+                .append(record.getRecordKey())
+                .toString(), record));
+    Ordering<String> ordering = Ordering$.MODULE$
+        .comparatorToOrdering(Comparator.<String>naturalOrder());
+    ClassTag<String> classTag = ClassTag$.MODULE$.apply(String.class);
+    return pairRDD.partitionBy(new RangePartitioner<String, HoodieRecord<T>>(

Review comment:
       This logic sorts the RDD partition ranges so that partition ranges are mutually exclusive, but inside each RDD the data is not sorted.  Do you think we need this?  If not, I can remove this class (also getting rid of the scala imports).

##########
File path: hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionRangePartitioner.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+
+import org.apache.spark.RangePartitioner;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+import scala.Tuple2;
+import scala.math.Ordering;
+import scala.math.Ordering$;
+import scala.reflect.ClassTag;
+import scala.reflect.ClassTag$;
+
+public class RDDPartitionRangePartitioner<T extends HoodieRecordPayload>

Review comment:
       Yes.  I'll take a pass for all the classes added.




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