1
0

[HUDI-1659] Basic Implement Of Spark Sql Support For Hoodie (#2645)

Main functions:
Support create table for hoodie.
Support CTAS.
Support Insert for hoodie. Including dynamic partition and static partition insert.
Support MergeInto for hoodie.
Support DELETE
Support UPDATE
Both support spark2 & spark3 based on DataSourceV1.

Main changes:
Add sql parser for spark2.
Add HoodieAnalysis for sql resolve and logical plan rewrite.
Add commands implementation for CREATE TABLE、INSERT、MERGE INTO & CTAS.
In order to push down the update&insert logical to the HoodieRecordPayload for MergeInto, I make same change to the
HoodieWriteHandler and other related classes.
1、Add the inputSchema for parser the incoming record. This is because the inputSchema for MergeInto is different from writeSchema as there are some transforms in the update& insert expression.
2、Add WRITE_SCHEMA to HoodieWriteConfig to pass the write schema for merge into.
3、Pass properties to HoodieRecordPayload#getInsertValue to pass the insert expression and table schema.


Verify this pull request
Add TestCreateTable for test create hoodie tables and CTAS.
Add TestInsertTable for test insert hoodie tables.
Add TestMergeIntoTable for test merge hoodie tables.
Add TestUpdateTable for test update hoodie tables.
Add TestDeleteTable for test delete hoodie tables.
Add TestSqlStatement for test supported ddl/dml currently.
This commit is contained in:
pengzhiwei
2021-06-08 14:24:32 +08:00
committed by GitHub
parent cf83f10f5b
commit f760ec543e
86 changed files with 7346 additions and 255 deletions

View File

@@ -57,6 +57,7 @@ import org.apache.hudi.hadoop.HoodieParquetInputFormat;
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
import org.apache.hudi.index.HoodieIndex.IndexType;
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
import org.apache.hudi.common.util.PartitionPathEncodeUtils;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.table.action.bootstrap.BootstrapUtils;
import org.apache.hudi.testutils.HoodieClientTestBase;
@@ -82,8 +83,6 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import java.io.IOException;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import java.time.Instant;
import java.util.ArrayList;
import java.util.Arrays;
@@ -568,8 +567,8 @@ public class TestBootstrap extends HoodieClientTestBase {
});
if (isPartitioned) {
sqlContext.udf().register("partgen",
(UDF1<String, String>) (val) -> URLEncoder.encode(partitionPaths.get(
Integer.parseInt(val.split("_")[1]) % partitionPaths.size()), StandardCharsets.UTF_8.toString()),
(UDF1<String, String>) (val) -> PartitionPathEncodeUtils.escapePathName(partitionPaths.get(
Integer.parseInt(val.split("_")[1]) % partitionPaths.size())),
DataTypes.StringType);
}
JavaRDD rdd = jsc.parallelize(records);

View File

