[HUDI-2345] Hoodie columns sort partitioner for bulk insert (#3523)
Co-authored-by: yuezhang <yuezhang@freewheel.tv>
This commit is contained in:
@@ -24,6 +24,7 @@ import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.config.SerializableSchema;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
@@ -41,6 +42,11 @@ public class RDDCustomColumnsSortPartitioner<T extends HoodieRecordPayload>
|
||||
private final String[] sortColumnNames;
|
||||
private final SerializableSchema serializableSchema;
|
||||
|
||||
public RDDCustomColumnsSortPartitioner(HoodieWriteConfig config) {
|
||||
this.serializableSchema = new SerializableSchema(new Schema.Parser().parse(config.getSchema()));
|
||||
this.sortColumnNames = getSortColumnName(config);
|
||||
}
|
||||
|
||||
public RDDCustomColumnsSortPartitioner(String[] columnNames, Schema schema) {
|
||||
this.sortColumnNames = columnNames;
|
||||
this.serializableSchema = new SerializableSchema(schema);
|
||||
@@ -79,4 +85,8 @@ public class RDDCustomColumnsSortPartitioner<T extends HoodieRecordPayload>
|
||||
throw new HoodieIOException("Unable to read record with key:" + record.getKey(), e);
|
||||
}
|
||||
}
|
||||
|
||||
private String[] getSortColumnName(HoodieWriteConfig config) {
|
||||
return config.getUserDefinedBulkInsertPartitionerSortColumns().split(",");
|
||||
}
|
||||
}
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
@@ -43,6 +44,7 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestBulkInsertInternalPartitioner extends HoodieClientTestBase {
|
||||
@@ -139,7 +141,8 @@ public class TestBulkInsertInternalPartitioner extends HoodieClientTestBase {
|
||||
|
||||
@Test
|
||||
public void testCustomColumnSortPartitioner() throws Exception {
|
||||
String[] sortColumns = new String[] {"rider"};
|
||||
String sortColumnString = "rider";
|
||||
String[] sortColumns = sortColumnString.split(",");
|
||||
Comparator<HoodieRecord<? extends HoodieRecordPayload>> columnComparator = getCustomColumnComparator(HoodieTestDataGenerator.AVRO_SCHEMA, sortColumns);
|
||||
|
||||
JavaRDD<HoodieRecord> records1 = generateTestRecordsForBulkInsert(jsc);
|
||||
@@ -148,6 +151,19 @@ public class TestBulkInsertInternalPartitioner extends HoodieClientTestBase {
|
||||
records1, true, true, generateExpectedPartitionNumRecords(records1), Option.of(columnComparator));
|
||||
testBulkInsertInternalPartitioner(new RDDCustomColumnsSortPartitioner(sortColumns, HoodieTestDataGenerator.AVRO_SCHEMA),
|
||||
records2, true, true, generateExpectedPartitionNumRecords(records2), Option.of(columnComparator));
|
||||
|
||||
HoodieWriteConfig config = HoodieWriteConfig
|
||||
.newBuilder()
|
||||
.withPath("/")
|
||||
.withSchema(TRIP_EXAMPLE_SCHEMA)
|
||||
.withUserDefinedBulkInsertPartitionerClass(RDDCustomColumnsSortPartitioner.class.getName())
|
||||
.withUserDefinedBulkInsertPartitionerSortColumns(sortColumnString)
|
||||
.build();
|
||||
testBulkInsertInternalPartitioner(new RDDCustomColumnsSortPartitioner(config),
|
||||
records1, true, true, generateExpectedPartitionNumRecords(records1), Option.of(columnComparator));
|
||||
testBulkInsertInternalPartitioner(new RDDCustomColumnsSortPartitioner(config),
|
||||
records2, true, true, generateExpectedPartitionNumRecords(records2), Option.of(columnComparator));
|
||||
|
||||
}
|
||||
|
||||
private Comparator<HoodieRecord<? extends HoodieRecordPayload>> getCustomColumnComparator(Schema schema, String[] sortColumns) {
|
||||
|
||||
Reference in New Issue
Block a user