[HUDI-3972] Fixing hoodie.properties/tableConfig for no preCombine field with writes (#5424)
Fixed instantiation of new table to set the null for preCombine if not explicitly set by the user.
This commit is contained in:
committed by
GitHub
parent
f2ba0fead2
commit
762623a15c
@@ -150,7 +150,9 @@ object HoodieSparkSqlWriter {
|
|||||||
.setBaseFileFormat(baseFileFormat)
|
.setBaseFileFormat(baseFileFormat)
|
||||||
.setArchiveLogFolder(archiveLogFolder)
|
.setArchiveLogFolder(archiveLogFolder)
|
||||||
.setPayloadClassName(hoodieConfig.getString(PAYLOAD_CLASS_NAME))
|
.setPayloadClassName(hoodieConfig.getString(PAYLOAD_CLASS_NAME))
|
||||||
.setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD, null))
|
// we can't fetch preCombine field from hoodieConfig object, since it falls back to "ts" as default value,
|
||||||
|
// but we are interested in what user has set, hence fetching from optParams.
|
||||||
|
.setPreCombineField(optParams.getOrElse(PRECOMBINE_FIELD.key(), null))
|
||||||
.setPartitionFields(partitionColumns)
|
.setPartitionFields(partitionColumns)
|
||||||
.setPopulateMetaFields(populateMetaFields)
|
.setPopulateMetaFields(populateMetaFields)
|
||||||
.setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD))
|
.setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD))
|
||||||
|
|||||||
@@ -96,6 +96,26 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
|||||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test def testNoPrecombine() {
|
||||||
|
// Insert Operation
|
||||||
|
val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||||
|
val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
|
||||||
|
|
||||||
|
val commonOptsNoPreCombine = Map(
|
||||||
|
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||||
|
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||||
|
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||||
|
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||||
|
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
|
||||||
|
)
|
||||||
|
inputDF.write.format("hudi")
|
||||||
|
.options(commonOptsNoPreCombine)
|
||||||
|
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||||
|
.mode(SaveMode.Overwrite)
|
||||||
|
.save(basePath)
|
||||||
|
|
||||||
|
spark.read.format("org.apache.hudi").load(basePath).count()
|
||||||
|
}
|
||||||
|
|
||||||
@Test def testHoodieIsDeletedNonBooleanField() {
|
@Test def testHoodieIsDeletedNonBooleanField() {
|
||||||
// Insert Operation
|
// Insert Operation
|
||||||
|
|||||||
@@ -489,6 +489,28 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
|
|||||||
hudiSnapshotDF2.show(1)
|
hudiSnapshotDF2.show(1)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test def testNoPrecombine() {
|
||||||
|
// Insert Operation
|
||||||
|
val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||||
|
val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
|
||||||
|
|
||||||
|
val commonOptsNoPreCombine = Map(
|
||||||
|
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||||
|
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||||
|
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||||
|
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||||
|
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
|
||||||
|
)
|
||||||
|
inputDF.write.format("hudi")
|
||||||
|
.options(commonOptsNoPreCombine)
|
||||||
|
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||||
|
.option(DataSourceWriteOptions.TABLE_TYPE.key(), "MERGE_ON_READ")
|
||||||
|
.mode(SaveMode.Overwrite)
|
||||||
|
.save(basePath)
|
||||||
|
|
||||||
|
spark.read.format("org.apache.hudi").load(basePath).count()
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testPreCombineFiledForReadMOR(): Unit = {
|
def testPreCombineFiledForReadMOR(): Unit = {
|
||||||
writeData((1, "a0", 10, 100, false))
|
writeData((1, "a0", 10, 100, false))
|
||||||
|
|||||||
@@ -263,12 +263,16 @@ class TestCreateTable extends TestHoodieSqlBase {
|
|||||||
|
|
||||||
test("Test Create Table As Select") {
|
test("Test Create Table As Select") {
|
||||||
withTempDir { tmp =>
|
withTempDir { tmp =>
|
||||||
|
Seq("cow", "mor").foreach { tableType =>
|
||||||
// Create Non-Partitioned table
|
// Create Non-Partitioned table
|
||||||
val tableName1 = generateTableName
|
val tableName1 = generateTableName
|
||||||
spark.sql(
|
spark.sql(
|
||||||
s"""
|
s"""
|
||||||
| create table $tableName1 using hudi
|
| create table $tableName1 using hudi
|
||||||
| tblproperties(primaryKey = 'id')
|
| tblproperties(
|
||||||
|
| primaryKey = 'id',
|
||||||
|
| type = '$tableType'
|
||||||
|
| )
|
||||||
| location '${tmp.getCanonicalPath}/$tableName1'
|
| location '${tmp.getCanonicalPath}/$tableName1'
|
||||||
| AS
|
| AS
|
||||||
| select 1 as id, 'a1' as name, 10 as price, 1000 as ts
|
| select 1 as id, 'a1' as name, 10 as price, 1000 as ts
|
||||||
@@ -283,7 +287,10 @@ class TestCreateTable extends TestHoodieSqlBase {
|
|||||||
s"""
|
s"""
|
||||||
| create table $tableName2 using hudi
|
| create table $tableName2 using hudi
|
||||||
| partitioned by (dt)
|
| partitioned by (dt)
|
||||||
| tblproperties(primaryKey = 'id')
|
| tblproperties(
|
||||||
|
| primaryKey = 'id',
|
||||||
|
| type = '$tableType'
|
||||||
|
| )
|
||||||
| location '${tmp.getCanonicalPath}/$tableName2'
|
| location '${tmp.getCanonicalPath}/$tableName2'
|
||||||
| AS
|
| AS
|
||||||
| select 1 as id, 'a1' as name, 10 as price, '2021-04-01' as dt
|
| select 1 as id, 'a1' as name, 10 as price, '2021-04-01' as dt
|
||||||
@@ -301,7 +308,10 @@ class TestCreateTable extends TestHoodieSqlBase {
|
|||||||
s"""
|
s"""
|
||||||
| create table $tableName3 using hudi
|
| create table $tableName3 using hudi
|
||||||
| partitioned by (dt)
|
| partitioned by (dt)
|
||||||
| tblproperties(primaryKey = 'id')
|
| tblproperties(
|
||||||
|
| primaryKey = 'id',
|
||||||
|
| type = '$tableType'
|
||||||
|
| )
|
||||||
| location '${tmp.getCanonicalPath}/$tableName3'
|
| location '${tmp.getCanonicalPath}/$tableName3'
|
||||||
| AS
|
| AS
|
||||||
| select null as id, 'a1' as name, 10 as price, '2021-05-07' as dt
|
| select null as id, 'a1' as name, 10 as price, '2021-05-07' as dt
|
||||||
@@ -314,7 +324,10 @@ class TestCreateTable extends TestHoodieSqlBase {
|
|||||||
s"""
|
s"""
|
||||||
| create table $tableName3 using hudi
|
| create table $tableName3 using hudi
|
||||||
| partitioned by (dt)
|
| partitioned by (dt)
|
||||||
| tblproperties(primaryKey = 'id')
|
| tblproperties(
|
||||||
|
| primaryKey = 'id',
|
||||||
|
| type = '$tableType'
|
||||||
|
| )
|
||||||
| location '${tmp.getCanonicalPath}/$tableName3'
|
| location '${tmp.getCanonicalPath}/$tableName3'
|
||||||
| AS
|
| AS
|
||||||
| select cast('2021-05-06 00:00:00' as timestamp) as dt, 1 as id, 'a1' as name, 10 as
|
| select cast('2021-05-06 00:00:00' as timestamp) as dt, 1 as id, 'a1' as name, 10 as
|
||||||
@@ -330,7 +343,10 @@ class TestCreateTable extends TestHoodieSqlBase {
|
|||||||
s"""
|
s"""
|
||||||
| create table $tableName4 using hudi
|
| create table $tableName4 using hudi
|
||||||
| partitioned by (dt)
|
| partitioned by (dt)
|
||||||
| tblproperties(primaryKey = 'id')
|
| tblproperties(
|
||||||
|
| primaryKey = 'id',
|
||||||
|
| type = '$tableType'
|
||||||
|
| )
|
||||||
| location '${tmp.getCanonicalPath}/$tableName4'
|
| location '${tmp.getCanonicalPath}/$tableName4'
|
||||||
| AS
|
| AS
|
||||||
| select cast('2021-05-06' as date) as dt, 1 as id, 'a1' as name, 10 as
|
| select cast('2021-05-06' as date) as dt, 1 as id, 'a1' as name, 10 as
|
||||||
@@ -342,6 +358,7 @@ class TestCreateTable extends TestHoodieSqlBase {
|
|||||||
)
|
)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
test("Test Create Table As Select when 'spark.sql.datetime.java8API.enabled' enables") {
|
test("Test Create Table As Select when 'spark.sql.datetime.java8API.enabled' enables") {
|
||||||
try {
|
try {
|
||||||
|
|||||||
Reference in New Issue
Block a user