@@ -0,0 +1,255 @@
# SET OPTION
set hoodie.insert.shuffle.parallelism = 1;
+----------+
| ok |
+----------+
set hoodie.upsert.shuffle.parallelism = 1;
+----------+
| ok |
+----------+
set hoodie.delete.shuffle.parallelism = 1;
+----------+
| ok |
+----------+
# CTAS
create table h0 using hudi options(type = '${tableType}')
as select 1 as id, 'a1' as name, 10 as price;
+----------+
| ok |
+----------+
select id, name, price from h0;
+-----------+
| 1 a1 10 |
+-----------+
create table h0_p using hudi partitioned by(dt)
options(type = '${tableType}')
as select cast('2021-05-07 00:00:00' as timestamp) as dt,
1 as id, 'a1' as name, 10 as price;
+----------+
| ok |
+----------+
select id, name, price, cast(dt as string) from h0_p;
+--------------------------------+
| 1 a1 10 2021-05-07 00:00:00 |
+--------------------------------+
# CREATE TABLE
create table h1 (
id bigint,
name string,
price double,
ts bigint
) using hudi
options (
type = '${tableType}',
primaryKey = 'id',
preCombineField = 'ts'
)
location '${tmpDir}/h1';
+----------+
| ok |
+----------+
create table h1_p (
id bigint,
name string,
price double,
ts bigint,
dt string
) using hudi
partitioned by (dt)
options (
type = '${tableType}',
primaryKey = 'id',
preCombineField = 'ts'
)
location '${tmpDir}/h1_p';
+----------+
| ok |
+----------+
# INSERT/UPDATE/MERGE/DELETE
insert into h1 values(1, 'a1', 10, 1000);
+----------+
| ok |
+----------+
insert into h1 values(2, 'a2', 11, 1000);
+----------+
| ok |
+----------+
# insert static partition
insert into h1_p partition(dt = '2021-05-07') select * from h1;
+----------+
| ok |
+----------+
select id, name, price, ts, dt from h1_p order by id;
+---------------------------+
| 1 a1 10.0 1000 2021-05-07 |
| 2 a2 11.0 1000 2021-05-07 |
+---------------------------+
# insert overwrite table
insert overwrite table h1_p partition(dt = '2021-05-07') select * from h1 limit 10;
+----------+
| ok |
+----------+
select id, name, price, ts, dt from h1_p order by id;
+---------------------------+
| 1 a1 10.0 1000 2021-05-07 |
| 2 a2 11.0 1000 2021-05-07 |
+---------------------------+
# insert dynamic partition
insert into h1_p
select id, concat('a', id) as name, price, ts, dt
from (
select id + 2 as id, price + 2 as price, ts, '2021-05-08' as dt from h1
)
union all
select 5 as id, 'a5' as name, 10 as price, 1000 as ts, '2021-05-08' as dt;
+----------+
| ok |
+----------+
select id, name, price, ts, dt from h1_p order by id;
+---------------------------+
| 1 a1 10.0 1000 2021-05-07 |
| 2 a2 11.0 1000 2021-05-07 |
| 3 a3 12.0 1000 2021-05-08 |
| 4 a4 13.0 1000 2021-05-08 |
| 5 a5 10.0 1000 2021-05-08 |
+---------------------------+
# update table
update h1_p set price = price * 2 where id % 2 = 1;
+----------+
| ok |
+----------+
select id, name, price, ts, dt from h1_p order by id;
+---------------------------+
| 1 a1 20.0 1000 2021-05-07 |
| 2 a2 11.0 1000 2021-05-07 |
| 3 a3 24.0 1000 2021-05-08 |
| 4 a4 13.0 1000 2021-05-08 |
| 5 a5 20.0 1000 2021-05-08 |
+---------------------------+
update h1 set price = if (id %2 = 1, price * 2, price);
+----------+
| ok |
+----------+
select id, name, price, ts from h1;
+----------------+
| 1 a1 20.0 1000 |
| 2 a2 11.0 1000 |
+----------------+
# delete table
delete from h1_p where id = 5;
+----------+
| ok |
+----------+
select count(1) from h1_p;
+----------+
| 4 |
+----------+
# merge into
merge into h1_p t0
using (
select *, '2021-05-07' as dt from h1
) s0
on t0.id = s0.id
when matched then update set id = s0.id, name = s0.name, price = s0.price *2, ts = s0.ts, dt = s0.dt
when not matched then insert *;
+----------+
| ok |
+----------+
select id, name, price, ts, dt from h1_p order by id;
+---------------------------+
| 1 a1 40.0 1000 2021-05-07 |
| 2 a2 22.0 1000 2021-05-07 |
| 3 a3 24.0 1000 2021-05-08 |
| 4 a4 13.0 1000 2021-05-08 |
+---------------------------+
merge into h1_p t0
using (
select 5 as _id, 'a5' as _name, 10 as _price, 1000 as _ts, '2021-05-08' as dt
) s0
on s0._id = t0.id
when matched then update set *
when not matched then insert (id, name, price, ts, dt) values(_id, _name, _price, _ts, s0.dt);
+----------+
| ok |
+----------+
select id, name, price, ts, dt from h1_p order by id;
+---------------------------+
| 1 a1 40.0 1000 2021-05-07 |
| 2 a2 22.0 1000 2021-05-07 |
| 3 a3 24.0 1000 2021-05-08 |
| 4 a4 13.0 1000 2021-05-08 |
| 5 a5 10.0 1000 2021-05-08 |
+---------------------------+
merge into h1_p t0
using (
select 1 as id, '_delete' as name, 10 as price, 1000 as ts, '2021-05-07' as dt
union
select 2 as id, '_update' as name, 12 as price, 1001 as ts, '2021-05-07' as dt
union
select 6 as id, '_insert' as name, 10 as price, 1000 as ts, '2021-05-08' as dt
) s0
on s0.id = t0.id
when matched and name = '_update'
then update set id = s0.id, name = s0.name, price = s0.price, ts = s0.ts, dt = s0.dt
when matched and name = '_delete' then delete
when not matched and name = '_insert' then insert *;
+----------+
| ok |
+----------+
select id, name, price, ts, dt from h1_p order by id;
+--------------------------------+
| 2 _update 12.0 1001 2021-05-07 |
| 3 a3 24.0 1000 2021-05-08 |
| 4 a4 13.0 1000 2021-05-08 |
| 5 a5 10.0 1000 2021-05-08 |
| 6 _insert 10.0 1000 2021-05-08 |
+--------------------------------+
# DROP TABLE
drop table h0;
+----------+
| ok |
+----------+
drop table h0_p;
+----------+
| ok |
+----------+
drop table h1;
+----------+
| ok |
+----------+
drop table h1_p;
+----------+
| ok |
+----------+

View File

