bhat-vinay commented on code in PR #10876:
URL: https://github.com/apache/hudi/pull/10876#discussion_r1535751978


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java:
##########
@@ -411,4 +427,90 @@ public Partitioner getLayoutPartitioner(WorkloadProfile 
profile, String layoutPa
   protected void 
runPrecommitValidators(HoodieWriteMetadata<HoodieData<WriteStatus>> 
writeMetadata) {
     SparkValidatorUtils.runValidators(config, writeMetadata, context, table, 
instantTime);
   }
+
+  private HoodieData<WriteStatus> 
sortAndMapPartitionsAsRDD(HoodieData<HoodieRecord<T>> dedupedRecords, 
Partitioner partitioner) {
+    JavaPairRDD<Tuple2<HoodieKey, Long>, HoodieRecord<T>> mappedRDD = 
getSortedIndexedRecords(dedupedRecords);
+    JavaPairRDD<Tuple2<HoodieKey, Long>, HoodieRecord<T>> partitionedRDD;
+    if (table.requireSortedRecords()) {
+      // Partition and sort within each partition as a single step. This is 
faster than partitioning first and then
+      // applying a sort.
+      Comparator<Tuple2<HoodieKey, Long>> comparator = 
(Comparator<Tuple2<HoodieKey, Long>> & Serializable) (t1, t2) -> {
+        HoodieKey key1 = t1._1();
+        HoodieKey key2 = t2._1();
+        return key1.getRecordKey().compareTo(key2.getRecordKey());
+      };
+      partitionedRDD = 
mappedRDD.repartitionAndSortWithinPartitions(partitioner, comparator);
+    } else {
+      // Partition only
+      partitionedRDD = mappedRDD.partitionBy(partitioner);
+    }
+
+    return 
HoodieJavaRDD.of(partitionedRDD.map(Tuple2::_2).mapPartitionsWithIndex((partition,
 recordItr) -> {
+      if (WriteOperationType.isChangingRecords(operationType)) {
+        return handleUpsertPartition(instantTime, partition, recordItr, 
partitioner);
+      } else {
+        return handleInsertPartition(instantTime, partition, recordItr, 
partitioner);
+      }
+    }, true).flatMap(List::iterator));
+  }
+
+  private boolean operationRequiresSorting() {
+    return operationType == WriteOperationType.INSERT && 
config.getBoolean(INSERT_SORT);
+  }
+
+  private JavaPairRDD<Tuple2<HoodieKey, Long>, HoodieRecord<T>> 
getSortedIndexedRecords(HoodieData<HoodieRecord<T>> dedupedRecords) {
+    // Get any user specified sort columns
+    String customSortColField = 
config.getString(INSERT_USER_DEFINED_SORT_COLUMNS);
+
+    String[] sortColumns;
+    if (!isNullOrEmpty(customSortColField)) {
+      // Extract user specified sort-column fields as an array
+      sortColumns = Arrays.stream(customSortColField.split(","))
+          .map(String::trim).toArray(String[]::new);
+    } else {
+      // Use record-key as sort column
+      sortColumns = 
Arrays.stream(HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD.getFieldName().split(","))
+          .map(String::trim).toArray(String[]::new);
+    }
+
+    // Get the record's schema from the write config
+    SerializableSchema serializableSchema = new SerializableSchema(new 
Schema.Parser().parse(config.getSchema()));
+
+    JavaRDD<HoodieRecord<T>> javaRdd = 
HoodieJavaRDD.getJavaRDD(dedupedRecords);
+    JavaRDD<HoodieRecord<T>> sortedRecords = javaRdd.sortBy(record -> {

Review Comment:
   My understanding is that `repartitionAndSortWithinPartitions` is to sort 
within a bucket (or a Spark RDD partition) after UpsertPartitioner has already 
partitioned the input batch. It is for handling the case of writing sorted 
key-values to file with file formats that depend on it (ex : HFile). I am not 
sure how partitioning first and then sorting within that partition will be 
useful.



-- 
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-unsubscr...@hudi.apache.org

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

Reply via email to