[HUDI-2345] Hoodie columns sort partitioner for bulk insert (#3523)
Co-authored-by: yuezhang <yuezhang@freewheel.tv>
This commit is contained in:
@@ -26,6 +26,7 @@ import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.avro.Conversions;
|
||||
@@ -74,6 +75,24 @@ public class TestDataSourceUtils {
|
||||
private ArgumentCaptor<Option> optionCaptor;
|
||||
private HoodieWriteConfig config;
|
||||
|
||||
// There are fields event_date1, event_date2, event_date3 with logical type as Date. event_date1 & event_date3 are
|
||||
// of UNION schema type, which is a union of null and date type in different orders. event_date2 is non-union
|
||||
// date type. event_cost1, event_cost2, event3 are decimal logical types with UNION schema, which is similar to
|
||||
// the event_date.
|
||||
private String avroSchemaString = "{\"type\": \"record\"," + "\"name\": \"events\"," + "\"fields\": [ "
|
||||
+ "{\"name\": \"event_date1\", \"type\" : [{\"type\" : \"int\", \"logicalType\" : \"date\"}, \"null\"]},"
|
||||
+ "{\"name\": \"event_date2\", \"type\" : {\"type\": \"int\", \"logicalType\" : \"date\"}},"
|
||||
+ "{\"name\": \"event_date3\", \"type\" : [\"null\", {\"type\" : \"int\", \"logicalType\" : \"date\"}]},"
|
||||
+ "{\"name\": \"event_name\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"event_organizer\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"event_cost1\", \"type\": "
|
||||
+ "[{\"type\": \"fixed\", \"name\": \"dc\", \"size\": 5, \"logicalType\": \"decimal\", \"precision\": 10, \"scale\": 6}, \"null\"]},"
|
||||
+ "{\"name\": \"event_cost2\", \"type\": "
|
||||
+ "{\"type\": \"fixed\", \"name\": \"ef\", \"size\": 5, \"logicalType\": \"decimal\", \"precision\": 10, \"scale\": 6}},"
|
||||
+ "{\"name\": \"event_cost3\", \"type\": "
|
||||
+ "[\"null\", {\"type\": \"fixed\", \"name\": \"fg\", \"size\": 5, \"logicalType\": \"decimal\", \"precision\": 10, \"scale\": 6}]}"
|
||||
+ "]}";
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() {
|
||||
config = HoodieWriteConfig.newBuilder().withPath("/").build();
|
||||
@@ -81,23 +100,6 @@ public class TestDataSourceUtils {
|
||||
|
||||
@Test
|
||||
public void testAvroRecordsFieldConversion() {
|
||||
// There are fields event_date1, event_date2, event_date3 with logical type as Date. event_date1 & event_date3 are
|
||||
// of UNION schema type, which is a union of null and date type in different orders. event_date2 is non-union
|
||||
// date type. event_cost1, event_cost2, event3 are decimal logical types with UNION schema, which is similar to
|
||||
// the event_date.
|
||||
String avroSchemaString = "{\"type\": \"record\"," + "\"name\": \"events\"," + "\"fields\": [ "
|
||||
+ "{\"name\": \"event_date1\", \"type\" : [{\"type\" : \"int\", \"logicalType\" : \"date\"}, \"null\"]},"
|
||||
+ "{\"name\": \"event_date2\", \"type\" : {\"type\": \"int\", \"logicalType\" : \"date\"}},"
|
||||
+ "{\"name\": \"event_date3\", \"type\" : [\"null\", {\"type\" : \"int\", \"logicalType\" : \"date\"}]},"
|
||||
+ "{\"name\": \"event_name\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"event_organizer\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"event_cost1\", \"type\": "
|
||||
+ "[{\"type\": \"fixed\", \"name\": \"dc\", \"size\": 5, \"logicalType\": \"decimal\", \"precision\": 10, \"scale\": 6}, \"null\"]},"
|
||||
+ "{\"name\": \"event_cost2\", \"type\": "
|
||||
+ "{\"type\": \"fixed\", \"name\": \"ef\", \"size\": 5, \"logicalType\": \"decimal\", \"precision\": 10, \"scale\": 6}},"
|
||||
+ "{\"name\": \"event_cost3\", \"type\": "
|
||||
+ "[\"null\", {\"type\": \"fixed\", \"name\": \"fg\", \"size\": 5, \"logicalType\": \"decimal\", \"precision\": 10, \"scale\": 6}]}"
|
||||
+ "]}";
|
||||
|
||||
Schema avroSchema = new Schema.Parser().parse(avroSchemaString);
|
||||
GenericRecord record = new GenericData.Record(avroSchema);
|
||||
@@ -183,6 +185,20 @@ public class TestDataSourceUtils {
|
||||
assertThat(partitioner.isPresent(), is(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateRDDCustomColumnsSortPartitionerWithValidPartitioner() throws HoodieException {
|
||||
config = HoodieWriteConfig
|
||||
.newBuilder()
|
||||
.withPath("/")
|
||||
.withUserDefinedBulkInsertPartitionerClass(RDDCustomColumnsSortPartitioner.class.getName())
|
||||
.withUserDefinedBulkInsertPartitionerSortColumns("column1, column2")
|
||||
.withSchema(avroSchemaString)
|
||||
.build();
|
||||
|
||||
Option<BulkInsertPartitioner<Dataset<Row>>> partitioner = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(config);
|
||||
assertThat(partitioner.isPresent(), is(true));
|
||||
}
|
||||
|
||||
private void setAndVerifyHoodieWriteClientWith(final String partitionerClassName) {
|
||||
config = HoodieWriteConfig.newBuilder().withPath(config.getBasePath())
|
||||
.withUserDefinedBulkInsertPartitionerClass(partitionerClassName)
|
||||
@@ -195,6 +211,8 @@ public class TestDataSourceUtils {
|
||||
public static class NoOpBulkInsertPartitioner<T extends HoodieRecordPayload>
|
||||
implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
|
||||
|
||||
public NoOpBulkInsertPartitioner(HoodieWriteConfig config) {}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions) {
|
||||
return records;
|
||||
@@ -209,6 +227,8 @@ public class TestDataSourceUtils {
|
||||
public static class NoOpBulkInsertPartitionerRows
|
||||
implements BulkInsertPartitioner<Dataset<Row>> {
|
||||
|
||||
public NoOpBulkInsertPartitionerRows(HoodieWriteConfig config) {}
|
||||
|
||||
@Override
|
||||
public Dataset<Row> repartitionRecords(Dataset<Row> records, int outputSparkPartitions) {
|
||||
return records;
|
||||
|
||||
Reference in New Issue
Block a user