alexeykudinkin commented on code in PR #5328:
URL: https://github.com/apache/hudi/pull/5328#discussion_r930466594
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java:
##########
@@ -20,46 +20,62 @@
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.table.BulkInsertPartitioner;
-
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
+import org.apache.spark.sql.HoodieJavaRDDUtils;
import scala.Tuple2;
+import java.util.Comparator;
+
/**
* A built-in partitioner that does local sorting for each RDD partition
- * after coalesce for bulk insert operation, corresponding to the
- * {@code BulkInsertSortMode.PARTITION_SORT} mode.
+ * after coalescing it to specified number of partitions.
+ * Corresponds to the {@link BulkInsertSortMode#PARTITION_SORT} mode.
*
* @param <T> HoodieRecordPayload type
*/
public class RDDPartitionSortPartitioner<T extends HoodieRecordPayload>
- implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
+ extends RepartitioningBulkInsertPartitionerBase<JavaRDD<HoodieRecord<T>>> {
+ public RDDPartitionSortPartitioner(HoodieTableConfig tableConfig) {
+ super(tableConfig);
+ }
+
+ @SuppressWarnings("unchecked")
@Override
public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>>
records,
int
outputSparkPartitions) {
- return records.coalesce(outputSparkPartitions)
- .mapToPair(record ->
- new Tuple2<>(
- new StringBuilder()
- .append(record.getPartitionPath())
- .append("+")
- .append(record.getRecordKey())
- .toString(), record))
- .mapPartitions(partition -> {
- // Sort locally in partition
- List<Tuple2<String, HoodieRecord<T>>> recordList = new ArrayList<>();
- for (; partition.hasNext(); ) {
- recordList.add(partition.next());
- }
- Collections.sort(recordList, (o1, o2) -> o1._1.compareTo(o2._1));
- return recordList.stream().map(e -> e._2).iterator();
- });
+
+ // NOTE: Datasets being ingested into partitioned tables are additionally
re-partitioned to better
+ // align dataset's logical partitioning with expected table's
physical partitioning to
+ // provide for appropriate file-sizing and better control of the
number of files created.
+ //
+ // Please check out {@code GlobalSortPartitioner} java-doc for more
details
+ if (isPartitionedTable) {
+ PartitionPathRDDPartitioner partitioner =
+ new PartitionPathRDDPartitioner((pair) -> ((Pair<String, String>)
pair).getKey(), outputSparkPartitions);
+
+ // Both partition-path and record-key are extracted, since
+ // - Partition-path will be used for re-partitioning (as called out
above)
+ // - Record-key will be used for sorting the records w/in individual
partitions
+ return records.mapToPair(record -> new
Tuple2<>(Pair.of(record.getPartitionPath(), record.getRecordKey()), record))
+ // NOTE: We're sorting by (partition-path, record-key) pair to make
sure that in case
+ // when there are less Spark partitions (requested) than there
are physical partitions
+ // (in which case multiple physical partitions, will be
handled w/in single Spark
+ // partition) records w/in a single Spark partition are still
ordered first by
+ // partition-path, then record's key
+ .repartitionAndSortWithinPartitions(partitioner,
Comparator.naturalOrder())
+ .values();
+ } else {
+ JavaPairRDD<String, HoodieRecord<T>> kvPairsRDD =
+ records.coalesce(outputSparkPartitions).mapToPair(record -> new
Tuple2<>(record.getRecordKey(), record));
+
+ // NOTE: [[JavaRDD]] doesn't expose an API to do the sorting w/o
(re-)shuffling, as such
+ // we're relying on our own sequence to achieve that
+ return HoodieJavaRDDUtils.sortWithinPartitions(kvPairsRDD,
Comparator.naturalOrder()).values();
Review Comment:
We can't as it could spill
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java:
##########
@@ -20,46 +20,62 @@
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.table.BulkInsertPartitioner;
-
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
+import org.apache.spark.sql.HoodieJavaRDDUtils;
import scala.Tuple2;
+import java.util.Comparator;
+
/**
* A built-in partitioner that does local sorting for each RDD partition
- * after coalesce for bulk insert operation, corresponding to the
- * {@code BulkInsertSortMode.PARTITION_SORT} mode.
+ * after coalescing it to specified number of partitions.
+ * Corresponds to the {@link BulkInsertSortMode#PARTITION_SORT} mode.
*
* @param <T> HoodieRecordPayload type
*/
public class RDDPartitionSortPartitioner<T extends HoodieRecordPayload>
- implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
+ extends RepartitioningBulkInsertPartitionerBase<JavaRDD<HoodieRecord<T>>> {
+ public RDDPartitionSortPartitioner(HoodieTableConfig tableConfig) {
+ super(tableConfig);
+ }
+
+ @SuppressWarnings("unchecked")
@Override
public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>>
records,
int
outputSparkPartitions) {
- return records.coalesce(outputSparkPartitions)
- .mapToPair(record ->
- new Tuple2<>(
- new StringBuilder()
- .append(record.getPartitionPath())
- .append("+")
- .append(record.getRecordKey())
- .toString(), record))
- .mapPartitions(partition -> {
- // Sort locally in partition
- List<Tuple2<String, HoodieRecord<T>>> recordList = new ArrayList<>();
- for (; partition.hasNext(); ) {
- recordList.add(partition.next());
- }
- Collections.sort(recordList, (o1, o2) -> o1._1.compareTo(o2._1));
- return recordList.stream().map(e -> e._2).iterator();
- });
+
+ // NOTE: Datasets being ingested into partitioned tables are additionally
re-partitioned to better
+ // align dataset's logical partitioning with expected table's
physical partitioning to
+ // provide for appropriate file-sizing and better control of the
number of files created.
+ //
+ // Please check out {@code GlobalSortPartitioner} java-doc for more
details
+ if (isPartitionedTable) {
+ PartitionPathRDDPartitioner partitioner =
+ new PartitionPathRDDPartitioner((pair) -> ((Pair<String, String>)
pair).getKey(), outputSparkPartitions);
+
+ // Both partition-path and record-key are extracted, since
+ // - Partition-path will be used for re-partitioning (as called out
above)
+ // - Record-key will be used for sorting the records w/in individual
partitions
+ return records.mapToPair(record -> new
Tuple2<>(Pair.of(record.getPartitionPath(), record.getRecordKey()), record))
+ // NOTE: We're sorting by (partition-path, record-key) pair to make
sure that in case
+ // when there are less Spark partitions (requested) than there
are physical partitions
+ // (in which case multiple physical partitions, will be
handled w/in single Spark
+ // partition) records w/in a single Spark partition are still
ordered first by
+ // partition-path, then record's key
+ .repartitionAndSortWithinPartitions(partitioner,
Comparator.naturalOrder())
Review Comment:
Fair call out. What's particularly troubling from your perspective?
--
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]