@@ -67,14 +67,6 @@ class TestConvertFilterToCatalystExpression {
"((`ts` < 10) AND (`ts` > 1))")
}
@Test
def testUnSupportConvert(): Unit = {
checkConvertFilters(Array(unsupport()), null)
checkConvertFilters(Array(and(unsupport(), eq("id", 1))), null)
checkConvertFilters(Array(or(unsupport(), eq("id", 1))), null)
checkConvertFilters(Array(and(eq("id", 1), not(unsupport()))), null)
}
private def checkConvertFilter(filter: Filter, expectExpression: String): Unit = {
val exp = convertToCatalystExpression(filter, tableSchema)
if (expectExpression == null) {
@@ -154,12 +146,4 @@ class TestConvertFilterToCatalystExpression {
private def contains(attribute: String, value: String): Filter = {
StringContains(attribute, value)
}
private def unsupport(): Filter = {
UnSupportFilter("")
}
case class UnSupportFilter(value: Any) extends Filter {
override def references: Array[String] = Array.empty
}
}

View File

@@ -17,14 +17,13 @@
package org.apache.hudi
import java.net.URLEncoder
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
import org.apache.hudi.common.util.PartitionPathEncodeUtils
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.keygen.ComplexKeyGenerator
import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.{Config, TimestampType}
@@ -137,7 +136,8 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
val fileIndex = HoodieFileIndex(spark, metaClient, None, Map("path" -> basePath))
val partitionFilter1 = EqualTo(attribute("partition"), literal("2021/03/08"))
val partitionName = if (partitionEncode) URLEncoder.encode("2021/03/08") else "2021/03/08"
val partitionName = if (partitionEncode) PartitionPathEncodeUtils.escapePathName("2021/03/08")
else "2021/03/08"
val partitionAndFilesAfterPrune = fileIndex.listFiles(Seq(partitionFilter1), Seq.empty)
assertEquals(1, partitionAndFilesAfterPrune.size)

View File

@@ -528,7 +528,13 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
"spark.sql.sources.schema.numParts=1\n" +
"spark.sql.sources.schema.numPartCols=1\n" +
"spark.sql.sources.schema.part.0=" +
"{\"type\":\"struct\",\"fields\":[{\"name\":\"_row_key\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}," +
"{\"type\":\"struct\",\"fields\":[{\"name\":\"_hoodie_commit_time\"," +
"\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":" +
"\"_hoodie_commit_seqno\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," +
"{\"name\":\"_hoodie_record_key\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," +
"{\"name\":\"_hoodie_partition_path\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," +
"{\"name\":\"_hoodie_file_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," +
"{\"name\":\"_row_key\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}," +
"{\"name\":\"ts\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}," +
"{\"name\":\"partition\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}]}")(hiveSyncConfig.tableProperties)

View File

@@ -0,0 +1,183 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.functional
import org.apache.hudi.common.util.FileIOUtils
import org.apache.spark.sql.hudi.TestHoodieSqlBase
class TestSqlStatement extends TestHoodieSqlBase {
val STATE_INIT = 0
val STATE_SKIP_COMMENT = 1
val STATE_FINISH_COMMENT = 2
val STATE_READ_SQL = 3
val STATE_FINISH_READ_SQL = 4
val STATE_START_FIRST_RESULT_LINE = 5
val STATE_END_FIRST_RESULT_LINE = 6
val STATE_READ_RESULT_LINE = 7
val STATE_FINISH_READ_RESULT_LINE = 8
val STATE_AFTER_FINISH_READ_RESULT_LINE = 9
val STATE_START_LAST_RESULT_LINE = 10
val STATE_END_LAST_RESULT_LINE = 11
val STATE_FINISH_ALL = 12
test("Test Sql Statements") {
Seq("cow", "mor").foreach { tableType =>
withTempDir { tmp =>
val params = Map(
"tableType" -> tableType,
"tmpDir" -> tmp.getCanonicalPath
)
execSqlFile("/sql-statements.sql", params)
}
}
}
private def execSqlFile(sqlFile: String, params: Map[String, String]): Unit = {
val inputStream = getClass.getResourceAsStream(sqlFile)
var sqlText = FileIOUtils.readAsUTFString(inputStream)
// replace parameters in the sql file
params.foreach { case (k, v) =>
sqlText = sqlText.replace("${" + k + "}", v)
}
var pos = 0
var state = STATE_INIT
val sqlBuffer = new StringBuilder
var sqlResult: String = null
val sqlExpectResult = new StringBuilder
val sqlExpectLineResult = new StringBuilder
while (pos < sqlText.length) {
var c = sqlText.charAt(pos)
val (changedState, needFetchNext) = changeState(c, state)
state = changedState
pos = pos + 1
if (needFetchNext) {
c = sqlText.charAt(pos)
}
state match {
case STATE_READ_SQL =>
sqlBuffer.append(c)
case STATE_FINISH_READ_SQL =>
val sql = sqlBuffer.toString().trim
try {
if (sql.startsWith("select")) {
sqlResult = spark.sql(sql).collect()
.map(row => row.toSeq.mkString(" ")).mkString("\n")
} else {
spark.sql(sql)
sqlResult = "ok"
}
} catch {
case e: Throwable =>
throw new RuntimeException(s"Error in execute: $sql", e)
}
case STATE_READ_RESULT_LINE =>
sqlExpectLineResult.append(c)
case STATE_FINISH_READ_RESULT_LINE =>
if (sqlExpectResult.nonEmpty) {
sqlExpectResult.append("\n")
}
sqlExpectResult.append(sqlExpectLineResult.toString().trim)
sqlExpectLineResult.clear()
case STATE_END_LAST_RESULT_LINE =>
val expectResult = sqlExpectResult.toString()
.split("\n").map(line => line.split("\\s+").mkString(" "))
.mkString("\n")
if (expectResult != sqlResult) {
throw new IllegalArgumentException(s"UnExpect result for: $sqlBuffer\n" +
s"Expect:\n $expectResult, Actual:\n $sqlResult")
}
sqlBuffer.clear()
sqlExpectResult.clear()
sqlResult = null
case _=>
}
}
state = STATE_FINISH_ALL
}
/**
* Change current state.
* @param c Current char.
* @param state Current state.
* @return (changedState, needFetchNext)
*/
private def changeState(c: Char, state: Int): (Int, Boolean) = {
state match {
case STATE_INIT | STATE_FINISH_COMMENT |
STATE_FINISH_READ_SQL | STATE_END_LAST_RESULT_LINE =>
if (c == '#') {
(STATE_SKIP_COMMENT, false)
} else if (c == '+') {
(STATE_START_FIRST_RESULT_LINE, false)
} else if (!Character.isWhitespace(c)) {
(STATE_READ_SQL, false)
} else {
(STATE_INIT, false)
}
case STATE_SKIP_COMMENT =>
if (c == '\n' || c == '\r') {
(STATE_FINISH_COMMENT, false)
} else {
(state, false)
}
case STATE_READ_SQL =>
if (c == ';') {
(STATE_FINISH_READ_SQL, false)
} else {
(state, false)
}
case STATE_START_FIRST_RESULT_LINE =>
if (c == '+') {
(STATE_END_FIRST_RESULT_LINE, false)
} else {
(state, false)
}
case STATE_END_FIRST_RESULT_LINE =>
if (c == '|') {
(STATE_READ_RESULT_LINE, true)
} else {
(state, false)
}
case STATE_READ_RESULT_LINE =>
if (c == '|') {
(STATE_FINISH_READ_RESULT_LINE, false)
} else {
(state, false)
}
case STATE_FINISH_READ_RESULT_LINE | STATE_AFTER_FINISH_READ_RESULT_LINE =>
if (c == '+') {
(STATE_START_LAST_RESULT_LINE, false)
} else if (c == '|') {
(STATE_READ_RESULT_LINE, true)
} else {
(STATE_AFTER_FINISH_READ_RESULT_LINE, false)
}
case STATE_START_LAST_RESULT_LINE =>
if (c == '+') {
(STATE_END_LAST_RESULT_LINE, false)
} else {
(state, false)
}
case _ =>
throw new IllegalArgumentException(s"Illegal State: $state meet '$c'")
}
}
}

View File

@@ -0,0 +1,275 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.hudi
import scala.collection.JavaConverters._
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat
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}
class TestCreateTable extends TestHoodieSqlBase {
test("Test Create Managed Hoodie Table") {
val tableName = generateTableName
// Create a managed table
spark.sql(
s"""
| create table $tableName (
| id int,
| name string,
| price double,
| ts long
| ) using hudi
| options (
| primaryKey = 'id',
| preCombineField = 'ts'
| )
""".stripMargin)
val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName))
assertResult(tableName)(table.identifier.table)
assertResult("hudi")(table.provider.get)
assertResult(CatalogTableType.MANAGED)(table.tableType)
assertResult(
HoodieRecord.HOODIE_META_COLUMNS.asScala.map(StructField(_, StringType))
++ Seq(
StructField("id", IntegerType),
StructField("name", StringType),
StructField("price", DoubleType),
StructField("ts", LongType))
)(table.schema.fields)
}
test("Test Create External Hoodie Table") {
withTempDir { tmp =>
// Test create cow table.
val tableName = generateTableName
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| options (
| primaryKey = 'id,name',
| type = 'cow'
| )
| location '${tmp.getCanonicalPath}'
""".stripMargin)
val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName))
assertResult(tableName)(table.identifier.table)
assertResult("hudi")(table.provider.get)
assertResult(CatalogTableType.EXTERNAL)(table.tableType)
assertResult(
HoodieRecord.HOODIE_META_COLUMNS.asScala.map(StructField(_, StringType))
++ Seq(
StructField("id", IntegerType),
StructField("name", StringType),
StructField("price", DoubleType),
StructField("ts", LongType))
)(table.schema.fields)
assertResult(table.storage.properties("type"))("cow")
assertResult(table.storage.properties("primaryKey"))("id,name")
spark.sql(s"drop table $tableName")
// Test create mor partitioned table
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long,
| dt string
|) using hudi
| partitioned by (dt)
| options (
| primaryKey = 'id',
| type = 'mor'
| )
| location '${tmp.getCanonicalPath}/h0'
""".stripMargin)
val table2 = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName))
assertResult(table2.storage.properties("type"))("mor")
assertResult(table2.storage.properties("primaryKey"))("id")
assertResult(Seq("dt"))(table2.partitionColumnNames)
assertResult(classOf[HoodieParquetRealtimeInputFormat].getCanonicalName)(table2.storage.inputFormat.get)
// Test create a external table with an exist table in the path
val tableName3 = generateTableName
spark.sql(
s"""
|create table $tableName3
|using hudi
|location '${tmp.getCanonicalPath}/h0'
""".stripMargin)
val table3 = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName3))
assertResult(table3.storage.properties("type"))("mor")
assertResult(table3.storage.properties("primaryKey"))("id")
assertResult(
HoodieRecord.HOODIE_META_COLUMNS.asScala.map(StructField(_, StringType))
++ Seq(
StructField("id", IntegerType),
StructField("name", StringType),
StructField("price", DoubleType),
StructField("ts", LongType),
StructField("dt", StringType)
)
)(table3.schema.fields)
}
}
test("Test Table Column Validate") {
withTempDir {tmp =>
val tableName = generateTableName
assertThrows[IllegalArgumentException] {
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| options (
| primaryKey = 'id1',
| type = 'cow'
| )
| location '${tmp.getCanonicalPath}'
""".stripMargin)
}
assertThrows[IllegalArgumentException] {
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| options (
| primaryKey = 'id',
| preCombineField = 'ts1',
| type = 'cow'
| )
| location '${tmp.getCanonicalPath}'
""".stripMargin)
}
assertThrows[IllegalArgumentException] {
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| options (
| primaryKey = 'id',
| preCombineField = 'ts',
| type = 'cow1'
| )
| location '${tmp.getCanonicalPath}'
""".stripMargin)
}
}
}
test("Test Create Table As Select") {
withTempDir { tmp =>
// Create Non-Partitioned table
val tableName1 = generateTableName
spark.sql(
s"""
|create table $tableName1 using hudi
| location '${tmp.getCanonicalPath}/$tableName1'
| AS
| select 1 as id, 'a1' as name, 10 as price, 1000 as ts
""".stripMargin)
checkAnswer(s"select id, name, price, ts from $tableName1")(
Seq(1, "a1", 10.0, 1000)
)
// Create Partitioned table
val tableName2 = generateTableName
spark.sql(
s"""
| create table $tableName2 using hudi
| partitioned by (dt)
| location '${tmp.getCanonicalPath}/$tableName2'
| AS
| select 1 as id, 'a1' as name, 10 as price, '2021-04-01' as dt
""".stripMargin
)
checkAnswer(s"select id, name, price, dt from $tableName2") (
Seq(1, "a1", 10, "2021-04-01")
)
// Create Partitioned table with timestamp data type
val tableName3 = generateTableName
// CTAS failed with null primaryKey
assertThrows[Exception] {
spark.sql(
s"""
| create table $tableName3 using hudi
| partitioned by (dt)
| options(primaryKey = 'id')
| location '${tmp.getCanonicalPath}/$tableName3'
| AS
| select null as id, 'a1' as name, 10 as price, '2021-05-07' as dt
|
""".stripMargin
)}
// Create table with timestamp type partition
spark.sql(
s"""
| create table $tableName3 using hudi
| partitioned by (dt)
| location '${tmp.getCanonicalPath}/$tableName3'
| AS
| select cast('2021-05-06 00:00:00' as timestamp) as dt, 1 as id, 'a1' as name, 10 as
| price
""".stripMargin
)
checkAnswer(s"select id, name, price, cast(dt as string) from $tableName3")(
Seq(1, "a1", 10, "2021-05-06 00:00:00")
)
// Create table with date type partition
val tableName4 = generateTableName
spark.sql(
s"""
| create table $tableName4 using hudi
| partitioned by (dt)
| location '${tmp.getCanonicalPath}/$tableName4'
| AS
| select cast('2021-05-06' as date) as dt, 1 as id, 'a1' as name, 10 as
| price
""".stripMargin
)
checkAnswer(s"select id, name, price, cast(dt as string) from $tableName4")(
Seq(1, "a1", 10, "2021-05-06")
)
}
}
}

View File

@@ -0,0 +1,67 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.hudi
class TestDeleteTable extends TestHoodieSqlBase {
test("Test Delete Table") {
withTempDir { tmp =>
Seq("cow", "mor").foreach {tableType =>
val tableName = generateTableName
// create table
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| location '${tmp.getCanonicalPath}/$tableName'
| options (
| type = '$tableType',
| primaryKey = 'id',
| preCombineField = 'ts'
| )
""".stripMargin)
// insert data to table
spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000")
checkAnswer(s"select id, name, price, ts from $tableName")(
Seq(1, "a1", 10.0, 1000)
)
// delete data from table
spark.sql(s"delete from $tableName where id = 1")
checkAnswer(s"select count(1) from $tableName") (
Seq(0)
)
spark.sql(s"insert into $tableName select 2, 'a2', 10, 1000")
spark.sql(s"delete from $tableName where id = 1")
checkAnswer(s"select id, name, price, ts from $tableName")(
Seq(2, "a2", 10.0, 1000)
)
spark.sql(s"delete from $tableName")
checkAnswer(s"select count(1) from $tableName")(
Seq(0)
)
}
}
}
}

