[HUDI-1129] Improving schema evolution support in hudi (#2927)
* Adding support to ingest records with old schema after table's schema is evolved * Rebasing against latest master - Trimming test file to be < 800 lines - Renaming config names * Addressing feedback Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
committed by
GitHub
parent
73d898322b
commit
1196736185
@@ -23,7 +23,7 @@ import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.SparkRDDWriteClient
|
||||
import org.apache.hudi.common.config.HoodieConfig
|
||||
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord, HoodieRecordPayload}
|
||||
import org.apache.hudi.common.table.HoodieTableConfig
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
@@ -48,7 +48,6 @@ import java.util.{Collections, Date, UUID}
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
var spark: SparkSession = _
|
||||
var sc: SparkContext = _
|
||||
var sqlContext: SQLContext = _
|
||||
@@ -82,15 +81,11 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
test("throw hoodie exception when there already exist a table with different name with Append Save mode") {
|
||||
|
||||
initSparkContext("test_append_mode")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
@@ -149,7 +144,6 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
})
|
||||
|
||||
def testBulkInsertWithSortMode(sortMode: BulkInsertSortMode, path: java.nio.file.Path, populateMetaFields : Boolean = true) : Unit = {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
@@ -193,17 +187,12 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
|
||||
|
||||
if (!populateMetaFields) {
|
||||
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(0)).filter(entry => !(entry.mkString(",").equals(""))).count())
|
||||
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(1)).filter(entry => !(entry.mkString(",").equals(""))).count())
|
||||
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(2)).filter(entry => !(entry.mkString(",").equals(""))).count())
|
||||
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(3)).filter(entry => !(entry.mkString(",").equals(""))).count())
|
||||
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(4)).filter(entry => !(entry.mkString(",").equals(""))).count())
|
||||
List(0, 1, 2, 3, 4).foreach(i => assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(i)).filter(entry => !(entry.mkString(",").equals(""))).count()))
|
||||
}
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
|
||||
|
||||
assert(df.except(trimmedDf).count() == 0)
|
||||
}
|
||||
|
||||
@@ -405,7 +394,6 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
HoodieWriteConfig.BASE_FILE_FORMAT.key -> baseFileFormat,
|
||||
@@ -433,7 +421,6 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
hoodieFooTableName,
|
||||
mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
|
||||
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df, Option.empty,
|
||||
Option(client))
|
||||
// Verify that asynchronous compaction is not scheduled
|
||||
@@ -479,12 +466,9 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val srcPath = java.nio.file.Files.createTempDirectory("hoodie_bootstrap_source_path")
|
||||
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(Instant.now.toEpochMilli, 0, 100, Collections.emptyList(), sc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Write source data non-partitioned
|
||||
sourceDF.write
|
||||
.format("parquet")
|
||||
@@ -533,7 +517,6 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path_schema_evol")
|
||||
try {
|
||||
val hoodieFooTableName = "hoodie_foo_tbl_schema_evolution_" + tableType
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
"hoodie.insert.shuffle.parallelism" -> "1",
|
||||
@@ -541,7 +524,9 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
DataSourceWriteOptions.TABLE_TYPE.key -> tableType,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS.key -> "org.apache.hudi.keygen.SimpleKeyGenerator",
|
||||
DataSourceWriteOptions.RECONCILE_SCHEMA.key -> "true"
|
||||
)
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
@@ -550,7 +535,6 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
var records = DataSourceTestUtils.generateRandomRows(10)
|
||||
var recordsSeq = convertRowListToSeq(records)
|
||||
var df1 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, fooTableParams, df1)
|
||||
|
||||
val snapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
@@ -565,10 +549,9 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
assert(df1.except(trimmedDf1).count() == 0)
|
||||
|
||||
// issue updates so that log files are created for MOR table
|
||||
var updates = DataSourceTestUtils.generateUpdates(records, 5);
|
||||
var updatesSeq = convertRowListToSeq(updates)
|
||||
var updatesDf = spark.createDataFrame(sc.parallelize(updatesSeq), structType)
|
||||
// write updates to Hudi
|
||||
val updates = DataSourceTestUtils.generateUpdates(records, 5);
|
||||
val updatesSeq = convertRowListToSeq(updates)
|
||||
val updatesDf = spark.createDataFrame(sc.parallelize(updatesSeq), structType)
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, updatesDf)
|
||||
|
||||
val snapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
@@ -584,11 +567,11 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
assert(updatesDf.intersect(trimmedDf2).except(updatesDf).count() == 0)
|
||||
|
||||
// getting new schema with new column
|
||||
schema = DataSourceTestUtils.getStructTypeExampleEvolvedSchema
|
||||
structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val evolSchema = DataSourceTestUtils.getStructTypeExampleEvolvedSchema
|
||||
val evolStructType = AvroConversionUtils.convertAvroSchemaToStructType(evolSchema)
|
||||
records = DataSourceTestUtils.generateRandomRowsEvolvedSchema(5)
|
||||
recordsSeq = convertRowListToSeq(records)
|
||||
val df3 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
val df3 = spark.createDataFrame(sc.parallelize(recordsSeq), evolStructType)
|
||||
// write to Hudi with new column
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df3)
|
||||
|
||||
@@ -604,6 +587,25 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
// ensure 2nd batch of updates matches.
|
||||
assert(df3.intersect(trimmedDf3).except(df3).count() == 0)
|
||||
|
||||
// ingest new batch with old schema.
|
||||
records = DataSourceTestUtils.generateRandomRows(10)
|
||||
recordsSeq = convertRowListToSeq(records)
|
||||
val df4 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df4)
|
||||
|
||||
val snapshotDF4 = spark.read.format("org.apache.hudi")
|
||||
.load(path.toAbsolutePath.toString + "/*/*/*/*")
|
||||
assertEquals(25, snapshotDF4.count())
|
||||
|
||||
val tableMetaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(spark.sparkContext.hadoopConfiguration)
|
||||
.setBasePath(path.toAbsolutePath.toString)
|
||||
.build()
|
||||
val actualSchema = new TableSchemaResolver(tableMetaClient).getTableAvroSchemaWithoutMetadataFields
|
||||
assertTrue(actualSchema != null)
|
||||
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(hoodieFooTableName)
|
||||
val expectedSchema = AvroConversionUtils.convertStructTypeToAvroSchema(evolStructType, structName, nameSpace)
|
||||
assertEquals(expectedSchema, actualSchema)
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
@@ -613,8 +615,6 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
test("Test build sync config for spark sql") {
|
||||
initSparkContext("test build sync config")
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val basePath = "/tmp/hoodie_test"
|
||||
val params = Map(
|
||||
"path" -> basePath,
|
||||
@@ -641,7 +641,6 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
test("Test build sync config for skip Ro Suffix vals") {
|
||||
initSparkContext("test build sync config for skip Ro suffix vals")
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val basePath = "/tmp/hoodie_test"
|
||||
val params = Map(
|
||||
"path" -> basePath,
|
||||
@@ -650,7 +649,6 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
)
|
||||
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(params)
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
|
||||
|
||||
val buildSyncConfigMethod =
|
||||
HoodieSparkSqlWriter.getClass.getDeclaredMethod("buildSyncConfig", classOf[Path],
|
||||
classOf[HoodieConfig], classOf[SQLConf])
|
||||
@@ -712,15 +710,14 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
.mode(SaveMode.Append).save(basePath)
|
||||
|
||||
val currentCommits = spark.read.format("hudi").load(basePath).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)
|
||||
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(basePath).select("keyid").orderBy("keyid").count
|
||||
assert(incrementalKeyIdNum == 1000)
|
||||
|
||||
// add bootstap test
|
||||
df.write.mode(SaveMode.Overwrite).save(baseBootStrapPath)
|
||||
// boostrap table
|
||||
spark.emptyDataFrame.write.format("hudi")
|
||||
.options(options)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP.key, baseBootStrapPath)
|
||||
@@ -736,7 +733,8 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
.mode(SaveMode.Append).save(basePath)
|
||||
|
||||
val currentCommitsBootstrap = spark.read.format("hudi").load(basePath).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)
|
||||
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(basePath).select("keyid").orderBy("keyid").count
|
||||
|
||||
@@ -18,16 +18,23 @@
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
|
||||
import java.io.File
|
||||
import java.nio.file.Paths
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.apache.hudi.testutils.DataSourceTestUtils
|
||||
import org.apache.spark.sql.avro.IncompatibleSchemaException
|
||||
import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertNull, assertTrue, fail}
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.io.TempDir
|
||||
|
||||
import java.util
|
||||
import scala.collection.JavaConverters
|
||||
|
||||
class TestHoodieSparkUtils {
|
||||
|
||||
@Test
|
||||
@@ -103,4 +110,124 @@ class TestHoodieSparkUtils {
|
||||
assertEquals(files.sortWith(_.toString < _.toString), indexedFilePaths.sortWith(_.toString < _.toString))
|
||||
spark.stop()
|
||||
}
|
||||
|
||||
@Test
|
||||
def testCreateRddSchemaEvol(): Unit = {
|
||||
val spark = SparkSession.builder
|
||||
.appName("Hoodie Datasource test")
|
||||
.master("local[2]")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.getOrCreate
|
||||
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
var records = DataSourceTestUtils.generateRandomRows(5)
|
||||
var recordsSeq = convertRowListToSeq(records)
|
||||
val df1 = spark.createDataFrame(spark.sparkContext.parallelize(recordsSeq), structType)
|
||||
|
||||
var genRecRDD = HoodieSparkUtils.createRdd(df1, "test_struct_name", "test_namespace", true,
|
||||
org.apache.hudi.common.util.Option.of(schema))
|
||||
genRecRDD.collect()
|
||||
|
||||
val evolSchema = DataSourceTestUtils.getStructTypeExampleEvolvedSchema
|
||||
records = DataSourceTestUtils.generateRandomRowsEvolvedSchema(5)
|
||||
recordsSeq = convertRowListToSeq(records)
|
||||
|
||||
genRecRDD = HoodieSparkUtils.createRdd(df1, "test_struct_name", "test_namespace", true,
|
||||
org.apache.hudi.common.util.Option.of(evolSchema))
|
||||
genRecRDD.collect()
|
||||
|
||||
// pass in evolved schema but with records serialized with old schema. should be able to convert with out any exception.
|
||||
// Before https://github.com/apache/hudi/pull/2927, this will throw exception.
|
||||
genRecRDD = HoodieSparkUtils.createRdd(df1, "test_struct_name", "test_namespace", true,
|
||||
org.apache.hudi.common.util.Option.of(evolSchema))
|
||||
val genRecs = genRecRDD.collect()
|
||||
// if this succeeds w/o throwing any exception, test succeeded.
|
||||
assertEquals(genRecs.size, 5)
|
||||
spark.stop()
|
||||
}
|
||||
|
||||
@Test
|
||||
def testCreateRddWithNestedSchemas(): Unit = {
|
||||
val spark = SparkSession.builder
|
||||
.appName("Hoodie Datasource test")
|
||||
.master("local[2]")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.getOrCreate
|
||||
|
||||
val innerStruct1 = new StructType().add("innerKey","string",false).add("innerValue", "long", true)
|
||||
val structType1 = new StructType().add("key", "string", false)
|
||||
.add("nonNullableInnerStruct",innerStruct1,false).add("nullableInnerStruct",innerStruct1,true)
|
||||
val schema1 = AvroConversionUtils.convertStructTypeToAvroSchema(structType1, "test_struct_name", "test_namespace")
|
||||
val records1 = Seq(Row("key1", Row("innerKey1_1", 1L), Row("innerKey1_2", 2L)))
|
||||
|
||||
val df1 = spark.createDataFrame(spark.sparkContext.parallelize(records1), structType1)
|
||||
val genRecRDD1 = HoodieSparkUtils.createRdd(df1, "test_struct_name", "test_namespace", true,
|
||||
org.apache.hudi.common.util.Option.of(schema1))
|
||||
assert(schema1.equals(genRecRDD1.collect()(0).getSchema))
|
||||
|
||||
// create schema2 which has one addition column at the root level compared to schema1
|
||||
val structType2 = new StructType().add("key", "string", false)
|
||||
.add("nonNullableInnerStruct",innerStruct1,false).add("nullableInnerStruct",innerStruct1,true)
|
||||
.add("nullableInnerStruct2",innerStruct1,true)
|
||||
val schema2 = AvroConversionUtils.convertStructTypeToAvroSchema(structType2, "test_struct_name", "test_namespace")
|
||||
val records2 = Seq(Row("key2", Row("innerKey2_1", 2L), Row("innerKey2_2", 2L), Row("innerKey2_3", 2L)))
|
||||
val df2 = spark.createDataFrame(spark.sparkContext.parallelize(records2), structType2)
|
||||
val genRecRDD2 = HoodieSparkUtils.createRdd(df2, "test_struct_name", "test_namespace", true,
|
||||
org.apache.hudi.common.util.Option.of(schema2))
|
||||
assert(schema2.equals(genRecRDD2.collect()(0).getSchema))
|
||||
|
||||
// send records1 with schema2. should succeed since the new column is nullable.
|
||||
val genRecRDD3 = HoodieSparkUtils.createRdd(df1, "test_struct_name", "test_namespace", true,
|
||||
org.apache.hudi.common.util.Option.of(schema2))
|
||||
assert(genRecRDD3.collect()(0).getSchema.equals(schema2))
|
||||
genRecRDD3.foreach(entry => assertNull(entry.get("nonNullableInnerStruct2")))
|
||||
|
||||
val innerStruct3 = new StructType().add("innerKey","string",false).add("innerValue", "long", true)
|
||||
.add("new_nested_col","string",true)
|
||||
|
||||
// create a schema which has one additional nested column compared to schema1, which is nullable
|
||||
val structType4 = new StructType().add("key", "string", false)
|
||||
.add("nonNullableInnerStruct",innerStruct1,false).add("nullableInnerStruct",innerStruct3,true)
|
||||
|
||||
val schema4 = AvroConversionUtils.convertStructTypeToAvroSchema(structType4, "test_struct_name", "test_namespace")
|
||||
val records4 = Seq(Row("key2", Row("innerKey2_1", 2L), Row("innerKey2_2", 2L, "new_nested_col_val1")))
|
||||
val df4 = spark.createDataFrame(spark.sparkContext.parallelize(records4), structType4)
|
||||
val genRecRDD4 = HoodieSparkUtils.createRdd(df4, "test_struct_name", "test_namespace", true,
|
||||
org.apache.hudi.common.util.Option.of(schema4))
|
||||
assert(schema4.equals(genRecRDD4.collect()(0).getSchema))
|
||||
|
||||
// convert batch 1 with schema4. should succeed.
|
||||
val genRecRDD5 = HoodieSparkUtils.createRdd(df1, "test_struct_name", "test_namespace", true,
|
||||
org.apache.hudi.common.util.Option.of(schema4))
|
||||
assert(schema4.equals(genRecRDD4.collect()(0).getSchema))
|
||||
val genRec = genRecRDD5.collect()(0)
|
||||
val nestedRec : GenericRecord = genRec.get("nullableInnerStruct").asInstanceOf[GenericRecord]
|
||||
assertNull(nestedRec.get("new_nested_col"))
|
||||
assertNotNull(nestedRec.get("innerKey"))
|
||||
assertNotNull(nestedRec.get("innerValue"))
|
||||
|
||||
val innerStruct4 = new StructType().add("innerKey","string",false).add("innerValue", "long", true)
|
||||
.add("new_nested_col","string",false)
|
||||
// create a schema which has one additional nested column compared to schema1, which is non nullable
|
||||
val structType6 = new StructType().add("key", "string", false)
|
||||
.add("nonNullableInnerStruct",innerStruct1,false).add("nullableInnerStruct",innerStruct4,true)
|
||||
|
||||
val schema6 = AvroConversionUtils.convertStructTypeToAvroSchema(structType6, "test_struct_name", "test_namespace")
|
||||
// convert batch 1 with schema5. should fail since the missed out column is not nullable.
|
||||
try {
|
||||
val genRecRDD6 = HoodieSparkUtils.createRdd(df1, "test_struct_name", "test_namespace", true,
|
||||
org.apache.hudi.common.util.Option.of(schema6))
|
||||
genRecRDD6.collect()
|
||||
fail("createRdd should fail, because records don't have a column which is not nullable in the passed in schema")
|
||||
} catch {
|
||||
case e: Exception =>
|
||||
e.getCause.asInstanceOf[NullPointerException]
|
||||
assertTrue(e.getMessage.contains("null of string in field new_nested_col of"))
|
||||
}
|
||||
spark.stop()
|
||||
}
|
||||
|
||||
def convertRowListToSeq(inputList: util.List[Row]): Seq[Row] =
|
||||
JavaConverters.asScalaIteratorConverter(inputList.iterator).asScala.toSeq
|
||||
}
|
||||
|
||||
@@ -17,19 +17,15 @@
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import java.sql.{Date, Timestamp}
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.JavaConverters._
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.deleteRecordsToStrings
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.{deleteRecordsToStrings, recordsToStrings}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieUpsertException
|
||||
import org.apache.hudi.keygen._
|
||||
import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.Config
|
||||
import org.apache.hudi.keygen._
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.spark.sql._
|
||||
@@ -42,6 +38,10 @@ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
|
||||
|
||||
import java.sql.{Date, Timestamp}
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
|
||||
/**
|
||||
* Basic tests on the spark datasource for COW table.
|
||||
@@ -532,10 +532,8 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
var writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName)
|
||||
writer.partitionBy("current_ts")
|
||||
.save(basePath)
|
||||
|
||||
var recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*")
|
||||
|
||||
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= col("current_ts").cast("string")).count() == 0)
|
||||
|
||||
// Specify fieldType as TIMESTAMP
|
||||
@@ -544,10 +542,8 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
.option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS")
|
||||
.option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd")
|
||||
.save(basePath)
|
||||
|
||||
recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*")
|
||||
|
||||
val udf_date_format = udf((data: Long) => new DateTime(data).toString(DateTimeFormat.forPattern("yyyyMMdd")))
|
||||
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= udf_date_format(col("current_ts"))).count() == 0)
|
||||
|
||||
@@ -557,7 +553,6 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
.option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS")
|
||||
.option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd")
|
||||
.save(basePath)
|
||||
|
||||
recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*")
|
||||
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!=
|
||||
@@ -582,20 +577,16 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
var writer = getDataFrameWriter(classOf[SimpleKeyGenerator].getName)
|
||||
writer.partitionBy("driver")
|
||||
.save(basePath)
|
||||
|
||||
var recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*")
|
||||
|
||||
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= col("driver")).count() == 0)
|
||||
|
||||
// Use the `driver,rider` field as the partition key, If no such field exists, the default value `default` is used
|
||||
writer = getDataFrameWriter(classOf[SimpleKeyGenerator].getName)
|
||||
writer.partitionBy("driver", "rider")
|
||||
.save(basePath)
|
||||
|
||||
recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*")
|
||||
|
||||
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("default")).count() == 0)
|
||||
}
|
||||
|
||||
@@ -604,20 +595,16 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
var writer = getDataFrameWriter(classOf[ComplexKeyGenerator].getName)
|
||||
writer.partitionBy("driver")
|
||||
.save(basePath)
|
||||
|
||||
var recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*")
|
||||
|
||||
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= col("driver")).count() == 0)
|
||||
|
||||
// Use the `driver`,`rider` field as the partition key
|
||||
writer = getDataFrameWriter(classOf[ComplexKeyGenerator].getName)
|
||||
writer.partitionBy("driver", "rider")
|
||||
.save(basePath)
|
||||
|
||||
recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*")
|
||||
|
||||
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= concat(col("driver"), lit("/"), col("rider"))).count() == 0)
|
||||
}
|
||||
|
||||
@@ -649,7 +636,6 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
var writer = getDataFrameWriter(classOf[NonpartitionedKeyGenerator].getName)
|
||||
writer.partitionBy("")
|
||||
.save(basePath)
|
||||
|
||||
var recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*")
|
||||
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("")).count() == 0)
|
||||
@@ -658,7 +644,6 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
writer = getDataFrameWriter(classOf[NonpartitionedKeyGenerator].getName)
|
||||
writer.partitionBy("abc")
|
||||
.save(basePath)
|
||||
|
||||
recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*")
|
||||
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("")).count() == 0)
|
||||
@@ -717,9 +702,10 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
@Test def testSchemaEvolution(): Unit = {
|
||||
// open the schema validate
|
||||
val opts = commonOpts ++ Map("hoodie.avro.schema.validate" -> "true")
|
||||
val opts = commonOpts ++ Map("hoodie.avro.schema.validate" -> "true") ++
|
||||
Map(DataSourceWriteOptions.RECONCILE_SCHEMA.key() -> "true")
|
||||
// 1. write records with schema1
|
||||
val schema1 = StructType(StructField("_row_key", StringType, true) :: StructField("name", StringType, true)::
|
||||
val schema1 = StructType(StructField("_row_key", StringType, true) :: StructField("name", StringType, false)::
|
||||
StructField("timestamp", IntegerType, true) :: StructField("partition", IntegerType, true)::Nil)
|
||||
val records1 = Seq(Row("1", "Andy", 1, 1),
|
||||
Row("2", "lisi", 1, 1),
|
||||
@@ -732,10 +718,9 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
.save(basePath)
|
||||
|
||||
// 2. write records with schema2 add column age
|
||||
val schema2 = StructType(StructField("_row_key", StringType, true) :: StructField("name", StringType, true) ::
|
||||
val schema2 = StructType(StructField("_row_key", StringType, true) :: StructField("name", StringType, false) ::
|
||||
StructField("age", StringType, true) :: StructField("timestamp", IntegerType, true) ::
|
||||
StructField("partition", IntegerType, true)::Nil)
|
||||
|
||||
val records2 = Seq(Row("11", "Andy", "10", 1, 1),
|
||||
Row("22", "lisi", "11",1, 1),
|
||||
Row("33", "zhangsan", "12", 1, 1))
|
||||
@@ -745,24 +730,25 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
.options(opts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*")
|
||||
val resultSchema = new StructType(recordsReadDF.schema.filter(p=> !p.name.startsWith("_hoodie")).toArray)
|
||||
assertEquals(resultSchema, schema2)
|
||||
val tableMetaClient = HoodieTableMetaClient.builder().setConf(spark.sparkContext.hadoopConfiguration).setBasePath(basePath).build()
|
||||
val actualSchema = new TableSchemaResolver(tableMetaClient).getTableAvroSchemaWithoutMetadataFields
|
||||
assertTrue(actualSchema != null)
|
||||
val actualStructType = AvroConversionUtils.convertAvroSchemaToStructType(actualSchema)
|
||||
assertEquals(actualStructType, schema2)
|
||||
|
||||
// 3. write records with schema3 delete column name
|
||||
// 3. write records with schema4 by omitting a non nullable column(name). should fail
|
||||
try {
|
||||
val schema3 = StructType(StructField("_row_key", StringType, true) ::
|
||||
val schema4 = StructType(StructField("_row_key", StringType, true) ::
|
||||
StructField("age", StringType, true) :: StructField("timestamp", IntegerType, true) ::
|
||||
StructField("partition", IntegerType, true)::Nil)
|
||||
|
||||
val records3 = Seq(Row("11", "10", 1, 1),
|
||||
val records4 = Seq(Row("11", "10", 1, 1),
|
||||
Row("22", "11",1, 1),
|
||||
Row("33", "12", 1, 1))
|
||||
val rdd3 = jsc.parallelize(records3)
|
||||
val recordsDF3 = spark.createDataFrame(rdd3, schema3)
|
||||
recordsDF3.write.format("org.apache.hudi")
|
||||
val rdd4 = jsc.parallelize(records4)
|
||||
val recordsDF4 = spark.createDataFrame(rdd4, schema4)
|
||||
recordsDF4.write.format("org.apache.hudi")
|
||||
.options(opts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
@@ -777,19 +763,15 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val opts = commonOpts ++ Map("hoodie.avro.schema.validate" -> "true")
|
||||
val schema1 = StructType(StructField("_row_key", StringType, true) :: StructField("name", StringType, true)::
|
||||
StructField("timestamp", IntegerType, true):: StructField("age", StringType, true) :: StructField("partition", IntegerType, true)::Nil)
|
||||
|
||||
val records = Array("{\"_row_key\":\"1\",\"name\":\"lisi\",\"timestamp\":1,\"partition\":1}",
|
||||
"{\"_row_key\":\"1\",\"name\":\"lisi\",\"timestamp\":1,\"partition\":1}")
|
||||
|
||||
val inputDF = spark.read.schema(schema1.toDDL).json(spark.sparkContext.parallelize(records, 2))
|
||||
|
||||
inputDF.write.format("org.apache.hudi")
|
||||
.options(opts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*")
|
||||
|
||||
val resultSchema = new StructType(recordsReadDF.schema.filter(p=> !p.name.startsWith("_hoodie")).toArray)
|
||||
assertEquals(resultSchema, schema1)
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user