|
|
|
|
@@ -30,7 +30,7 @@ import org.apache.hudi.exception.HoodieException
|
|
|
|
|
import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode
|
|
|
|
|
import org.apache.hudi.functional.TestBootstrap
|
|
|
|
|
import org.apache.hudi.hive.HiveSyncConfig
|
|
|
|
|
import org.apache.hudi.keygen.{NonpartitionedKeyGenerator, SimpleKeyGenerator}
|
|
|
|
|
import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator}
|
|
|
|
|
import org.apache.hudi.testutils.DataSourceTestUtils
|
|
|
|
|
import org.apache.spark.SparkContext
|
|
|
|
|
import org.apache.spark.api.java.JavaSparkContext
|
|
|
|
|
@@ -48,8 +48,10 @@ import org.scalatest.Matchers.{assertResult, be, convertToAnyShouldWrapper, inte
|
|
|
|
|
import java.time.Instant
|
|
|
|
|
import java.util
|
|
|
|
|
import java.util.{Collections, Date, UUID}
|
|
|
|
|
|
|
|
|
|
import scala.collection.JavaConversions._
|
|
|
|
|
import scala.collection.JavaConverters
|
|
|
|
|
import scala.util.control.NonFatal
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Test suite for SparkSqlWriter class.
|
|
|
|
|
@@ -161,7 +163,6 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
.updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true")
|
|
|
|
|
.updated(HoodieTableConfig.POPULATE_META_FIELDS.key(), String.valueOf(populateMetaFields))
|
|
|
|
|
.updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), sortMode.name())
|
|
|
|
|
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
|
|
|
|
|
|
|
|
|
// generate the inserts
|
|
|
|
|
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
|
|
|
|
@@ -175,7 +176,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
val recordsSeq = convertRowListToSeq(records)
|
|
|
|
|
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
|
|
|
|
// write to Hudi
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df)
|
|
|
|
|
|
|
|
|
|
// collect all partition paths to issue read of parquet files
|
|
|
|
|
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
|
|
|
|
@@ -242,21 +243,19 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
//create a new table
|
|
|
|
|
val fooTableModifier = Map("path" -> tempBasePath, HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName,
|
|
|
|
|
"hoodie.insert.shuffle.parallelism" -> "4", "hoodie.upsert.shuffle.parallelism" -> "4")
|
|
|
|
|
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
|
|
|
|
val dataFrame = spark.createDataFrame(Seq(StringLongTest(UUID.randomUUID().toString, new Date().getTime)))
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, dataFrame)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, dataFrame)
|
|
|
|
|
|
|
|
|
|
//on same path try append with different("hoodie_bar_tbl") table name which should throw an exception
|
|
|
|
|
val barTableModifier = Map("path" -> tempBasePath, HoodieWriteConfig.TBL_NAME.key -> "hoodie_bar_tbl",
|
|
|
|
|
"hoodie.insert.shuffle.parallelism" -> "4", "hoodie.upsert.shuffle.parallelism" -> "4")
|
|
|
|
|
val barTableParams = HoodieWriterUtils.parametersWithWriteDefaults(barTableModifier)
|
|
|
|
|
val dataFrame2 = spark.createDataFrame(Seq(StringLongTest(UUID.randomUUID().toString, new Date().getTime)))
|
|
|
|
|
val tableAlreadyExistException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, barTableParams, dataFrame2))
|
|
|
|
|
val tableAlreadyExistException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, barTableModifier, dataFrame2))
|
|
|
|
|
assert(tableAlreadyExistException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist"))
|
|
|
|
|
|
|
|
|
|
//on same path try append with delete operation and different("hoodie_bar_tbl") table name which should throw an exception
|
|
|
|
|
val deleteTableParams = barTableParams ++ Map(OPERATION.key -> "delete")
|
|
|
|
|
val deleteCmdException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, deleteTableParams, dataFrame2))
|
|
|
|
|
val deleteTableModifier = barTableModifier ++ Map(OPERATION.key -> "delete")
|
|
|
|
|
val deleteCmdException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, deleteTableModifier, dataFrame2))
|
|
|
|
|
assert(deleteCmdException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist"))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@@ -295,7 +294,6 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
.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())
|
|
|
|
|
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
|
|
|
|
|
|
|
|
|
// generate the inserts
|
|
|
|
|
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
|
|
|
|
@@ -304,7 +302,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
val df = spark.createDataFrame(sc.parallelize(inserts), structType)
|
|
|
|
|
try {
|
|
|
|
|
// write to Hudi
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df)
|
|
|
|
|
Assertions.fail("Should have thrown exception")
|
|
|
|
|
} catch {
|
|
|
|
|
case e: HoodieException => assertTrue(e.getMessage.contains("hoodie.populate.meta.fields already disabled for the table. Can't be re-enabled back"))
|
|
|
|
|
@@ -323,7 +321,6 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
|
|
|
|
|
.updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true")
|
|
|
|
|
.updated(INSERT_DROP_DUPS.key, "true")
|
|
|
|
|
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
|
|
|
|
|
|
|
|
|
// generate the inserts
|
|
|
|
|
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
|
|
|
|
@@ -332,7 +329,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
val recordsSeq = convertRowListToSeq(records)
|
|
|
|
|
val df = spark.createDataFrame(spark.sparkContext.parallelize(recordsSeq), structType)
|
|
|
|
|
// write to Hudi
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df)
|
|
|
|
|
fail("Drop duplicates with bulk insert in row writing should have thrown exception")
|
|
|
|
|
} catch {
|
|
|
|
|
case e: HoodieException => assertTrue(e.getMessage.contains("Dropping duplicates with bulk_insert in row writer path is not supported yet"))
|
|
|
|
|
@@ -348,7 +345,6 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
//create a new table
|
|
|
|
|
val fooTableModifier = commonTableModifier.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
|
|
|
|
.updated(DataSourceWriteOptions.INSERT_DROP_DUPS.key, "false")
|
|
|
|
|
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
|
|
|
|
|
|
|
|
|
// generate the inserts
|
|
|
|
|
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
|
|
|
|
@@ -357,7 +353,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
val recordsSeq = convertRowListToSeq(records)
|
|
|
|
|
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
|
|
|
|
// write to Hudi
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams - DataSourceWriteOptions.PRECOMBINE_FIELD.key, df)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier - DataSourceWriteOptions.PRECOMBINE_FIELD.key, df)
|
|
|
|
|
|
|
|
|
|
// collect all partition paths to issue read of parquet files
|
|
|
|
|
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
|
|
|
|
@@ -384,7 +380,6 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
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")
|
|
|
|
|
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
|
|
|
|
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
|
|
|
|
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
|
|
|
|
|
val fullPartitionPaths = new Array[String](3)
|
|
|
|
|
@@ -400,7 +395,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
val recordsSeq = convertRowListToSeq(records)
|
|
|
|
|
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
|
|
|
|
// write to Hudi
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df)
|
|
|
|
|
// Fetch records from entire dataset
|
|
|
|
|
val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
|
|
|
|
|
// remove metadata columns so that expected and actual DFs can be compared as is
|
|
|
|
|
@@ -450,7 +445,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
new JavaSparkContext(sc), modifiedSchema.toString, tempBasePath, hoodieFooTableName,
|
|
|
|
|
mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
|
|
|
|
|
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df, Option.empty, Option(client))
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df, Option.empty, Option(client))
|
|
|
|
|
// Verify that asynchronous compaction is not scheduled
|
|
|
|
|
verify(client, times(0)).scheduleCompaction(any())
|
|
|
|
|
// Verify that HoodieWriteClient is closed correctly
|
|
|
|
|
@@ -504,14 +499,14 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
|
|
|
|
initializeMetaClientForBootstrap(fooTableParams, tableType, true)
|
|
|
|
|
|
|
|
|
|
val client = spy(DataSourceUtils.createHoodieClient(
|
|
|
|
|
val client = spy(DataSourceUtils.createHoodieClient(
|
|
|
|
|
new JavaSparkContext(sc),
|
|
|
|
|
null,
|
|
|
|
|
tempBasePath,
|
|
|
|
|
hoodieFooTableName,
|
|
|
|
|
mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
|
|
|
|
|
|
|
|
|
|
HoodieSparkSqlWriter.bootstrap(sqlContext, SaveMode.Append, fooTableParams, spark.emptyDataFrame, Option.empty,
|
|
|
|
|
HoodieSparkSqlWriter.bootstrap(sqlContext, SaveMode.Append, fooTableModifier, spark.emptyDataFrame, Option.empty,
|
|
|
|
|
Option(client))
|
|
|
|
|
|
|
|
|
|
// Verify that HoodieWriteClient is closed correctly
|
|
|
|
|
@@ -556,7 +551,6 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
//create a new table
|
|
|
|
|
val fooTableModifier = getCommonParams(tempPath, hoodieFooTableName, tableType)
|
|
|
|
|
.updated(DataSourceWriteOptions.RECONCILE_SCHEMA.key, "true")
|
|
|
|
|
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
|
|
|
|
|
|
|
|
|
// generate the inserts
|
|
|
|
|
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
|
|
|
|
@@ -564,7 +558,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
var records = DataSourceTestUtils.generateRandomRows(10)
|
|
|
|
|
var recordsSeq = convertRowListToSeq(records)
|
|
|
|
|
val df1 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, fooTableParams, df1)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, fooTableModifier, df1)
|
|
|
|
|
|
|
|
|
|
val snapshotDF1 = spark.read.format("org.apache.hudi")
|
|
|
|
|
.load(tempBasePath + "/*/*/*/*")
|
|
|
|
|
@@ -577,7 +571,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
// issue updates so that log files are created for MOR table
|
|
|
|
|
val updatesSeq = convertRowListToSeq(DataSourceTestUtils.generateUpdates(records, 5))
|
|
|
|
|
val updatesDf = spark.createDataFrame(sc.parallelize(updatesSeq), structType)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, updatesDf)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, updatesDf)
|
|
|
|
|
|
|
|
|
|
val snapshotDF2 = spark.read.format("org.apache.hudi")
|
|
|
|
|
.load(tempBasePath + "/*/*/*/*")
|
|
|
|
|
@@ -595,7 +589,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
recordsSeq = convertRowListToSeq(records)
|
|
|
|
|
val df3 = spark.createDataFrame(sc.parallelize(recordsSeq), evolStructType)
|
|
|
|
|
// write to Hudi with new column
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df3)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df3)
|
|
|
|
|
|
|
|
|
|
val snapshotDF3 = spark.read.format("org.apache.hudi")
|
|
|
|
|
.load(tempBasePath + "/*/*/*/*")
|
|
|
|
|
@@ -610,7 +604,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
records = DataSourceTestUtils.generateRandomRows(10)
|
|
|
|
|
recordsSeq = convertRowListToSeq(records)
|
|
|
|
|
val df4 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df4)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df4)
|
|
|
|
|
|
|
|
|
|
val snapshotDF4 = spark.read.format("org.apache.hudi")
|
|
|
|
|
.load(tempBasePath + "/*/*/*/*")
|
|
|
|
|
@@ -743,14 +737,13 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
@ValueSource(booleans = Array(true, false))
|
|
|
|
|
def testDeletePartitionsV2(usePartitionsToDeleteConfig: Boolean): Unit = {
|
|
|
|
|
val fooTableModifier = getCommonParams(tempPath, hoodieFooTableName, HoodieTableType.COPY_ON_WRITE.name())
|
|
|
|
|
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
|
|
|
|
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
|
|
|
|
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
|
|
|
|
val records = DataSourceTestUtils.generateRandomRows(10)
|
|
|
|
|
val recordsSeq = convertRowListToSeq(records)
|
|
|
|
|
val df1 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
|
|
|
|
// write to Hudi
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, fooTableParams, df1)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, fooTableModifier, df1)
|
|
|
|
|
val snapshotDF1 = spark.read.format("org.apache.hudi")
|
|
|
|
|
.load(tempBasePath + "/*/*/*/*")
|
|
|
|
|
assertEquals(10, snapshotDF1.count())
|
|
|
|
|
@@ -761,7 +754,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
val updatesSeq = convertRowListToSeq(DataSourceTestUtils.generateUpdates(records, 5))
|
|
|
|
|
val updatesDf = spark.createDataFrame(sc.parallelize(updatesSeq), structType)
|
|
|
|
|
// write updates to Hudi
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, updatesDf)
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, updatesDf)
|
|
|
|
|
val snapshotDF2 = spark.read.format("org.apache.hudi")
|
|
|
|
|
.load(tempBasePath + "/*/*/*/*")
|
|
|
|
|
assertEquals(10, snapshotDF2.count())
|
|
|
|
|
@@ -770,7 +763,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
// ensure 2nd batch of updates matches.
|
|
|
|
|
assert(updatesDf.intersect(trimmedDf2).except(updatesDf).count() == 0)
|
|
|
|
|
if (usePartitionsToDeleteConfig) {
|
|
|
|
|
fooTableParams.updated(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key(), HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
|
|
|
|
|
fooTableModifier.updated(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key(), HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
|
|
|
|
|
}
|
|
|
|
|
// delete partitions contains the primary key
|
|
|
|
|
val recordsToDelete = df1.filter(entry => {
|
|
|
|
|
@@ -778,7 +771,7 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
partitionPath.equals(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) ||
|
|
|
|
|
partitionPath.equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)
|
|
|
|
|
})
|
|
|
|
|
val updatedParams = fooTableParams.updated(DataSourceWriteOptions.OPERATION.key(), WriteOperationType.DELETE_PARTITION.name())
|
|
|
|
|
val updatedParams = fooTableModifier.updated(DataSourceWriteOptions.OPERATION.key(), WriteOperationType.DELETE_PARTITION.name())
|
|
|
|
|
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, updatedParams, recordsToDelete)
|
|
|
|
|
val snapshotDF3 = spark.read.format("org.apache.hudi")
|
|
|
|
|
.load(tempBasePath + "/*/*/*/*")
|
|
|
|
|
@@ -819,4 +812,88 @@ class HoodieSparkSqlWriterSuite {
|
|
|
|
|
assert(spark.read.format("hudi").load(tempBasePath).where("age >= 2000").count() == 10)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Test case for no need to specify hiveStylePartitioning/urlEncodePartitioning/KeyGenerator included in HoodieTableConfig except the first time write
|
|
|
|
|
*/
|
|
|
|
|
@Test
|
|
|
|
|
def testToWriteWithoutParametersIncludedInHoodieTableConfig(): Unit = {
|
|
|
|
|
val _spark = spark
|
|
|
|
|
import _spark.implicits._
|
|
|
|
|
val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
|
|
|
|
|
val options = Map(
|
|
|
|
|
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
|
|
|
|
|
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
|
|
|
|
|
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt"
|
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
// case 1: test table which created by sql
|
|
|
|
|
val (tableName1, tablePath1) = ("hoodie_test_params_1", s"$tempBasePath" + "_1")
|
|
|
|
|
spark.sql(
|
|
|
|
|
s"""
|
|
|
|
|
| create table $tableName1 (
|
|
|
|
|
| id int,
|
|
|
|
|
| name string,
|
|
|
|
|
| price double,
|
|
|
|
|
| ts long,
|
|
|
|
|
| dt string
|
|
|
|
|
| ) using hudi
|
|
|
|
|
| partitioned by (dt)
|
|
|
|
|
| options (
|
|
|
|
|
| primaryKey = 'id'
|
|
|
|
|
| )
|
|
|
|
|
| location '$tablePath1'
|
|
|
|
|
""".stripMargin)
|
|
|
|
|
val tableConfig1 = HoodieTableMetaClient.builder()
|
|
|
|
|
.setConf(spark.sparkContext.hadoopConfiguration)
|
|
|
|
|
.setBasePath(tablePath1).build().getTableConfig
|
|
|
|
|
assert(tableConfig1.getHiveStylePartitioningEnable == "true")
|
|
|
|
|
assert(tableConfig1.getUrlEncodePartitoning == "false")
|
|
|
|
|
assert(tableConfig1.getKeyGeneratorClassName == classOf[ComplexKeyGenerator].getName)
|
|
|
|
|
df.write.format("hudi")
|
|
|
|
|
.options(options)
|
|
|
|
|
.option(HoodieWriteConfig.TBL_NAME.key, tableName1)
|
|
|
|
|
.mode(SaveMode.Append).save(tablePath1)
|
|
|
|
|
assert(spark.read.format("hudi").load(tablePath1 + "/*").count() == 1)
|
|
|
|
|
|
|
|
|
|
// case 2: test table which created by dataframe
|
|
|
|
|
val (tableName2, tablePath2) = ("hoodie_test_params_2", s"$tempBasePath" + "_2")
|
|
|
|
|
// the first write need to specify params
|
|
|
|
|
df.write.format("hudi")
|
|
|
|
|
.options(options)
|
|
|
|
|
.option(HoodieWriteConfig.TBL_NAME.key, tableName2)
|
|
|
|
|
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, "true")
|
|
|
|
|
.option(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, classOf[SimpleKeyGenerator].getName)
|
|
|
|
|
.mode(SaveMode.Overwrite).save(tablePath2)
|
|
|
|
|
val tableConfig2 = HoodieTableMetaClient.builder()
|
|
|
|
|
.setConf(spark.sparkContext.hadoopConfiguration)
|
|
|
|
|
.setBasePath(tablePath2).build().getTableConfig
|
|
|
|
|
assert(tableConfig2.getHiveStylePartitioningEnable == "false")
|
|
|
|
|
assert(tableConfig2.getUrlEncodePartitoning == "true")
|
|
|
|
|
assert(tableConfig2.getKeyGeneratorClassName == classOf[SimpleKeyGenerator].getName)
|
|
|
|
|
|
|
|
|
|
val df2 = Seq((2, "a2", 20, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
|
|
|
|
|
// raise exception when use params which is not same with HoodieTableConfig
|
|
|
|
|
try {
|
|
|
|
|
df2.write.format("hudi")
|
|
|
|
|
.options(options)
|
|
|
|
|
.option(HoodieWriteConfig.TBL_NAME.key, tableName2)
|
|
|
|
|
.option(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
|
|
|
|
|
.mode(SaveMode.Append).save(tablePath2)
|
|
|
|
|
} catch {
|
|
|
|
|
case NonFatal(e) =>
|
|
|
|
|
assert(e.getMessage.contains("Config conflict"))
|
|
|
|
|
assert(e.getMessage.contains(
|
|
|
|
|
s"${HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key}\t${classOf[ComplexKeyGenerator].getName}\t${classOf[SimpleKeyGenerator].getName}"))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// do not need to specify hiveStylePartitioning/urlEncodePartitioning/KeyGenerator params
|
|
|
|
|
df2.write.format("hudi")
|
|
|
|
|
.options(options)
|
|
|
|
|
.option(HoodieWriteConfig.TBL_NAME.key, tableName2)
|
|
|
|
|
.mode(SaveMode.Append).save(tablePath2)
|
|
|
|
|
val data = spark.read.format("hudi").load(tablePath2 + "/*")
|
|
|
|
|
assert(data.count() == 2)
|
|
|
|
|
assert(data.select("_hoodie_partition_path").map(_.getString(0)).distinct.collect.head == "dt=2021-10-16")
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|