View File

@@ -0,0 +1,81 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.hudi
import java.io.File
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.util.Utils
import org.scalactic.source
import org.scalatest.{BeforeAndAfterAll, FunSuite, Tag}
class TestHoodieSqlBase extends FunSuite with BeforeAndAfterAll {
private lazy val sparkWareHouse = {
val dir = Utils.createTempDir()
Utils.deleteRecursively(dir)
dir
}
protected lazy val spark: SparkSession = SparkSession.builder()
.master("local[1]")
.appName("hoodie sql test")
.withExtensions(new HoodieSparkSessionExtension)
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.config("hoodie.datasource.meta.sync.enable", "false")
.config("hoodie.insert.shuffle.parallelism", "4")
.config("hoodie.upsert.shuffle.parallelism", "4")
.config("hoodie.delete.shuffle.parallelism", "4")
.config("spark.sql.warehouse.dir", sparkWareHouse.getCanonicalPath)
.getOrCreate()
private var tableId = 0
protected def withTempDir(f: File => Unit): Unit = {
val tempDir = Utils.createTempDir()
try f(tempDir) finally {
Utils.deleteRecursively(tempDir)
}
}
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)
}
}
})
}
protected def generateTableName: String = {
val name = s"h$tableId"
tableId = tableId + 1
name
}
override protected def afterAll(): Unit = {
Utils.deleteRecursively(sparkWareHouse)
spark.stop()
}
protected def checkAnswer(sql: String)(expects: Seq[Any]*): Unit = {
assertResult(expects.map(row => Row(row: _*)).toArray)(spark.sql(sql).collect())
}
}

