Skip to content

Conversation

@xiarixiaoyao
Copy link
Contributor

Change Logs

Fix the bug: After performing the update operation, the hodie table cannot be read normally by spark

Impact

if user create table with decimal type,
after performing the update operation, the hodie table cannot be read normally by spark

Risk level (write none, low medium or high below)

high

Documentation Update

Describe any necessary documentation update if there is any new feature, config, or user-facing change

  • The config description must be updated if new configs are added or the default value of the configs are changed
  • Any new feature or user-facing change requires updating the Hudi website. Please create a Jira ticket, attach the
    ticket number here and follow the instruction to make
    changes to the website.

Contributor's checklist

  • Read through contributor's guide
  • Change Logs and Impact were stated clearly
  • Adequate tests were added if applicable
  • CI passed

@xiarixiaoyao
Copy link
Contributor Author

@alexeykudinkin @XuQianJin-Stars
could you pls help me review this pr, thanks

@danny0405 danny0405 added area:schema Schema evolution and data types engine:spark Spark integration engine:flink Flink integration labels Feb 23, 2023
val avroNameAndSpace = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName)
val avroSchema = internalSchemaOpt.map { is =>
AvroInternalSchemaConverter.convert(is, "schema")
AvroInternalSchemaConverter.convert(is, avroNameAndSpace._2 + "." + avroNameAndSpace._1)
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we should pass avroSchema.getFullName to this convert
Otherwise, the schema converted by the converter will be incompatible with the real schema of the hodie table

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@xiarixiaoyao can you please share the stacktrace you've observed? Avro name/namespaces shouldn't matter in that case.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For the context: this name/namespace are actually generated from the table name so that qualified name is no better than the previous one (using just "schema").

We need to understand the real root-cause of the issue

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@alexeykudinkin thanks for your review.

  1. schema evolution has nothing to do with this scene,since schema evolution will call HoodieAvroUtils.rewriteRecordWithNewSchema to uinfy namespace. i change this line just want to ensure that the namespace of reading schema and writing schema are consistent.
  2. The namespace of the schema used by hudi when writing the log is from tableName, but the namespace of read schema is “schema"
  3. When the schema evolution is not enabled,For decimal types, different namespaces produce different names, avro is name sensitive. we should keep the read schema and write schema has the same namespace just as previous versions of hudi
    eg:
    ff decimal(38, 10)
    hudi log write schema will be : {"name":"ff","type":[{"type":"fixed","name":"fixed","namespace":"hoodie.h0.h0_record.ff","size":16,"logicalType":"decimal","precision":38,"scale":10}

spark read schema will be
ff type is : "name":"ff","type":[{"type":"fixed","name":"fixed","namespace":"Record.ff","size":16,"logicalType":"decimal","precision":38,"scale":10},"null"]}

the read schema and write schema is incompatible, we cannot use read schema to read log。previous versions of hudi does not have this problem

Caused by: org.apache.avro.AvroTypeException: Found hoodie.h0.h0_record.ff.fixed, expecting union
at org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:308)
at org.apache.avro.io.parsing.Parser.advance(Parser.java:86)
at org.apache.avro.io.ResolvingDecoder.readIndex(ResolvingDecoder.java:275)
at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:188)
at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:161)
at org.apache.avro.generic.GenericDatumReader.readField(GenericDatumReader.java:260)
at org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:248)
at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:180)
at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:161)
at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:154)
at org.apache.hudi.common.table.log.block.HoodieAvroDataBlock$RecordIterator.next(HoodieAvroDataBlock.java:201)
at org.apache.hudi.common.table.log.block.HoodieAvroDataBlock$RecordIterator.next(HoodieAvroDataBlock.java:149)

val (recordName, namespace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName)
val avroSchema = sparkAdapter.getAvroSchemaConverters.toAvroType(structSchema, nullable = false, recordName, namespace)
getAvroSchemaWithDefaults(avroSchema, structSchema)
}
Copy link
Contributor Author

@xiarixiaoyao xiarixiaoyao Feb 23, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@qidian99 keep default value for avro schema,
as we need do schema evolution. this modify also fix the bug #7915

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@xiarixiaoyao i still don't understand why we need to set defaults in the schema. Can you please elaborate on that one?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes,
schemaConverters.toAvroType will lost default value. see #2765
In the schema evolution scenario, the default value is very important, avroSchema cares about this。

