[HUDI-1104] Adding support for UserDefinedPartitioners and SortModes to BulkInsert with Rows (#3149)
This commit is contained in:
committed by
GitHub
parent
55ecbc662e
commit
ea9e5d0e8b
@@ -35,6 +35,8 @@ import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericFixed;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
@@ -162,6 +164,25 @@ public class TestDataSourceUtils {
|
||||
assertThat(optionCaptor.getValue().get(), is(instanceOf(NoOpBulkInsertPartitioner.class)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateUserDefinedBulkInsertPartitionerRowsWithInValidPartitioner() throws HoodieException {
|
||||
config = HoodieWriteConfig.newBuilder().withPath("/").withUserDefinedBulkInsertPartitionerClass("NonExistantUserDefinedClass").build();
|
||||
|
||||
Exception exception = assertThrows(HoodieException.class, () -> {
|
||||
DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(config);
|
||||
});
|
||||
|
||||
assertThat(exception.getMessage(), containsString("Could not create UserDefinedBulkInsertPartitionerRows"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateUserDefinedBulkInsertPartitionerRowsWithValidPartitioner() throws HoodieException {
|
||||
config = HoodieWriteConfig.newBuilder().withPath("/").withUserDefinedBulkInsertPartitionerClass(NoOpBulkInsertPartitionerRows.class.getName()).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)
|
||||
@@ -184,4 +205,18 @@ public class TestDataSourceUtils {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public static class NoOpBulkInsertPartitionerRows
|
||||
implements BulkInsertPartitioner<Dataset<Row>> {
|
||||
|
||||
@Override
|
||||
public Dataset<Row> repartitionRecords(Dataset<Row> records, int outputSparkPartitions) {
|
||||
return records;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean arePartitionRecordsSorted() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.execution.bulkinsert.NonSortPartitionerWithRows;
|
||||
import org.apache.hudi.testutils.DataSourceTestUtils;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
@@ -62,7 +63,8 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
||||
HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(getPropsAllSet()).build();
|
||||
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
|
||||
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
|
||||
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
||||
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace",
|
||||
new NonSortPartitionerWithRows());
|
||||
StructType resultSchema = result.schema();
|
||||
|
||||
assertEquals(result.count(), 10);
|
||||
@@ -117,7 +119,8 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
||||
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
|
||||
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
|
||||
try {
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
|
||||
"testNamespace", new NonSortPartitionerWithRows());
|
||||
fail("Should have thrown exception");
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
@@ -127,7 +130,8 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
||||
rows = DataSourceTestUtils.generateRandomRows(10);
|
||||
dataset = sqlContext.createDataFrame(rows, structType);
|
||||
try {
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
|
||||
"testNamespace", new NonSortPartitionerWithRows());
|
||||
fail("Should have thrown exception");
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
@@ -137,7 +141,8 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
||||
rows = DataSourceTestUtils.generateRandomRows(10);
|
||||
dataset = sqlContext.createDataFrame(rows, structType);
|
||||
try {
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
|
||||
"testNamespace", new NonSortPartitionerWithRows());
|
||||
fail("Should have thrown exception");
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
@@ -147,7 +152,8 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
||||
rows = DataSourceTestUtils.generateRandomRows(10);
|
||||
dataset = sqlContext.createDataFrame(rows, structType);
|
||||
try {
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
|
||||
"testNamespace", new NonSortPartitionerWithRows());
|
||||
fail("Should have thrown exception");
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
|
||||
@@ -29,6 +29,7 @@ import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordPayload}
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode
|
||||
import org.apache.hudi.keygen.{NonpartitionedKeyGenerator, SimpleKeyGenerator}
|
||||
import org.apache.hudi.hive.HiveSyncConfig
|
||||
import org.apache.hudi.testutils.DataSourceTestUtils
|
||||
@@ -119,9 +120,9 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
}
|
||||
}
|
||||
|
||||
List(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.foreach(tableType => {
|
||||
test("test bulk insert dataset with datasource impl for " + tableType) {
|
||||
List(BulkInsertSortMode.GLOBAL_SORT.name(), BulkInsertSortMode.NONE.name(), BulkInsertSortMode.PARTITION_SORT.name())
|
||||
.foreach(sortMode => {
|
||||
test("test_bulk_insert_for_" + sortMode) {
|
||||
initSparkContext("test_bulk_insert_datasource")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
@@ -131,7 +132,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> tableType,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY.key -> "true",
|
||||
@@ -143,7 +144,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
// generate the inserts
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val records = DataSourceTestUtils.generateRandomRows(100)
|
||||
val records = DataSourceTestUtils.generateRandomRows(1000)
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
// write to Hudi
|
||||
|
||||
Reference in New Issue
Block a user