[HUDI-2208] Support Bulk Insert For Spark Sql (#3328)
This commit is contained in:
@@ -79,12 +79,15 @@ class TestHoodieSqlBase extends FunSuite with BeforeAndAfterAll {
|
||||
}
|
||||
|
||||
protected def checkException(sql: String)(errorMsg: String): Unit = {
|
||||
var hasException = false
|
||||
try {
|
||||
spark.sql(sql)
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
assertResult(errorMsg)(e.getMessage)
|
||||
hasException = true
|
||||
}
|
||||
assertResult(true)(hasException)
|
||||
}
|
||||
|
||||
protected def removeQuotes(value: Any): Any = {
|
||||
|
||||
@@ -64,6 +64,7 @@ class TestInsertTable extends TestHoodieSqlBase {
|
||||
test("Test Insert Into None Partitioned Table") {
|
||||
withTempDir { tmp =>
|
||||
val tableName = generateTableName
|
||||
spark.sql(s"set hoodie.sql.insert.mode=strict")
|
||||
// Create none partitioned cow table
|
||||
spark.sql(
|
||||
s"""
|
||||
@@ -80,7 +81,6 @@ class TestInsertTable extends TestHoodieSqlBase {
|
||||
| preCombineField = 'ts'
|
||||
| )
|
||||
""".stripMargin)
|
||||
|
||||
spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)")
|
||||
checkAnswer(s"select id, name, price, ts from $tableName")(
|
||||
Seq(1, "a1", 10.0, 1000)
|
||||
@@ -127,6 +127,9 @@ class TestInsertTable extends TestHoodieSqlBase {
|
||||
checkAnswer(s"select id, name, price, ts from $tableName2")(
|
||||
Seq(1, "a1", 10.0, 1000)
|
||||
)
|
||||
// disable this config to avoid affect other test in this class.
|
||||
spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false")
|
||||
spark.sql(s"set hoodie.sql.insert.mode=upsert")
|
||||
}
|
||||
}
|
||||
|
||||
@@ -146,7 +149,6 @@ class TestInsertTable extends TestHoodieSqlBase {
|
||||
| partitioned by (dt)
|
||||
| location '${tmp.getCanonicalPath}/$tableName'
|
||||
""".stripMargin)
|
||||
|
||||
// Insert overwrite dynamic partition
|
||||
spark.sql(
|
||||
s"""
|
||||
@@ -246,7 +248,6 @@ class TestInsertTable extends TestHoodieSqlBase {
|
||||
| partitioned by (dt)
|
||||
| location '${tmp.getCanonicalPath}/$tableName'
|
||||
""".stripMargin)
|
||||
|
||||
spark.sql(s"insert into $tableName partition(dt = $partitionValue) select 1, 'a1', 10")
|
||||
spark.sql(s"insert into $tableName select 2, 'a2', 10, $partitionValue")
|
||||
checkAnswer(s"select id, name, price, cast(dt as string) from $tableName order by id")(
|
||||
@@ -303,5 +304,220 @@ class TestInsertTable extends TestHoodieSqlBase {
|
||||
"assertion failed: Required select columns count: 4, Current select columns(including static partition column)" +
|
||||
" count: 3,columns: (1,a1,10)"
|
||||
)
|
||||
spark.sql("set hoodie.sql.bulk.insert.enable = true")
|
||||
spark.sql("set hoodie.sql.insert.mode= strict")
|
||||
|
||||
val tableName2 = generateTableName
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName2 (
|
||||
| id int,
|
||||
| name string,
|
||||
| price double,
|
||||
| ts long
|
||||
|) using hudi
|
||||
| options (
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'ts'
|
||||
| )
|
||||
""".stripMargin)
|
||||
checkException(s"insert into $tableName2 values(1, 'a1', 10, 1000)")(
|
||||
"Table with primaryKey can not use bulk insert in strict mode."
|
||||
)
|
||||
|
||||
val tableName3 = generateTableName
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName3 (
|
||||
| id int,
|
||||
| name string,
|
||||
| price double,
|
||||
| dt string
|
||||
|) using hudi
|
||||
| partitioned by (dt)
|
||||
""".stripMargin)
|
||||
checkException(s"insert overwrite table $tableName3 values(1, 'a1', 10, '2021-07-18')")(
|
||||
"Insert Overwrite Partition can not use bulk insert."
|
||||
)
|
||||
spark.sql("set hoodie.sql.bulk.insert.enable = false")
|
||||
spark.sql("set hoodie.sql.insert.mode= upsert")
|
||||
}
|
||||
|
||||
test("Test bulk insert") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach {tableType =>
|
||||
// Test bulk insert for single partition
|
||||
val tableName = generateTableName
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName (
|
||||
| id int,
|
||||
| name string,
|
||||
| price double,
|
||||
| dt string
|
||||
|) using hudi
|
||||
| options (
|
||||
| type = '$tableType'
|
||||
| )
|
||||
| partitioned by (dt)
|
||||
| location '${tmp.getCanonicalPath}/$tableName'
|
||||
""".stripMargin)
|
||||
spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false")
|
||||
|
||||
// Enable the bulk insert
|
||||
spark.sql("set hoodie.sql.bulk.insert.enable = true")
|
||||
spark.sql(s"insert into $tableName values(1, 'a1', 10, '2021-07-18')")
|
||||
|
||||
checkAnswer(s"select id, name, price, dt from $tableName")(
|
||||
Seq(1, "a1", 10.0, "2021-07-18")
|
||||
)
|
||||
// Disable the bulk insert
|
||||
spark.sql("set hoodie.sql.bulk.insert.enable = false")
|
||||
spark.sql(s"insert into $tableName values(2, 'a2', 10, '2021-07-18')")
|
||||
|
||||
checkAnswer(s"select id, name, price, dt from $tableName order by id")(
|
||||
Seq(1, "a1", 10.0, "2021-07-18"),
|
||||
Seq(2, "a2", 10.0, "2021-07-18")
|
||||
)
|
||||
|
||||
// Test bulk insert for multi-level partition
|
||||
val tableMultiPartition = generateTableName
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableMultiPartition (
|
||||
| id int,
|
||||
| name string,
|
||||
| price double,
|
||||
| dt string,
|
||||
| hh string
|
||||
|) using hudi
|
||||
| options (
|
||||
| type = '$tableType'
|
||||
| )
|
||||
| partitioned by (dt, hh)
|
||||
| location '${tmp.getCanonicalPath}/$tableMultiPartition'
|
||||
""".stripMargin)
|
||||
|
||||
// Enable the bulk insert
|
||||
spark.sql("set hoodie.sql.bulk.insert.enable = true")
|
||||
spark.sql(s"insert into $tableMultiPartition values(1, 'a1', 10, '2021-07-18', '12')")
|
||||
|
||||
checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition")(
|
||||
Seq(1, "a1", 10.0, "2021-07-18", "12")
|
||||
)
|
||||
// Disable the bulk insert
|
||||
spark.sql("set hoodie.sql.bulk.insert.enable = false")
|
||||
spark.sql(s"insert into $tableMultiPartition " +
|
||||
s"values(2, 'a2', 10, '2021-07-18','12')")
|
||||
|
||||
checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition order by id")(
|
||||
Seq(1, "a1", 10.0, "2021-07-18", "12"),
|
||||
Seq(2, "a2", 10.0, "2021-07-18", "12")
|
||||
)
|
||||
// Test bulk insert for non-partitioned table
|
||||
val nonPartitionedTable = generateTableName
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $nonPartitionedTable (
|
||||
| id int,
|
||||
| name string,
|
||||
| price double
|
||||
|) using hudi
|
||||
| options (
|
||||
| type = '$tableType'
|
||||
| )
|
||||
| location '${tmp.getCanonicalPath}/$nonPartitionedTable'
|
||||
""".stripMargin)
|
||||
spark.sql("set hoodie.sql.bulk.insert.enable = true")
|
||||
spark.sql(s"insert into $nonPartitionedTable values(1, 'a1', 10)")
|
||||
checkAnswer(s"select id, name, price from $nonPartitionedTable")(
|
||||
Seq(1, "a1", 10.0)
|
||||
)
|
||||
spark.sql(s"insert overwrite table $nonPartitionedTable values(2, 'a2', 10)")
|
||||
checkAnswer(s"select id, name, price from $nonPartitionedTable")(
|
||||
Seq(2, "a2", 10.0)
|
||||
)
|
||||
spark.sql("set hoodie.sql.bulk.insert.enable = false")
|
||||
|
||||
// Test CTAS for bulk insert
|
||||
val tableName2 = generateTableName
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName2
|
||||
|using hudi
|
||||
|options(
|
||||
| type = '$tableType',
|
||||
| primaryKey = 'id'
|
||||
|)
|
||||
| location '${tmp.getCanonicalPath}/$tableName2'
|
||||
| as
|
||||
| select * from $tableName
|
||||
|""".stripMargin)
|
||||
checkAnswer(s"select id, name, price, dt from $tableName2 order by id")(
|
||||
Seq(1, "a1", 10.0, "2021-07-18"),
|
||||
Seq(2, "a2", 10.0, "2021-07-18")
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test combine before insert") {
|
||||
withTempDir{tmp =>
|
||||
val tableName = generateTableName
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName (
|
||||
| id int,
|
||||
| name string,
|
||||
| price double,
|
||||
| ts long
|
||||
|) using hudi
|
||||
| location '${tmp.getCanonicalPath}/$tableName'
|
||||
| options (
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'ts'
|
||||
| )
|
||||
""".stripMargin)
|
||||
spark.sql(
|
||||
s"""
|
||||
|insert overwrite table $tableName
|
||||
|select * from (
|
||||
| select 1 as id, 'a1' as name, 10 as price, 1000 as ts
|
||||
| union all
|
||||
| select 1 as id, 'a1' as name, 11 as price, 1001 as ts
|
||||
| )
|
||||
|""".stripMargin
|
||||
)
|
||||
checkAnswer(s"select id, name, price, ts from $tableName")(
|
||||
Seq(1, "a1", 11.0, 1001)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
test("Test insert pk-table") {
|
||||
withTempDir{tmp =>
|
||||
val tableName = generateTableName
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName (
|
||||
| id int,
|
||||
| name string,
|
||||
| price double,
|
||||
| ts long
|
||||
|) using hudi
|
||||
| location '${tmp.getCanonicalPath}/$tableName'
|
||||
| options (
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'ts'
|
||||
| )
|
||||
""".stripMargin)
|
||||
spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)")
|
||||
spark.sql(s"insert into $tableName values(1, 'a1', 11, 1000)")
|
||||
checkAnswer(s"select id, name, price, ts from $tableName")(
|
||||
Seq(1, "a1", 11.0, 1000)
|
||||
)
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user