1
0

[HUDI-2811] Support Spark 3.2 (#4270)

This commit is contained in:
Yann Byron
2021-12-28 16:12:44 +08:00
committed by GitHub
parent 32505d5adb
commit 05942e018c
36 changed files with 596 additions and 167 deletions

View File

@@ -19,6 +19,7 @@ package org.apache.hudi
import org.apache.hudi.HoodieSparkUtils.convertToCatalystExpressions
import org.apache.hudi.HoodieSparkUtils.convertToCatalystExpression
import org.apache.spark.sql.sources.{And, EqualNullSafe, EqualTo, Filter, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, LessThan, LessThanOrEqual, Not, Or, StringContains, StringEndsWith, StringStartsWith}
import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType}
import org.junit.jupiter.api.Assertions.assertEquals
@@ -68,22 +69,36 @@ class TestConvertFilterToCatalystExpression {
}
private def checkConvertFilter(filter: Filter, expectExpression: String): Unit = {
// [SPARK-25769][SPARK-34636][SPARK-34626][SQL] sql method in UnresolvedAttribute,
// AttributeReference and Alias don't quote qualified names properly
val removeQuotesIfNeed = if (expectExpression != null && HoodieSparkUtils.isSpark3_2) {
expectExpression.replace("`", "")
} else {
expectExpression
}
val exp = convertToCatalystExpression(filter, tableSchema)
if (expectExpression == null) {
if (removeQuotesIfNeed == null) {
assertEquals(exp.isEmpty, true)
} else {
assertEquals(exp.isDefined, true)
assertEquals(expectExpression, exp.get.sql)
assertEquals(removeQuotesIfNeed, exp.get.sql)
}
}
private def checkConvertFilters(filters: Array[Filter], expectExpression: String): Unit = {
// [SPARK-25769][SPARK-34636][SPARK-34626][SQL] sql method in UnresolvedAttribute,
// AttributeReference and Alias don't quote qualified names properly
val removeQuotesIfNeed = if (expectExpression != null && HoodieSparkUtils.isSpark3_2) {
expectExpression.replace("`", "")
} else {
expectExpression
}
val exp = convertToCatalystExpressions(filters, tableSchema)
if (expectExpression == null) {
if (removeQuotesIfNeed == null) {
assertEquals(exp.isEmpty, true)
} else {
assertEquals(exp.isDefined, true)
assertEquals(expectExpression, exp.get.sql)
assertEquals(removeQuotesIfNeed, exp.get.sql)
}
}

View File

@@ -293,28 +293,26 @@ class TestHoodieSparkSqlWriter {
*/
@Test
def testDisableAndEnableMetaFields(): Unit = {
try {
testBulkInsertWithSortMode(BulkInsertSortMode.NONE, populateMetaFields = false)
//create a new table
val fooTableModifier = commonTableModifier.updated("hoodie.bulkinsert.shuffle.parallelism", "4")
.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
.updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true")
.updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), BulkInsertSortMode.NONE.name())
.updated(HoodieTableConfig.POPULATE_META_FIELDS.key(), "true")
testBulkInsertWithSortMode(BulkInsertSortMode.NONE, populateMetaFields = false)
//create a new table
val fooTableModifier = commonTableModifier.updated("hoodie.bulkinsert.shuffle.parallelism", "4")
.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
.updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true")
.updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), BulkInsertSortMode.NONE.name())
.updated(HoodieTableConfig.POPULATE_META_FIELDS.key(), "true")
// generate the inserts
val schema = DataSourceTestUtils.getStructTypeExampleSchema
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
val inserts = DataSourceTestUtils.generateRandomRows(1000)
val df = spark.createDataFrame(sc.parallelize(inserts), structType)
try {
// write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df)
fail("Should have thrown exception")
} catch {
case e: HoodieException => assertTrue(e.getMessage.startsWith("Config conflict"))
case e: Exception => fail(e);
}
// generate the inserts
val schema = DataSourceTestUtils.getStructTypeExampleSchema
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
val inserts = DataSourceTestUtils.generateRandomRows(1000)
val df = spark.createDataFrame(sc.parallelize(inserts), structType)
try {
// write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df)
fail("Should have thrown exception")
} catch {
case e: HoodieException => assertTrue(e.getMessage.startsWith("Config conflict"))
case e: Exception => fail(e);
}
}
@@ -711,51 +709,49 @@ class TestHoodieSparkSqlWriter {
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test")
try {
val df = spark.range(0, 1000).toDF("keyid")
.withColumn("col3", expr("keyid"))
.withColumn("age", lit(1))
.withColumn("p", lit(2))
val df = spark.range(0, 1000).toDF("keyid")
.withColumn("col3", expr("keyid"))
.withColumn("age", lit(1))
.withColumn("p", lit(2))
df.write.format("hudi")
.options(options)
.option(DataSourceWriteOptions.OPERATION.key, "insert")
.option("hoodie.insert.shuffle.parallelism", "4")
.mode(SaveMode.Overwrite).save(tempBasePath)
df.write.format("hudi")
.options(options)
.option(DataSourceWriteOptions.OPERATION.key, "insert")
.option("hoodie.insert.shuffle.parallelism", "4")
.mode(SaveMode.Overwrite).save(tempBasePath)
df.write.format("hudi")
.options(options)
.option(DataSourceWriteOptions.OPERATION.key, "insert_overwrite_table")
.option("hoodie.insert.shuffle.parallelism", "4")
.mode(SaveMode.Append).save(tempBasePath)
df.write.format("hudi")
.options(options)
.option(DataSourceWriteOptions.OPERATION.key, "insert_overwrite_table")
.option("hoodie.insert.shuffle.parallelism", "4")
.mode(SaveMode.Append).save(tempBasePath)
val currentCommits = spark.read.format("hudi").load(tempBasePath).select("_hoodie_commit_time").take(1).map(_.getString(0))
val incrementalKeyIdNum = spark.read.format("hudi")
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
.option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "0000")
.option(DataSourceReadOptions.END_INSTANTTIME.key, currentCommits(0))
.load(tempBasePath).select("keyid").orderBy("keyid").count
assert(incrementalKeyIdNum == 1000)
val currentCommits = spark.read.format("hudi").load(tempBasePath).select("_hoodie_commit_time").take(1).map(_.getString(0))
val incrementalKeyIdNum = spark.read.format("hudi")
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
.option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "0000")
.option(DataSourceReadOptions.END_INSTANTTIME.key, currentCommits(0))
.load(tempBasePath).select("keyid").orderBy("keyid").count
assert(incrementalKeyIdNum == 1000)
df.write.mode(SaveMode.Overwrite).save(baseBootStrapPath)
spark.emptyDataFrame.write.format("hudi")
.options(options)
.option(HoodieBootstrapConfig.BASE_PATH.key, baseBootStrapPath)
.option(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getCanonicalName)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
.option(HoodieBootstrapConfig.PARALLELISM_VALUE.key, "4")
.mode(SaveMode.Overwrite).save(tempBasePath)
df.write.format("hudi").options(options)
.option(DataSourceWriteOptions.OPERATION.key, "insert_overwrite_table")
.option("hoodie.insert.shuffle.parallelism", "4").mode(SaveMode.Append).save(tempBasePath)
val currentCommitsBootstrap = spark.read.format("hudi").load(tempBasePath).select("_hoodie_commit_time").take(1).map(_.getString(0))
val incrementalKeyIdNumBootstrap = spark.read.format("hudi")
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
.option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "0000")
.option(DataSourceReadOptions.END_INSTANTTIME.key, currentCommitsBootstrap(0))
.load(tempBasePath).select("keyid").orderBy("keyid").count
assert(incrementalKeyIdNumBootstrap == 1000)
}
df.write.mode(SaveMode.Overwrite).save(baseBootStrapPath)
spark.emptyDataFrame.write.format("hudi")
.options(options)
.option(HoodieBootstrapConfig.BASE_PATH.key, baseBootStrapPath)
.option(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getCanonicalName)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
.option(HoodieBootstrapConfig.PARALLELISM_VALUE.key, "4")
.mode(SaveMode.Overwrite).save(tempBasePath)
df.write.format("hudi").options(options)
.option(DataSourceWriteOptions.OPERATION.key, "insert_overwrite_table")
.option("hoodie.insert.shuffle.parallelism", "4").mode(SaveMode.Append).save(tempBasePath)
val currentCommitsBootstrap = spark.read.format("hudi").load(tempBasePath).select("_hoodie_commit_time").take(1).map(_.getString(0))
val incrementalKeyIdNumBootstrap = spark.read.format("hudi")
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
.option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "0000")
.option(DataSourceReadOptions.END_INSTANTTIME.key, currentCommitsBootstrap(0))
.load(tempBasePath).select("keyid").orderBy("keyid").count
assert(incrementalKeyIdNumBootstrap == 1000)
}
}