View File

@@ -0,0 +1,223 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.hudi
import org.apache.hudi.exception.HoodieDuplicateKeyException
class TestInsertTable extends TestHoodieSqlBase {
test("Test Insert Into") {
withTempDir { tmp =>
val tableName = generateTableName
// Create a partitioned table
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long,
| dt string
|) using hudi
| partitioned by (dt)
| location '${tmp.getCanonicalPath}'
""".stripMargin)
// Insert into dynamic partition
spark.sql(
s"""
| insert into $tableName
| select 1 as id, 'a1' as name, 10 as price, 1000 as ts, '2021-01-05' as dt
""".stripMargin)
checkAnswer(s"select id, name, price, ts, dt from $tableName")(
Seq(1, "a1", 10.0, 1000, "2021-01-05")
)
// Insert into static partition
spark.sql(
s"""
| insert into $tableName partition(dt = '2021-01-05')
| select 2 as id, 'a2' as name, 10 as price, 1000 as ts
""".stripMargin)
checkAnswer(s"select id, name, price, ts, dt from $tableName")(
Seq(1, "a1", 10.0, 1000, "2021-01-05"),
Seq(2, "a2", 10.0, 1000, "2021-01-05")
)
}
}
test("Test Insert Into None Partitioned Table") {
withTempDir { tmp =>
val tableName = generateTableName
// Create none partitioned cow table
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| location '${tmp.getCanonicalPath}/$tableName'
| options (
| type = 'cow',
| primaryKey = 'id',
| 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)
)
spark.sql(s"insert into $tableName select 2, 'a2', 12, 1000")
checkAnswer(s"select id, name, price, ts from $tableName")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 12.0, 1000)
)
assertThrows[HoodieDuplicateKeyException] {
try {
spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000")
} catch {
case e: Exception =>
var root: Throwable = e
while (root.getCause != null) {
root = root.getCause
}
throw root
}
}
// Create table with dropDup is true
val tableName2 = generateTableName
spark.sql("set hoodie.datasource.write.insert.drop.duplicates = true")
spark.sql(
s"""
|create table $tableName2 (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| location '${tmp.getCanonicalPath}/$tableName2'
| options (
| type = 'mor',
| primaryKey = 'id',
| preCombineField = 'ts'
| )
""".stripMargin)
spark.sql(s"insert into $tableName2 select 1, 'a1', 10, 1000")
// This record will be drop when dropDup is true
spark.sql(s"insert into $tableName2 select 1, 'a1', 12, 1000")
checkAnswer(s"select id, name, price, ts from $tableName2")(
Seq(1, "a1", 10.0, 1000)
)
}
}
test("Test Insert Overwrite") {
withTempDir { tmp =>
val tableName = generateTableName
// Create a partitioned table
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long,
| dt string
|) using hudi
| partitioned by (dt)
| location '${tmp.getCanonicalPath}/$tableName'
""".stripMargin)
// Insert overwrite dynamic partition
spark.sql(
s"""
| insert overwrite table $tableName
| select 1 as id, 'a1' as name, 10 as price, 1000 as ts, '2021-01-05' as dt
""".stripMargin)
checkAnswer(s"select id, name, price, ts, dt from $tableName")(
Seq(1, "a1", 10.0, 1000, "2021-01-05")
)
// Insert overwrite dynamic partition
spark.sql(
s"""
| insert overwrite table $tableName
| select 2 as id, 'a2' as name, 10 as price, 1000 as ts, '2021-01-06' as dt
""".stripMargin)
checkAnswer(s"select id, name, price, ts, dt from $tableName order by id")(
Seq(1, "a1", 10.0, 1000, "2021-01-05"),
Seq(2, "a2", 10.0, 1000, "2021-01-06")
)
// Insert overwrite static partition
spark.sql(
s"""
| insert overwrite table $tableName partition(dt = '2021-01-05')
| select * from (select 2 , 'a2', 12, 1000) limit 10
""".stripMargin)
checkAnswer(s"select id, name, price, ts, dt from $tableName order by dt")(
Seq(2, "a2", 12.0, 1000, "2021-01-05"),
Seq(2, "a2", 10.0, 1000, "2021-01-06")
)
// Insert data from another table
val tblNonPartition = generateTableName
spark.sql(
s"""
| create table $tblNonPartition (
| id int,
| name string,
| price double,
| ts long
| ) using hudi
| location '${tmp.getCanonicalPath}/$tblNonPartition'
""".stripMargin)
spark.sql(s"insert into $tblNonPartition select 1, 'a1', 10, 1000")
spark.sql(
s"""
| insert overwrite table $tableName partition(dt ='2021-01-04')
| select * from $tblNonPartition limit 10
""".stripMargin)
checkAnswer(s"select id, name, price, ts, dt from $tableName order by id,dt")(
Seq(1, "a1", 10.0, 1000, "2021-01-04"),
Seq(2, "a2", 12.0, 1000, "2021-01-05"),
Seq(2, "a2", 10.0, 1000, "2021-01-06")
)
spark.sql(
s"""
| insert overwrite table $tableName
| select id + 2, name, price, ts , '2021-01-04' from $tblNonPartition limit 10
""".stripMargin)
checkAnswer(s"select id, name, price, ts, dt from $tableName " +
s"where dt >='2021-01-04' and dt <= '2021-01-06' order by id,dt")(
Seq(2, "a2", 12.0, 1000, "2021-01-05"),
Seq(2, "a2", 10.0, 1000, "2021-01-06"),
Seq(3, "a1", 10.0, 1000, "2021-01-04")
)
// test insert overwrite non-partitioned table
spark.sql(s"insert overwrite table $tblNonPartition select 2, 'a2', 10, 1000")
checkAnswer(s"select id, name, price, ts from $tblNonPartition")(
Seq(2, "a2", 10.0, 1000)
)
}
}
}

