1
0

[HUDI-4276] Reconcile schema-inject null values for missing fields and add new fields (#6017)

* [HUDI-4276] Reconcile schema-inject null values for missing fields and add new fields.

* fix comments

Co-authored-by: public (bdcee5037027) <mengtao0326@qq.com>
This commit is contained in:
xiarixiaoyao
2022-07-09 03:08:38 +08:00
committed by GitHub
parent fc8d96246a
commit b686c07407
15 changed files with 277 additions and 114 deletions

View File

@@ -199,9 +199,7 @@ class TestHoodieSparkUtils {
fail("createRdd should fail, because records don't have a column which is not nullable in the passed in schema")
} catch {
case e: Exception =>
val cause = e.getCause
assertTrue(cause.isInstanceOf[SchemaCompatibilityException])
assertTrue(e.getMessage.contains("Unable to validate the rewritten record {\"innerKey\": \"innerKey1_2\", \"innerValue\": 2} against schema"))
assertTrue(e.getMessage.contains("null of string in field new_nested_col of test_namespace.test_struct_name.nullableInnerStruct of union"))
}
spark.stop()
}

View File

@@ -19,10 +19,13 @@ package org.apache.spark.sql.hudi
import org.apache.hadoop.fs.Path
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig}
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.common.testutils.RawTripTestPayload
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
import org.apache.spark.sql.functions.{arrays_zip, col}
import org.apache.spark.sql.{Row, SaveMode, SparkSession}
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
@@ -460,4 +463,65 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
}
}
}
test("Test schema auto evolution") {
withTempDir { tmp =>
Seq("COPY_ON_WRITE", "MERGE_ON_READ").foreach { tableType =>
val tableName = generateTableName
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
if (HoodieSparkUtils.gteqSpark3_1) {
val dataGen = new HoodieTestDataGenerator
val schema = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA
val records1 = RawTripTestPayload.recordsToStrings(dataGen.generateInsertsAsPerSchema("001", 1000, schema)).toList
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
// drop tip_history.element.amount, city_to_state, distance_in_meters, drivers
val orgStringDf = inputDF1.drop("city_to_state", "distance_in_meters", "drivers")
.withColumn("tip_history", arrays_zip(col("tip_history.currency")))
spark.sql("set hoodie.schema.on.read.enable=true")
val hudiOptions = Map[String,String](
HoodieWriteConfig.TABLE_NAME -> tableName,
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType,
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
"hoodie.schema.on.read.enable" -> "true",
"hoodie.datasource.write.reconcile.schema" -> "true",
DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY -> "true"
)
orgStringDf.write
.format("org.apache.hudi")
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.options(hudiOptions)
.mode(SaveMode.Overwrite)
.save(tablePath)
val oldView = spark.read.format("hudi").load(tablePath)
oldView.show(false)
val records2 = RawTripTestPayload.recordsToStrings(dataGen.generateUpdatesAsPerSchema("002", 100, schema)).toList
val inputD2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
val updatedStringDf = inputD2.drop("fare").drop("height")
val checkRowKey = inputD2.select("_row_key").collectAsList().map(_.getString(0)).get(0)
updatedStringDf.write
.format("org.apache.hudi")
.options(hudiOptions)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
.option("hoodie.datasource.write.reconcile.schema", "true")
.mode(SaveMode.Append)
.save(tablePath)
spark.read.format("hudi").load(tablePath).registerTempTable("newView")
val checkResult = spark.sql(s"select tip_history.amount,city_to_state,distance_in_meters,fare,height from newView where _row_key='$checkRowKey' ")
.collect().map(row => (row.isNullAt(0), row.isNullAt(1), row.isNullAt(2), row.isNullAt(3), row.isNullAt(4)))
assertResult((false, false, false, true, true))(checkResult(0))
checkAnswer(spark.sql(s"select fare,height from newView where _row_key='$checkRowKey'").collect())(
Seq(null, null)
)
}
}
}
}
}