1
0

[HUDI-2345] Hoodie columns sort partitioner for bulk insert (#3523)

Co-authored-by: yuezhang <yuezhang@freewheel.tv>
This commit is contained in:
zhangyue19921010
2021-08-24 21:45:17 +08:00
committed by GitHub
parent 05e6f44d53
commit de94787a85
5 changed files with 83 additions and 21 deletions

View File

@@ -158,11 +158,18 @@ public class HoodieWriteConfig extends HoodieConfig {
.withDocumentation("For large initial imports using bulk_insert operation, controls the parallelism to use for sort modes or custom partitioning done"
+ "before writing records to the table.");
public static final ConfigProperty<String> BULKINSERT_USER_DEFINED_PARTITIONER_SORT_COLUMNS = ConfigProperty
.key("hoodie.bulkinsert.user.defined.partitioner.sort.columns")
.noDefaultValue()
.withDocumentation("Columns to sort the data by when use org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner as user defined partitioner during bulk_insert. "
+ "For example 'column1,column2'");
public static final ConfigProperty<String> BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME = ConfigProperty
.key("hoodie.bulkinsert.user.defined.partitioner.class")
.noDefaultValue()
.withDocumentation("If specified, this class will be used to re-partition records before they are bulk inserted. This can be used to sort, pack, cluster data"
+ " optimally for common query patterns.");
+ " optimally for common query patterns. For now we support a build-in user defined bulkinsert partitioner org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner"
+ " which can does sorting based on specified column values set by " + BULKINSERT_USER_DEFINED_PARTITIONER_SORT_COLUMNS.key());
public static final ConfigProperty<String> UPSERT_PARALLELISM_VALUE = ConfigProperty
.key("hoodie.upsert.shuffle.parallelism")
@@ -894,6 +901,10 @@ public class HoodieWriteConfig extends HoodieConfig {
return getString(BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME);
}
public String getUserDefinedBulkInsertPartitionerSortColumns() {
return getString(BULKINSERT_USER_DEFINED_PARTITIONER_SORT_COLUMNS);
}
public int getInsertShuffleParallelism() {
return getInt(INSERT_PARALLELISM_VALUE);
}
@@ -1832,6 +1843,11 @@ public class HoodieWriteConfig extends HoodieConfig {
return this;
}
public Builder withUserDefinedBulkInsertPartitionerSortColumns(String columns) {
writeConfig.setValue(BULKINSERT_USER_DEFINED_PARTITIONER_SORT_COLUMNS, columns);
return this;
}
public Builder withDeleteParallelism(int parallelism) {
writeConfig.setValue(DELETE_PARALLELISM_VALUE, String.valueOf(parallelism));
return this;

View File

@@ -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(",");
}
}

View File

@@ -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) {