eg: If we add a new column newCol: string to the table, the default value of newCol will be null
after schemaConverters.toAvroType , the default vaule of newCol will be lost
now if we use this schema to read old avro log(not contains column newCol), avro will complain that there is no default value, and throw exception.
#7915 The root cause of this pr is that we lost the default value in the conversion process

@xiarixiaoyao xiarixiaoyao changed the title [HUDI-5835] After performing the update operation, the hodie table cannot be read normally by spark [HUDI-5835] After performing the update operation, the hoodie table cannot be read normally by spark Feb 23, 2023
Copy link
Contributor

@alexeykudinkin alexeykudinkin left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@xiarixiaoyao raised a few questions, let's hold on landing until we clear these up.

val avroNameAndSpace = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName)
val avroSchema = internalSchemaOpt.map { is =>
AvroInternalSchemaConverter.convert(is, "schema")
AvroInternalSchemaConverter.convert(is, avroNameAndSpace._2 + "." + avroNameAndSpace._1)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For the context: this name/namespace are actually generated from the table name so that qualified name is no better than the previous one (using just "schema").

We need to understand the real root-cause of the issue

val (recordName, namespace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName)
val avroSchema = sparkAdapter.getAvroSchemaConverters.toAvroType(structSchema, nullable = false, recordName, namespace)
getAvroSchemaWithDefaults(avroSchema, structSchema)
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@xiarixiaoyao i still don't understand why we need to set defaults in the schema. Can you please elaborate on that one?

}
}

val avroNameAndSpace = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: we can move this inside the map and also make it val (name, namespace) = getAvroRecordNameAndNamespace(...)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

@alexeykudinkin alexeykudinkin added priority:blocker Production down; release blocker release-0.13.1 labels Feb 24, 2023
@xiarixiaoyao xiarixiaoyao force-pushed the logcc branch 3 times, most recently from 0b0ca82 to edc3bd9 Compare February 25, 2023 02:13
@xiarixiaoyao
Copy link
Contributor Author

@hudi-bot run azure

@xiarixiaoyao
Copy link
Contributor Author

@hudi-bot run azure

@xiarixiaoyao xiarixiaoyao force-pushed the logcc branch 3 times, most recently from 180059d to 56c75fe Compare February 27, 2023 10:08
@hudi-bot
Copy link
Collaborator

CI report:

Bot commands @hudi-bot supports the following commands:
  • @hudi-bot run azure re-run the last Azure build

@xiarixiaoyao xiarixiaoyao merged commit 31e94ab into apache:master Feb 28, 2023
danny0405 pushed a commit to danny0405/hudi that referenced this pull request Mar 23, 2023
…annot be read normally by spark (apache#8026)

(cherry picked from commit 31e94ab)
danny0405 pushed a commit to danny0405/hudi that referenced this pull request Mar 23, 2023
…annot be read normally by spark (apache#8026)

(cherry picked from commit 31e94ab)
nsivabalan pushed a commit to nsivabalan/hudi that referenced this pull request Mar 23, 2023
…annot be read normally by spark (apache#8026)

(cherry picked from commit 31e94ab)
nsivabalan pushed a commit to nsivabalan/hudi that referenced this pull request Mar 23, 2023
…annot be read normally by spark (apache#8026)

(cherry picked from commit 31e94ab)
fengjian428 pushed a commit to fengjian428/hudi that referenced this pull request Apr 5, 2023
h1ap pushed a commit to h1ap/hudi that referenced this pull request Apr 12, 2023
…annot be read normally by spark (apache#8026)

(cherry picked from commit 31e94ab)

# Conflicts:
#	hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala
stayrascal pushed a commit to stayrascal/hudi that referenced this pull request Apr 20, 2023
KnightChess pushed a commit to KnightChess/hudi that referenced this pull request Jan 2, 2024
…annot be read normally by spark (apache#8026)

(cherry picked from commit 31e94ab)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

area:schema Schema evolution and data types engine:flink Flink integration engine:spark Spark integration priority:blocker Production down; release blocker

Projects

Archived in project

Development

Successfully merging this pull request may close these issues.

5 participants