1
0

[HUDI-1104] Adding support for UserDefinedPartitioners and SortModes to BulkInsert with Rows (#3149)

This commit is contained in:
Sivabalan Narayanan
2021-07-07 11:15:25 -04:00
committed by GitHub
parent 55ecbc662e
commit ea9e5d0e8b
31 changed files with 618 additions and 82 deletions

View File

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