1
0

[HUDI-4040] Bulk insert Support CustomColumnsSortPartitioner with Row (#5502)

* Along the lines of RDDCustomColumnsSortPartitioner but for Row
This commit is contained in:
RexAn
2022-05-26 13:09:04 +08:00
committed by GitHub
parent 4e42ed5eae
commit 98c5c6c654
3 changed files with 112 additions and 8 deletions

View File

@@ -20,6 +20,8 @@ package org.apache.hudi.execution.bulkinsert;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.SparkDatasetTestUtils;
@@ -29,6 +31,7 @@ import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
@@ -48,6 +51,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
*/
public class TestBulkInsertInternalPartitionerForRows extends HoodieClientTestHarness {
private static final Comparator<Row> KEY_COMPARATOR =
Comparator.comparing(o -> (o.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + o.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)));
@BeforeEach
public void setUp() throws Exception {
initSparkContexts("TestBulkInsertInternalPartitionerForRows");
@@ -77,21 +82,47 @@ public class TestBulkInsertInternalPartitionerForRows extends HoodieClientTestHa
Dataset<Row> records1 = generateTestRecords();
Dataset<Row> records2 = generateTestRecords();
testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerWithRowsFactory.get(sortMode),
records1, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records1));
records1, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records1), Option.empty());
testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerWithRowsFactory.get(sortMode),
records2, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records2));
records2, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records2), Option.empty());
}
@Test
public void testCustomColumnSortPartitionerWithRows() {
Dataset<Row> records1 = generateTestRecords();
Dataset<Row> records2 = generateTestRecords();
String sortColumnString = records1.columns()[5];
String[] sortColumns = sortColumnString.split(",");
Comparator<Row> comparator = getCustomColumnComparator(sortColumns);
testBulkInsertInternalPartitioner(new RowCustomColumnsSortPartitioner(sortColumns),
records1, false, true, generateExpectedPartitionNumRecords(records1), Option.of(comparator));
testBulkInsertInternalPartitioner(new RowCustomColumnsSortPartitioner(sortColumns),
records2, false, true, generateExpectedPartitionNumRecords(records2), Option.of(comparator));
HoodieWriteConfig config = HoodieWriteConfig
.newBuilder()
.withPath("/")
.withUserDefinedBulkInsertPartitionerClass(RowCustomColumnsSortPartitioner.class.getName())
.withUserDefinedBulkInsertPartitionerSortColumns(sortColumnString)
.build();
testBulkInsertInternalPartitioner(new RowCustomColumnsSortPartitioner(config),
records1, false, true, generateExpectedPartitionNumRecords(records1), Option.of(comparator));
testBulkInsertInternalPartitioner(new RowCustomColumnsSortPartitioner(config),
records2, false, true, generateExpectedPartitionNumRecords(records2), Option.of(comparator));
}
private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner,
Dataset<Row> rows,
boolean isGloballySorted, boolean isLocallySorted,
Map<String, Long> expectedPartitionNumRecords) {
Map<String, Long> expectedPartitionNumRecords,
Option<Comparator<Row>> comparator) {
int numPartitions = 2;
Dataset<Row> actualRecords = (Dataset<Row>) partitioner.repartitionRecords(rows, numPartitions);
List<Row> collectedActualRecords = actualRecords.collectAsList();
if (isGloballySorted) {
// Verify global order
verifyRowsAscendingOrder(collectedActualRecords);
verifyRowsAscendingOrder(collectedActualRecords, comparator);
} else if (isLocallySorted) {
// Verify local order
actualRecords.mapPartitions((MapPartitionsFunction<Row, Object>) input -> {
@@ -99,7 +130,7 @@ public class TestBulkInsertInternalPartitionerForRows extends HoodieClientTestHa
while (input.hasNext()) {
partitionRows.add(input.next());
}
verifyRowsAscendingOrder(partitionRows);
verifyRowsAscendingOrder(partitionRows, comparator);
return Collections.emptyList().iterator();
}, SparkDatasetTestUtils.ENCODER);
}
@@ -130,10 +161,20 @@ public class TestBulkInsertInternalPartitionerForRows extends HoodieClientTestHa
return rowsPart1.union(rowsPart2);
}
private void verifyRowsAscendingOrder(List<Row> records) {
private void verifyRowsAscendingOrder(List<Row> records, Option<Comparator<Row>> comparator) {
List<Row> expectedRecords = new ArrayList<>(records);
Collections.sort(expectedRecords, Comparator.comparing(o -> (o.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + o.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD))));
Collections.sort(expectedRecords,comparator.orElse(KEY_COMPARATOR));
assertEquals(expectedRecords, records);
}
private Comparator<Row> getCustomColumnComparator(String[] sortColumns) {
Comparator<Row> comparator = Comparator.comparing(row -> {
StringBuilder sb = new StringBuilder();
for (String col : sortColumns) {
sb.append(row.getAs(col).toString());
}
return sb.toString();
});
return comparator;
}
}