1
0

[HUDI-1842] Spark Sql Support For pre-existing Hoodie Table (#3393)

This commit is contained in:
pengzhiwei
2021-08-07 19:49:26 +08:00
committed by GitHub
parent 70b6bd485f
commit 55d2e786db
19 changed files with 451 additions and 99 deletions

View File

@@ -17,9 +17,15 @@
package org.apache.spark.sql.hudi
import org.apache.hudi.DataSourceWriteOptions._
import scala.collection.JavaConverters._
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat
import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator}
import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.CatalogTableType
import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField}
@@ -272,4 +278,231 @@ class TestCreateTable extends TestHoodieSqlBase {
)
}
}
test("Test Create Table From Exist Hoodie Table") {
withTempDir { tmp =>
Seq("2021-08-02", "2021/08/02").foreach { partitionValue =>
val tableName = generateTableName
val tablePath = s"${tmp.getCanonicalPath}/$tableName"
import spark.implicits._
val df = Seq((1, "a1", 10, 1000, partitionValue)).toDF("id", "name", "value", "ts", "dt")
// Write a table by spark dataframe.
df.write.format("hudi")
.option(HoodieWriteConfig.TABLE_NAME.key, tableName)
.option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL)
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "ts")
.option(PARTITIONPATH_FIELD.key, "dt")
.option(KEYGENERATOR_CLASS.key, classOf[SimpleKeyGenerator].getName)
.option(HoodieWriteConfig.INSERT_PARALLELISM.key, "1")
.option(HoodieWriteConfig.UPSERT_PARALLELISM.key, "1")
.mode(SaveMode.Overwrite)
.save(tablePath)
// Create a table over the exist old table.
spark.sql(
s"""
|create table $tableName using hudi
| options (
| primaryKey = 'id',
| preCombineField = 'ts'
|)
|partitioned by (dt)
|location '$tablePath'
|""".stripMargin)
checkAnswer(s"select id, name, value, ts, dt from $tableName")(
Seq(1, "a1", 10, 1000, partitionValue)
)
// Check the missing properties for spark sql
val metaClient = HoodieTableMetaClient.builder()
.setBasePath(tablePath)
.setConf(spark.sessionState.newHadoopConf())
.build()
val properties = metaClient.getTableConfig.getProps.asScala.toMap
assertResult(true)(properties.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA.key))
assertResult("dt")(properties(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key))
assertResult("ts")(properties(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key))
// Test insert into
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000, '$partitionValue')")
checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, dt from $tableName order by id")(
Seq("1", partitionValue, 1, "a1", 10, 1000, partitionValue),
Seq("2", partitionValue, 2, "a2", 10, 1000, partitionValue)
)
// Test merge into
spark.sql(
s"""
|merge into $tableName h0
|using (select 1 as id, 'a1' as name, 11 as value, 1001 as ts, '$partitionValue' as dt) s0
|on h0.id = s0.id
|when matched then update set *
|""".stripMargin)
checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, dt from $tableName order by id")(
Seq("1", partitionValue, 1, "a1", 11, 1001, partitionValue),
Seq("2", partitionValue, 2, "a2", 10, 1000, partitionValue)
)
// Test update
spark.sql(s"update $tableName set value = value + 1 where id = 2")
checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, dt from $tableName order by id")(
Seq("1", partitionValue, 1, "a1", 11, 1001, partitionValue),
Seq("2", partitionValue, 2, "a2", 11, 1000, partitionValue)
)
// Test delete
spark.sql(s"delete from $tableName where id = 1")
checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, dt from $tableName order by id")(
Seq("2", partitionValue, 2, "a2", 11, 1000, partitionValue)
)
}
}
}
test("Test Create Table From Exist Hoodie Table For Multi-Level Partitioned Table") {
withTempDir { tmp =>
Seq("2021-08-02", "2021/08/02").foreach { day =>
val tableName = generateTableName
val tablePath = s"${tmp.getCanonicalPath}/$tableName"
import spark.implicits._
val df = Seq((1, "a1", 10, 1000, day, 12)).toDF("id", "name", "value", "ts", "day", "hh")
// Write a table by spark dataframe.
df.write.format("hudi")
.option(HoodieWriteConfig.TABLE_NAME.key, tableName)
.option(TABLE_TYPE.key, MOR_TABLE_TYPE_OPT_VAL)
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "ts")
.option(PARTITIONPATH_FIELD.key, "day,hh")
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
.option(HoodieWriteConfig.INSERT_PARALLELISM.key, "1")
.option(HoodieWriteConfig.UPSERT_PARALLELISM.key, "1")
.mode(SaveMode.Overwrite)
.save(tablePath)
// Create a table over the exist old table.
spark.sql(
s"""
|create table $tableName using hudi
| options (
| primaryKey = 'id',
| preCombineField = 'ts'
|)
|partitioned by (day, hh)
|location '$tablePath'
|""".stripMargin)
checkAnswer(s"select id, name, value, ts, day, hh from $tableName")(
Seq(1, "a1", 10, 1000, day, 12)
)
// Check the missing properties for spark sql
val metaClient = HoodieTableMetaClient.builder()
.setBasePath(tablePath)
.setConf(spark.sessionState.newHadoopConf())
.build()
val properties = metaClient.getTableConfig.getProps.asScala.toMap
assertResult(true)(properties.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA.key))
assertResult("day,hh")(properties(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key))
assertResult("ts")(properties(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key))
// Test insert into
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000, '$day', 12)")
checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, day, hh from $tableName order by id")(
Seq("id:1", s"$day/12", 1, "a1", 10, 1000, day, 12),
Seq("id:2", s"$day/12", 2, "a2", 10, 1000, day, 12)
)
// Test merge into
spark.sql(
s"""
|merge into $tableName h0
|using (select 1 as id, 'a1' as name, 11 as value, 1001 as ts, '$day' as day, 12 as hh) s0
|on h0.id = s0.id
|when matched then update set *
|""".stripMargin)
checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, day, hh from $tableName order by id")(
Seq("id:1", s"$day/12", 1, "a1", 11, 1001, day, 12),
Seq("id:2", s"$day/12", 2, "a2", 10, 1000, day, 12)
)
// Test update
spark.sql(s"update $tableName set value = value + 1 where id = 2")
checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, day, hh from $tableName order by id")(
Seq("id:1", s"$day/12", 1, "a1", 11, 1001, day, 12),
Seq("id:2", s"$day/12", 2, "a2", 11, 1000, day, 12)
)
// Test delete
spark.sql(s"delete from $tableName where id = 1")
checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, day, hh from $tableName order by id")(
Seq("id:2", s"$day/12", 2, "a2", 11, 1000, day, 12)
)
}
}
}
test("Test Create Table From Exist Hoodie Table For None Partitioned Table") {
withTempDir{tmp =>
// Write a table by spark dataframe.
val tableName = generateTableName
import spark.implicits._
val df = Seq((1, "a1", 10, 1000)).toDF("id", "name", "value", "ts")
df.write.format("hudi")
.option(HoodieWriteConfig.TABLE_NAME.key, tableName)
.option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL)
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "ts")
.option(PARTITIONPATH_FIELD.key, "")
.option(KEYGENERATOR_CLASS.key, classOf[NonpartitionedKeyGenerator].getName)
.option(HoodieWriteConfig.INSERT_PARALLELISM.key, "1")
.option(HoodieWriteConfig.UPSERT_PARALLELISM.key, "1")
.mode(SaveMode.Overwrite)
.save(tmp.getCanonicalPath)
// Create a table over the exist old table.
spark.sql(
s"""
|create table $tableName using hudi
| options (
| primaryKey = 'id',
| preCombineField = 'ts'
|)
|location '${tmp.getCanonicalPath}'
|""".stripMargin)
checkAnswer(s"select id, name, value, ts from $tableName")(
Seq(1, "a1", 10, 1000)
)
// Check the missing properties for spark sql
val metaClient = HoodieTableMetaClient.builder()
.setBasePath(tmp.getCanonicalPath)
.setConf(spark.sessionState.newHadoopConf())
.build()
val properties = metaClient.getTableConfig.getProps.asScala.toMap
assertResult(true)(properties.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA.key))
assertResult("ts")(properties(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key))
// Test insert into
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000)")
checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts from $tableName order by id")(
Seq("1", "", 1, "a1", 10, 1000),
Seq("2", "", 2, "a2", 10, 1000)
)
// Test merge into
spark.sql(
s"""
|merge into $tableName h0
|using (select 1 as id, 'a1' as name, 11 as value, 1001 as ts) s0
|on h0.id = s0.id
|when matched then update set *
|""".stripMargin)
checkAnswer(s"select id, name, value, ts from $tableName order by id")(
Seq(1, "a1", 11, 1001),
Seq(2, "a2", 10, 1000)
)
// Test update
spark.sql(s"update $tableName set value = value + 1 where id = 2")
checkAnswer(s"select id, name, value, ts from $tableName order by id")(
Seq(1, "a1", 11, 1001),
Seq(2, "a2", 11, 1000)
)
// Test delete
spark.sql(s"delete from $tableName where id = 1")
checkAnswer(s"select id, name, value, ts from $tableName order by id")(
Seq(2, "a2", 11, 1000)
)
}
}
}

View File

@@ -126,9 +126,9 @@ class TestPartialUpdateForMergeInto extends TestHoodieSqlBase {
checkException(
s"""
|merge into $tableName2 t0
|using ( select 1 as id, 'a1' as name, 12 as price) s0
|using ( select 1 as id, 'a1' as name, 12 as price, 1000 as ts) s0
|on t0.id = s0.id
|when matched then update set price = s0.price
|when matched then update set price = s0.price, _ts = s0.ts
""".stripMargin)(
"Missing specify the value for target field: 'id' in merge into update action for MOR table. " +
"Currently we cannot support partial update for MOR, please complete all the target fields " +