[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:
@@ -399,12 +399,7 @@ object DataSourceWriteOptions {
|
||||
.defaultValue(classOf[HiveSyncTool].getName)
|
||||
.withDocumentation("Sync tool class name used to sync to metastore. Defaults to Hive.")
|
||||
|
||||
val RECONCILE_SCHEMA: ConfigProperty[Boolean] = ConfigProperty
|
||||
.key("hoodie.datasource.write.reconcile.schema")
|
||||
.defaultValue(false)
|
||||
.withDocumentation("When a new batch of write has records with old schema, but latest table schema got "
|
||||
+ "evolved, this config will upgrade the records to leverage latest table schema(default values will be "
|
||||
+ "injected to missing fields). If not, the write batch would fail.")
|
||||
val RECONCILE_SCHEMA: ConfigProperty[Boolean] = HoodieCommonConfig.RECONCILE_SCHEMA
|
||||
|
||||
// HIVE SYNC SPECIFIC CONFIGS
|
||||
// NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes
|
||||
|
||||
@@ -40,6 +40,7 @@ import org.apache.hudi.hive.{HiveSyncConfigHolder, HiveSyncTool}
|
||||
import org.apache.hudi.index.SparkHoodieIndexFactory
|
||||
import org.apache.hudi.internal.DataSourceInternalWriterHelper
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
|
||||
import org.apache.hudi.internal.schema.utils.{AvroSchemaEvolutionUtils, SerDeHelper}
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||
import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator}
|
||||
@@ -242,16 +243,29 @@ object HoodieSparkSqlWriter {
|
||||
classOf[org.apache.avro.Schema]))
|
||||
var schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace)
|
||||
val lastestSchema = getLatestTableSchema(fs, basePath, sparkContext, schema)
|
||||
val internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, sparkContext)
|
||||
var internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, sparkContext)
|
||||
if (reconcileSchema && parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), "false").toBoolean
|
||||
&& internalSchemaOpt.isEmpty) {
|
||||
// force apply full schema evolution.
|
||||
internalSchemaOpt = Some(AvroInternalSchemaConverter.convert(schema))
|
||||
}
|
||||
if (reconcileSchema) {
|
||||
schema = lastestSchema
|
||||
}
|
||||
if (internalSchemaOpt.isDefined) {
|
||||
schema = {
|
||||
val newSparkSchema = AvroConversionUtils.convertAvroSchemaToStructType(AvroSchemaEvolutionUtils.canonicalizeColumnNullability(schema, lastestSchema))
|
||||
AvroConversionUtils.convertStructTypeToAvroSchema(newSparkSchema, structName, nameSpace)
|
||||
|
||||
// Apply schema evolution.
|
||||
val mergedSparkSchema = if (!reconcileSchema) {
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(AvroSchemaEvolutionUtils.canonicalizeColumnNullability(schema, lastestSchema))
|
||||
} else {
|
||||
// Auto merge write schema and read schema.
|
||||
val mergedInternalSchema = AvroSchemaEvolutionUtils.reconcileSchema(schema, internalSchemaOpt.get)
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(AvroInternalSchemaConverter.convert(mergedInternalSchema, lastestSchema.getName))
|
||||
}
|
||||
schema = AvroConversionUtils.convertStructTypeToAvroSchema(mergedSparkSchema, structName, nameSpace)
|
||||
}
|
||||
|
||||
if (reconcileSchema && internalSchemaOpt.isEmpty) {
|
||||
schema = lastestSchema
|
||||
}
|
||||
validateSchemaForHoodieIsDeleted(schema)
|
||||
sparkContext.getConf.registerAvroSchemas(schema)
|
||||
|
||||
Reference in New Issue
Block a user