View File

@@ -0,0 +1,535 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.hudi
import org.apache.hudi.{DataSourceReadOptions, HoodieDataSourceHelpers}
import org.apache.hudi.common.fs.FSUtils
class TestMergeIntoTable extends TestHoodieSqlBase {
test("Test MergeInto Basic") {
withTempDir { tmp =>
val tableName = generateTableName
// Create table
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| location '${tmp.getCanonicalPath}'
| options (
| primaryKey ='id',
| preCombineField = 'ts'
| )
""".stripMargin)
// First merge with a extra input field 'flag' (insert a new record)
spark.sql(
s"""
| merge into $tableName
| using (
| select 1 as id, 'a1' as name, 10 as price, 1000 as ts, '1' as flag
| ) s0
| on s0.id = $tableName.id
| when matched and flag = '1' then update set
| id = s0.id, name = s0.name, price = s0.price, ts = s0.ts
| when not matched and flag = '1' then insert *
""".stripMargin)
checkAnswer(s"select id, name, price, ts from $tableName")(
Seq(1, "a1", 10.0, 1000)
)
// Second merge (update the record)
spark.sql(
s"""
| merge into $tableName
| using (
| select 1 as id, 'a1' as name, 10 as price, 1001 as ts
| ) s0
| on s0.id = $tableName.id
| when matched then update set
| id = s0.id, name = s0.name, price = s0.price + $tableName.price, ts = s0.ts
| when not matched then insert *
""".stripMargin)
checkAnswer(s"select id, name, price, ts from $tableName")(
Seq(1, "a1", 20.0, 1001)
)
// the third time merge (update & insert the record)
spark.sql(
s"""
| merge into $tableName
| using (
| select * from (
| select 1 as id, 'a1' as name, 10 as price, 1002 as ts
| union all
| select 2 as id, 'a2' as name, 12 as price, 1001 as ts
| )
| ) s0
| on s0.id = $tableName.id
| when matched then update set
| id = s0.id, name = s0.name, price = s0.price + $tableName.price, ts = s0.ts
| when not matched and id % 2 = 0 then insert *
""".stripMargin)
checkAnswer(s"select id, name, price, ts from $tableName")(
Seq(1, "a1", 30.0, 1002),
Seq(2, "a2", 12.0, 1001)
)
// the fourth merge (delete the record)
spark.sql(
s"""
| merge into $tableName
| using (
| select 1 as id, 'a1' as name, 12 as price, 1003 as ts
| ) s0
| on s0.id = $tableName.id
| when matched and id != 1 then update set
| id = s0.id, name = s0.name, price = s0.price, ts = s0.ts
| when matched and id = 1 then delete
| when not matched then insert *
""".stripMargin)
val cnt = spark.sql(s"select * from $tableName where id = 1").count()
assertResult(0)(cnt)
}
}
test("Test MergeInto with ignored record") {
withTempDir {tmp =>
val sourceTable = generateTableName
val targetTable = generateTableName
// Create source table
spark.sql(
s"""
| create table $sourceTable (
| id int,
| name string,
| price double,
| ts long
| ) using parquet
| location '${tmp.getCanonicalPath}/$sourceTable'
""".stripMargin)
// Create target table
spark.sql(
s"""
|create table $targetTable (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| location '${tmp.getCanonicalPath}/$targetTable'
| options (
| primaryKey ='id',
| preCombineField = 'ts'
| )
""".stripMargin)
// Insert data to source table
spark.sql(s"insert into $sourceTable values(1, 'a1', 10, 1000)")
spark.sql(s"insert into $sourceTable values(2, 'a2', 11, 1000)")
spark.sql(
s"""
| merge into $targetTable as t0
| using (select * from $sourceTable) as s0
| on t0.id = s0.id
| when matched then update set *
| when not matched and s0.name = 'a1' then insert *
""".stripMargin)
// The record of "name = 'a2'" will be filter
checkAnswer(s"select id, name, price, ts from $targetTable")(
Seq(1, "a1", 10.0, 1000)
)
spark.sql(s"insert into $targetTable select 3, 'a3', 12, 1000")
checkAnswer(s"select id, name, price, ts from $targetTable")(
Seq(1, "a1", 10.0, 1000),
Seq(3, "a3", 12, 1000)
)
spark.sql(
s"""
| merge into $targetTable as t0
| using (
| select * from (
| select 1 as s_id, 'a1' as name, 20 as price, 1001 as ts
| union all
| select 3 as s_id, 'a3' as name, 20 as price, 1003 as ts
| union all
| select 4 as s_id, 'a4' as name, 10 as price, 1004 as ts
| )
| ) s0
| on s0.s_id = t0.id
| when matched and ts = 1001 then update set id = s0.s_id, name = t0.name, price =
| s0.price, ts = s0.ts
""".stripMargin
)
// Ignore the update for id = 3
checkAnswer(s"select id, name, price, ts from $targetTable")(
Seq(1, "a1", 20.0, 1001),
Seq(3, "a3", 12.0, 1000)
)
}
}
test("Test MergeInto for MOR table ") {
withTempDir {tmp =>
val tableName = generateTableName
// Create a mor partitioned table.
spark.sql(
s"""
| create table $tableName (
| id int,
| name string,
| price double,
| ts long,
| dt string
| ) using hudi
| options (
| type = 'mor',
| primaryKey = 'id',
| preCombineField = 'ts'
| )
| partitioned by(dt)
| location '${tmp.getCanonicalPath}'
""".stripMargin)
// Insert data
spark.sql(
s"""
| merge into $tableName as t0
| using (
| select 1 as id, 'a1' as name, 10 as price, 1000 as ts, '2021-03-21' as dt
| ) as s0
| on t0.id = s0.id
| when not matched and s0.id % 2 = 1 then insert *
""".stripMargin
)
checkAnswer(s"select id,name,price,dt from $tableName")(
Seq(1, "a1", 10, "2021-03-21")
)
// Update data when matched-condition not matched.
spark.sql(
s"""
| merge into $tableName as t0
| using (
| select 1 as id, 'a1' as name, 12 as price, 1001 as ts, '2021-03-21' as dt
| ) as s0
| on t0.id = s0.id
| when matched and id % 2 = 0 then update set *
""".stripMargin
)
checkAnswer(s"select id,name,price,dt from $tableName")(
Seq(1, "a1", 10, "2021-03-21")
)
// Update data when matched-condition matched.
spark.sql(
s"""
| merge into $tableName as t0
| using (
| select 1 as id, 'a1' as name, 12 as price, 1001 as ts, '2021-03-21' as dt
| ) as s0
| on t0.id = s0.id
| when matched and s0.id % 2 = 1 then update set *
""".stripMargin
)
checkAnswer(s"select id,name,price,dt from $tableName")(
Seq(1, "a1", 12, "2021-03-21")
)
// Insert a new data.
spark.sql(
s"""
| merge into $tableName as t0
| using (
| select 2 as id, 'a2' as name, 10 as price, 1000 as ts, '2021-03-21' as dt
| ) as s0
| on t0.id = s0.id
| when not matched and s0.id % 2 = 0 then insert *
""".stripMargin
)
checkAnswer(s"select id,name,price,dt from $tableName order by id")(
Seq(1, "a1", 12, "2021-03-21"),
Seq(2, "a2", 10, "2021-03-21")
)
// Update with different source column names.
spark.sql(
s"""
| merge into $tableName t0
| using (
| select 2 as s_id, 'a2' as s_name, 15 as s_price, 1001 as s_ts, '2021-03-21' as dt
| ) s0
| on t0.id = s0.s_id
| when matched and s_ts = 1001 then update set *
""".stripMargin
)
checkAnswer(s"select id,name,price,dt from $tableName order by id")(
Seq(1, "a1", 12, "2021-03-21"),
Seq(2, "a2", 15, "2021-03-21")
)
// Delete with condition expression.
spark.sql(
s"""
| merge into $tableName t0
| using (
| select 1 as s_id, 'a2' as s_name, 15 as s_price, 1001 as s_ts, '2021-03-21' as dt
| ) s0
| on t0.id = s0.s_id + 1
| when matched and s_ts = 1001 then delete
""".stripMargin
)
checkAnswer(s"select id,name,price,dt from $tableName order by id")(
Seq(1, "a1", 12, "2021-03-21")
)
}
}
test("Test MergeInto with insert only") {
withTempDir {tmp =>
// Create a partitioned mor table
val tableName = generateTableName
spark.sql(
s"""
| create table $tableName (
| id bigint,
| name string,
| price double,
| dt string
| ) using hudi
| options (
| type = 'mor',
| primaryKey = 'id'
| )
| partitioned by(dt)
| location '${tmp.getCanonicalPath}'
""".stripMargin)
spark.sql(s"insert into $tableName select 1, 'a1', 10, '2021-03-21'")
spark.sql(
s"""
| merge into $tableName as t0
| using (
| select 2 as id, 'a2' as name, 10 as price, 1000 as ts, '2021-03-20' as dt
| ) s0
| on s0.id = t0.id
| when not matched and s0.id % 2 = 0 then insert (id,name,price,dt)
| values(s0.id,s0.name,s0.price,s0.dt)
""".stripMargin)
checkAnswer(s"select id,name,price,dt from $tableName order by id")(
Seq(1, "a1", 10, "2021-03-21"),
Seq(2, "a2", 10, "2021-03-20")
)
spark.sql(
s"""
| merge into $tableName as t0
| using (
| select 3 as id, 'a3' as name, 10 as price, 1000 as ts, '2021-03-20' as dt
| ) s0
| on s0.id = t0.id
| when not matched and s0.id % 2 = 0 then insert (id,name,price,dt)
| values(s0.id,s0.name,s0.price,s0.dt)
""".stripMargin)
// id = 3 should not write to the table as it has filtered by id % 2 = 0
checkAnswer(s"select id,name,price,dt from $tableName order by id")(
Seq(1, "a1", 10, "2021-03-21"),
Seq(2, "a2", 10, "2021-03-20")
)
}
}
test("Test MergeInto For PreCombineField") {
withTempDir { tmp =>
Seq("cow", "mor").foreach { tableType =>
val tableName1 = generateTableName
// Create a mor partitioned table.
spark.sql(
s"""
| create table $tableName1 (
| id int,
| name string,
| price double,
| v long,
| dt string
| ) using hudi
| options (
| type = '$tableType',
| primaryKey = 'id',
| preCombineField = 'v'
| )
| partitioned by(dt)
| location '${tmp.getCanonicalPath}/$tableName1'
""".stripMargin)
// Insert data
spark.sql(
s"""
| merge into $tableName1 as t0
| using (
| select 1 as id, 'a1' as name, 10 as price, 1001 as v, '2021-03-21' as dt
| ) as s0
| on t0.id = s0.id
| when not matched and s0.id % 2 = 1 then insert *
""".stripMargin
)
checkAnswer(s"select id,name,price,dt,v from $tableName1")(
Seq(1, "a1", 10, "2021-03-21", 1001)
)
// Update data with a smaller version value
spark.sql(
s"""
| merge into $tableName1 as t0
| using (
| select 1 as id, 'a1' as name, 11 as price, 1000 as v, '2021-03-21' as dt
| ) as s0
| on t0.id = s0.id
| when matched and s0.id % 2 = 1 then update set *
""".stripMargin
)
// Update failed as v = 1000 < 1001
checkAnswer(s"select id,name,price,dt,v from $tableName1")(
Seq(1, "a1", 10, "2021-03-21", 1001)
)
// Update data with a bigger version value
spark.sql(
s"""
| merge into $tableName1 as t0
| using (
| select 1 as id, 'a1' as name, 12 as price, 1002 as v, '2021-03-21' as dt
| ) as s0
| on t0.id = s0.id
| when matched and s0.id % 2 = 1 then update set *
""".stripMargin
)
// Update success
checkAnswer(s"select id,name,price,dt,v from $tableName1")(
Seq(1, "a1", 12, "2021-03-21", 1002)
)
}
}
}
test("Merge Hudi to Hudi") {
withTempDir { tmp =>
Seq("cow", "mor").foreach { tableType =>
val sourceTable = generateTableName
spark.sql(
s"""
|create table $sourceTable (
| id int,
| name string,
| price double,
| _ts long
|) using hudi
|options(
| type ='$tableType',
| primaryKey = 'id',
| preCombineField = '_ts'
|)
|location '${tmp.getCanonicalPath}/$sourceTable'
""".stripMargin)
val targetTable = generateTableName
val targetBasePath = s"${tmp.getCanonicalPath}/$targetTable"
spark.sql(
s"""
|create table $targetTable (
| id int,
| name string,
| price double,
| _ts long
|) using hudi
|options(
| type ='$tableType',
| primaryKey = 'id',
| preCombineField = '_ts'
|)
|location '$targetBasePath'
""".stripMargin)
// First merge
spark.sql(s"insert into $sourceTable values(1, 'a1', 10, 1000)")
spark.sql(
s"""
|merge into $targetTable t0
|using $sourceTable s0
|on t0.id = s0.id
|when not matched then insert *
""".stripMargin)
checkAnswer(s"select id, name, price, _ts from $targetTable")(
Seq(1, "a1", 10, 1000)
)
val fs = FSUtils.getFs(targetBasePath, spark.sessionState.newHadoopConf())
val firstCommitTime = HoodieDataSourceHelpers.latestCommit(fs, targetBasePath)
// Second merge
spark.sql(s"update $sourceTable set price = 12, _ts = 1001 where id = 1")
spark.sql(
s"""
|merge into $targetTable t0
|using $sourceTable s0
|on t0.id = s0.id
|when matched and cast(_ts as string) > '1000' then update set *
""".stripMargin)
checkAnswer(s"select id, name, price, _ts from $targetTable")(
Seq(1, "a1", 12, 1001)
)
// Test incremental query
val hudiIncDF1 = spark.read.format("org.apache.hudi")
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommitTime)
.load(targetBasePath)
hudiIncDF1.createOrReplaceTempView("inc1")
checkAnswer(s"select id, name, price, _ts from inc1")(
Seq(1, "a1", 10, 1000)
)
val secondCommitTime = HoodieDataSourceHelpers.latestCommit(fs, targetBasePath)
// Third merge
spark.sql(s"insert into $sourceTable values(2, 'a2', 10, 1001)")
spark.sql(
s"""
|merge into $targetTable t0
|using $sourceTable s0
|on t0.id = s0.id
|when matched then update set *
|when not matched and name = 'a2' then insert *
""".stripMargin)
checkAnswer(s"select id, name, price, _ts from $targetTable order by id")(
Seq(1, "a1", 12, 1001),
Seq(2, "a2", 10, 1001)
)
// Test incremental query
val hudiIncDF2 = spark.read.format("org.apache.hudi")
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, secondCommitTime)
.load(targetBasePath)
hudiIncDF2.createOrReplaceTempView("inc2")
checkAnswer(s"select id, name, price, _ts from inc2 order by id")(
Seq(1, "a1", 12, 1001),
Seq(2, "a2", 10, 1001)
)
}
}
}
}

View File

@@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.hudi
class TestUpdateTable extends TestHoodieSqlBase {
test("Test Update Table") {
withTempDir { tmp =>
Seq("cow", "mor").foreach {tableType =>
val tableName = generateTableName
// create table
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| location '${tmp.getCanonicalPath}/$tableName'
| options (
| type = '$tableType',
| primaryKey = 'id',
| preCombineField = 'ts'
| )
""".stripMargin)
// insert data to table
spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000")
checkAnswer(s"select id, name, price, ts from $tableName")(
Seq(1, "a1", 10.0, 1000)
)
// update data
spark.sql(s"update $tableName set price = 20 where id = 1")
checkAnswer(s"select id, name, price, ts from $tableName")(
Seq(1, "a1", 20.0, 1000)
)
// update data
spark.sql(s"update $tableName set price = price * 2 where id = 1")
checkAnswer(s"select id, name, price, ts from $tableName")(
Seq(1, "a1", 40.0, 1000)
)
}
}
}
}