-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-4588][HUDI-4472] Addressing schema handling issues in the write path #6358
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
378a375 to
3c4adb6
Compare
3e4361a to
8915ca3
Compare
HoodieParquetReader to properly specify projected schema when reading Parquet fileHoodieParquetReader to properly specify projected schema when reading Parquet file
| SchemaCompatibility.SchemaPairCompatibility result = | ||
| SchemaCompatibility.checkReaderWriterCompatibility(newSchema, prevSchema); | ||
| return result.getType() == SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Basically the schema check is relaxed here without field-by-field comparison, with only read compatibility check.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Discussed offline: previously we had custom implementation that was specifically disallowing columns being dropped. Since we're relaxing this constraint in this PR there's no reason for us to continue maintain that custom logic. Instead we can solely rely on Avro schema compatibility checking.
| // Register Avro classes ([[Schema]], [[GenericData]]) w/ Kryo | ||
| sparkContext.getConf.registerKryoClasses( | ||
| Array(classOf[org.apache.avro.generic.GenericData], | ||
| classOf[org.apache.avro.Schema])) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do we need this now?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We always had that, this code just been moved from below to make sure we handle the schema in the same way for bulk-insert (w/ row-writing) as we do for any other operation
| val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tblName) | ||
| val reconcileSchema = parameters(DataSourceWriteOptions.RECONCILE_SCHEMA.key()).toBoolean | ||
|
|
||
| val schemaEvolutionEnabled = parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), "false").toBoolean | ||
| var internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, sparkContext) | ||
|
|
||
| val sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace) | ||
| val latestTableSchemaOpt = getLatestTableSchema(spark, basePath, tableIdentifier, sparkContext.hadoopConfiguration) | ||
|
|
||
| val writerSchema: Schema = latestTableSchemaOpt match { | ||
| // In case table schema is empty we're just going to use the source schema as a | ||
| // writer's schema. No additional handling is required | ||
| case None => sourceSchema | ||
| // Otherwise, we need to make sure we reconcile incoming and latest table schemas | ||
| case Some(latestTableSchema) => | ||
| if (reconcileSchema) { | ||
| // In case we need to reconcile the schema and schema evolution is enabled, | ||
| // we will force-apply schema evolution to the writer's schema | ||
| if (schemaEvolutionEnabled && internalSchemaOpt.isEmpty) { | ||
| internalSchemaOpt = Some(AvroInternalSchemaConverter.convert(sourceSchema)) | ||
| } | ||
|
|
||
| if (internalSchemaOpt.isDefined) { | ||
| // Apply schema evolution, by auto-merging write schema and read schema | ||
| val mergedInternalSchema = AvroSchemaEvolutionUtils.reconcileSchema(sourceSchema, internalSchemaOpt.get) | ||
| AvroInternalSchemaConverter.convert(mergedInternalSchema, latestTableSchema.getName) | ||
| } else if (TableSchemaResolver.isSchemaCompatible(sourceSchema, latestTableSchema)) { | ||
| // In case schema reconciliation is enabled and source and latest table schemas | ||
| // are compatible (as defined by [[TableSchemaResolver#isSchemaCompatible]]), then we | ||
| // will rebase incoming batch onto the table's latest schema (ie, reconcile them) | ||
| // | ||
| // NOTE: Since we'll be converting incoming batch from [[sourceSchema]] into [[latestTableSchema]] | ||
| // we're validating in that order (where [[sourceSchema]] is treated as a reader's schema, | ||
| // and [[latestTableSchema]] is treated as a writer's schema) | ||
| latestTableSchema | ||
| } else { | ||
| log.error( | ||
| s""" | ||
| |Failed to reconcile incoming batch schema with the table's one. | ||
| |Incoming schema ${sourceSchema.toString(true)} | ||
|
|
||
| |Table's schema ${latestTableSchema.toString(true)} | ||
|
|
||
| |""".stripMargin) | ||
| throw new SchemaCompatibilityException("Failed to reconcile incoming schema with the table's one") | ||
| } | ||
| } else { | ||
| // Before validating whether schemas are compatible, we need to "canonicalize" source's schema | ||
| // relative to the table's one, by doing a (minor) reconciliation of the nullability constraints: | ||
| // for ex, if in incoming schema column A is designated as non-null, but it's designated as nullable | ||
| // in the table's one we want to proceed w/ such operation, simply relaxing such constraint in the | ||
| // source schema. | ||
| val canonicalizedSourceSchema = AvroSchemaEvolutionUtils.canonicalizeColumnNullability(sourceSchema, latestTableSchema) | ||
| // In case reconciliation is disabled, we have to validate that the source's schema | ||
| // is compatible w/ the table's latest schema, such that we're able to read existing table's | ||
| // records using [[sourceSchema]]. | ||
| if (TableSchemaResolver.isSchemaCompatible(latestTableSchema, canonicalizedSourceSchema)) { | ||
| canonicalizedSourceSchema | ||
| } else { | ||
| log.error( | ||
| s""" | ||
| |Incoming batch schema is not compatible with the table's one. | ||
| |Incoming schema ${canonicalizedSourceSchema.toString(true)} | ||
| |Table's schema ${latestTableSchema.toString(true)} | ||
| |""".stripMargin) | ||
| throw new SchemaCompatibilityException("Incoming batch schema is not compatible with the table's one") | ||
| } | ||
| } | ||
| } | ||
|
|
||
| validateSchemaForHoodieIsDeleted(writerSchema) | ||
|
|
||
| // NOTE: PLEASE READ CAREFULLY BEFORE CHANGING THIS | ||
| // We have to register w/ Kryo all of the Avro schemas that might potentially be used to decode | ||
| // records into Avro format. Otherwise, Kryo wouldn't be able to apply an optimization allowing | ||
| // it to avoid the need to ser/de the whole schema along _every_ Avro record | ||
| val targetAvroSchemas = sourceSchema +: writerSchema +: latestTableSchemaOpt.toSeq |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It looks like this schema reconcilation logic is not specific to Spark. Could we extract it and allow Flink/Java engine to leverage it as well?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good call! Am going to extract it as a standalone method (we can take the unification as a follow-up)
7d71695 to
c4b6bb8
Compare
|
As outlined in #6196 (comment), this PR should go hand in hand w/ the #6196, which flips Schema Reconciliation to be enabled by default (entailing that every incoming batch would be reconciled relative to the table's schema) |
a0ebc2a to
d19a4db
Compare
|
@alexeykudinkin could you check the CI failure? |
d4f0c5f to
2382b4e
Compare
| val trimmedSourceDF = removeMetaFields(sourceDF) | ||
|
|
||
| // Supply original record's Avro schema to provided to [[ExpressionPayload]] | ||
| writeParams += (PAYLOAD_RECORD_AVRO_SCHEMA -> |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@wzx140 this is what i'm referring to
5558316 to
ae59f6f
Compare
|
@hudi-bot run azure |
| HoodieMergeHandle<T, HoodieData<HoodieRecord<T>>, HoodieData<HoodieKey>, HoodieData<WriteStatus>> mergeHandle) throws IOException { | ||
| final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation(); | ||
| Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf()); | ||
| HoodieWriteConfig writeConfig = table.getConfig(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This piece had to change b/c we're now setting up a projection schema when reading from Parquet and that fails unless projection schema is matching the file-schema. This unfortunately has not been the case as this code was evolving over the years, so it forced me to untangle flows here to fix it.
Changes:
- Abstracted and extracted Advanced Schema Evolution into standalone method (this code doesn't change, except it was abstracted to return a function rewriting the record instead of returning multiple values)
- Simplified logic pertaining to record rewriting to be shared by all flows (bootstrapping, etc)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 def looks better now.
|
|
||
| public InternalSchema(List<Field> columns) { | ||
| this(DEFAULT_VERSION_ID, columns); | ||
| public InternalSchema(RecordType recordType) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
These changes are necessary to allow to preserve Avro record's name during conversion to InternalSchema and back.
Changes
- Rebasing ctors to accept
RecordTypeinstead of list of fields - Inlining redundant ctors
| log.info(s"Registered avro schema : ${schema.toString(true)}") | ||
| if (parameters(INSERT_DROP_DUPS.key).toBoolean) { | ||
| writerSchema: Schema): (Boolean, common.util.Option[String]) = { | ||
| if (hoodieConfig.getBoolean(INSERT_DROP_DUPS)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
These changes make sure that Row-writing Bulk Insert flow is relying on the same schema handling as any other operation:
Changes
- Rebased to rely on
writerSchemapassed in externally - Cleaning up duplication
| var writeParams = parameters + | ||
| (OPERATION.key -> operation) + | ||
| (HoodieWriteConfig.WRITE_SCHEMA.key -> getTableSchema.toString) + | ||
| (HoodieWriteConfig.WRITE_SCHEMA_OVERRIDE.key -> getTableSchema.toString) + |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
These changes are necessary to align and streamline schema handling for MERGE INTO operation.
Changes:
- Make sure it passes expected record's schema to the ExpressionPayload (via
PAYLOAD_RECORD_AVRO_SCHEMAprop) - Make sure proper configs are specified to disable inapplicable operations such as: a) schema reconciliation, b) schema canonicalization (null/non-null), c) schema validation (since input to MI might not necessarily be matching the target table
2441295 to
f873230
Compare
| public static final ConfigProperty<String> AVRO_SCHEMA_VALIDATE_ENABLE = ConfigProperty | ||
| .key("hoodie.avro.schema.validate") | ||
| .defaultValue("false") | ||
| .defaultValue("true") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is flipped to default to make sure proper schema validation are run for every operation on the table
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java
Show resolved
Hide resolved
| */ | ||
| public abstract void runMerge(HoodieTable<T, I, K, O> table, HoodieMergeHandle<T, I, K, O> upsertHandle) throws IOException; | ||
|
|
||
| protected GenericRecord transformRecordBasedOnNewSchema(GenericDatumReader<GenericRecord> gReader, GenericDatumWriter<GenericRecord> gWriter, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This method is not necessary anymore
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what was the purpose of this method?
my understanding is.
incase of Bootstrapping feature, we need to read records from old file using the schema w/ which it was written and then read back using newer schema thats of interest to us.
also,
Within composeSchemaEvolutionTransformer() as per this patch, we could return Option.empty() when there is no InternalSchema even for bootstrap scenarios. So, in that case, there is no rewrite of records happening. can you help throw some light please.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
-
Yes, this method was rewriting records, but it was using ser/de for that. Now, records are being rewritten using
rewriteRecord*utils. -
composeSchemaEvolutionTransformeronly refers to schema evolution case. Bootstrap case is handled as before (it wasn't calling into this method previously)
| * </ol> | ||
| * | ||
| */ | ||
| public class AvroSchemaCompatibility { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Context: Avro requires at all times that schema's names have to match in order for them to be counted as compatible. Provided that only Avro bears the names on the schemas themselves (Spark does not, for ex) this makes for ex, some schemas converted from Spark's [[StructType]] incompatible w/ Avro
This code is mostly borrowed as is from Avro 1.10 w/ the following critical adjustments:
-
Schema names now are only checked in following 2 cases:
1.a In case it's a top-level schema
1.b In case schema is enclosed into a union (in which case its name might be used for reverse-lookup) -
Default value access is adapted to work for both Avro 1.8.x as well as 1.10.x
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we avoid to copy the clazz directly here, it's hard to maintain for following avro upgrade, say Avro may have some bug fix for it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm not a big fan of such borrowings myself, but unfortunately there's no other way for us to extend it to modify it's behavior: we need to avoid namespace matching, b/c we do numerous rounds of conversions Avro > Spark > InternalSchema and back and b/c of that we can't reconstruct the namespaces that sometimes used/gen'd by Avro.
f873230 to
18ef7b4
Compare
6cd469c to
5357f6f
Compare
…ching atomic-types
… path (apache#6358) Currently, HoodieParquetReader is not specifying projected schema properly when reading Parquet files which ends up failing in cases when the provided schema is not equal to the schema of the file being read (even though it might be a proper projection, ie subset of it), like in common CDC case when column is being dropped from the RDS and the schema of the new batch is lacking the old column. To address the original issue described in HUDI-4588, we also have to relax the constraints imposed by TableSchemaResolver.isSchemaCompatible method not allowing columns to be evolved by the way of dropping columns. After addressing the original problem a considerable amount of new issues have been discovered including following: - Writer's schemas were deduced differently for different flows (for ex, Bulk Insert vs other ops) - Writer's schema was not reconciled against table's schemas in terms of nullability (this is asserted w/in AvroSchemaConverter, that is now invoked as part of the projection) - (After enabling schema validations) Incorrect schema handling w/in HoodieWriteHandle has been detected (there were 2 ambiguous schema references, set incorrectly, creating confusion) - (After enabling schema validations) Incorrect schema handling w/in MergeHelper impls, where writer's schema was used as a existing file's reader-schema (failing in cases when these 2 diverge) Changes: - Adding missing schema projection when reading Parquet file (using AvroParquetReader) - Relaxing schema evolution constraints to allow columns to be dropped - Revisiting schema reconciliation logic to make sure it's coherent - Streamlining schema handling in HoodieSparkSqlWriter to make sure it's uniform for all operations (it isn't applied properly for Bulk-insert at the moment) - Added comprehensive test for Basic Schema Evolution (columns being added, dropped) - Fixing HoodieWriteHandle impls to properly handle writer schema and avoid duplication - Fixing MergeHelper impls to properly handle schema evolution
|
@aditiwari01 @ad1happy2go @alexeykudinkin alexeykudinkin Execuse me. I use hudi 0.13.1 and set hoodie.datasource.write.reconcile.schema=true; but, spark-sql with hudi query still error. Caused by: org.apache.hudi.exception.HoodieException: Exception when reading log file |
| } | ||
|
|
||
| latestTableSchemaFromCommitMetadata.orElse { | ||
| getCatalogTable(spark, tableId).map { catalogTable => |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
not sure why we added this fix to poll the catalog if its table does not have any valid commits.
we should just remove this.

Change Logs
Currently,
HoodieParquetReaderis not specifying projected schema properly when reading Parquet files which ends up failing in cases when the provided schema is not equal to the schema of the file being read (even though it might be a proper projection, ie subset of it), like in common CDC case when column is being dropped from the RDS and the schema of the new batch is lacking the old column.To address the original issue described in HUDI-4588, we also have to relax the constraints imposed by
TableSchemaResolver.isSchemaCompatiblemethod not allowing columns to be evolved by the way of dropping columns.After addressing the original problem a considerable amount of new issues have been discovered including following:
AvroSchemaConverter, that is now invoked as part of the projection)HoodieWriteHandlehas been detected (there were 2 ambiguous schema references, set incorrectly, creating confusion)MergeHelperimpls, where writer's schema was used as a existing file's reader-schema (failing in cases when these 2 diverge)Changes:
AvroParquetReader)HoodieSparkSqlWriterto make sure it's uniform for all operations (it isn't applied properly for Bulk-insert at the moment)HoodieWriteHandleimpls to properly handle writer schema and avoid duplicationMergeHelperimpls to properly handle schema evolutionImpact
High
There are a few critical changes taken forward by this PR:
Now, w/ incoming batches being able to drop columns (relative to the Table's existing schema), unless
hoodie.datasource.write.reconcile.schemais enabled:This subtle change in behavior (dropped columns would not be leading to failures anymore) could open up a new set of problems where data quality issues (for ex, column is missing while it shouldn't) could trickle down into existing table.
To alleviate that we should consider flipping
hoodie.datasource.write.reconcile.schematotrueby default (there's already #6196 for that)Contributor's checklist