View File

@@ -61,14 +61,18 @@ class TestHoodieSqlBase extends FunSuite with BeforeAndAfterAll {
}
override protected def test(testName: String, testTags: Tag*)(testFun: => Any /* Assertion */)(implicit pos: source.Position): Unit = {
try super.test(testName, testTags: _*)(try testFun finally {
val catalog = spark.sessionState.catalog
catalog.listDatabases().foreach{db =>
catalog.listTables(db).foreach {table =>
catalog.dropTable(table, true, true)
super.test(testName, testTags: _*)(
try {
testFun
} finally {
val catalog = spark.sessionState.catalog
catalog.listDatabases().foreach{db =>
catalog.listTables(db).foreach {table =>
catalog.dropTable(table, true, true)
}
}
}
})
)
}
protected def generateTableName: String = {

View File

@@ -17,6 +17,7 @@
package org.apache.spark.sql.hudi
import org.apache.hudi.HoodieSparkUtils
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.spark.sql.Row
@@ -352,7 +353,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase {
| when not matched and flag = '1' then insert *
|""".stripMargin
if (HoodieSqlUtils.isSpark3) {
if (HoodieSparkUtils.isSpark3) {
checkExceptionContain(mergeSql)("Columns aliases are not allowed in MERGE")
} else {
spark.sql(mergeSql)