Skip to content

feat: add variant type adapter for Flink#18702

Merged
voonhous merged 2 commits into
apache:masterfrom
danny0405:flink-variant-converter
May 11, 2026
Merged

feat: add variant type adapter for Flink#18702
voonhous merged 2 commits into
apache:masterfrom
danny0405:flink-variant-converter

Conversation

@danny0405
Copy link
Copy Markdown
Contributor

Describe the issue this Pull Request addresses

add a new utils class DataTypeAdapter that can work as an adapter for variant type related APIs.

Summary and Changelog

  • add a new class DataTypeAdapter;
  • support type conversions for avro <-> row data converters;
  • support type conversion between parquet and flink data type;
  • impl the getVariant on Flink row data

Impact

none

Risk Level

low

Documentation Update

might need to update the site for variant data type support for Flink

Contributor's checklist

  • Read through contributor's guide
  • Enough context is provided in the sections above
  • Adequate tests were added if applicable

Copy link
Copy Markdown
Contributor

@hudi-agent hudi-agent left a comment

Choose a reason for hiding this comment

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

🤖 This review was generated by an AI agent and may contain mistakes. Please verify any suggestions before applying.

Thanks for the contribution! This PR introduces a DataTypeAdapter indirection for Flink's Variant type, wires it through the Avro/RowData/Parquet/Schema converters, and provides a real implementation for Flink 2.1.x with stub-throwing variants for older Flink versions. The shredded-variant rejection path looks consistent across HoodieSchemaConverter and ParquetSchemaConverter, and the Avro field order assumption (metadata at index 0, value at index 1) matches HoodieSchema.Variant's contract. No correctness issues found. A few style/readability suggestions in the inline comments. Please take a look, and this should be ready for a Hudi committer or PMC member to take it from here. A couple of spots use raw positional indices or string literals where the named HoodieSchema.Variant field constants (already used in ParquetSchemaConverter in this same PR) would make the intent clearer.

cc @yihua

* GenericRecord carrying metadata/value binary fields and produces a Flink
* {@code BinaryVariant}.
*/
private static AvroToRowDataConverter createVariantConverter() {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

🤖 nit: record.get(0) and record.get(1) are opaque — could you cast to GenericRecord and use HoodieSchema.Variant.VARIANT_METADATA_FIELD / VARIANT_VALUE_FIELD for named access? The same constants are already used in ParquetSchemaConverter in this PR.

- AI-generated; verify before applying. React 👍/👎 to flag quality.

return new RowDataToAvroConverter() {
private static final long serialVersionUID = 1L;

@Override
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

🤖 nit: the string literals "metadata" and "value" here could use HoodieSchema.Variant.VARIANT_METADATA_FIELD / VARIANT_VALUE_FIELD to stay consistent with how ParquetSchemaConverter references the same fields in this PR.

- AI-generated; verify before applying. React 👍/👎 to flag quality.

@github-actions github-actions Bot added the size:L PR with lines of changes in (300, 1000] label May 8, 2026
try {
return DataTypeAdapter.createVariant(value, metadata);
} catch (Exception e) {
throw new RuntimeException("Failed to create Flink BinaryVariant via reflection.", e);
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

This is a direct constructor call, not reflection.

* Uses class-name matching so this compiles against parquet-java versions that predate the
* {@code VariantLogicalTypeAnnotation} class (< 1.15.2).
*/
private static boolean hasVariantAnnotation(LogicalTypeAnnotation logicalType) {
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

Since the writer does not attach the variant logical annotation (todo in line 227), a Flink-written variant column will read back as a plain row and HoodieSchemaConverter.convertToSchema() will lose the VARIANT type?

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

private def isVariantGroup(group: GroupType): Boolean = {
group.containsField("value") &&
group.containsField("metadata") &&
group.getType("value").isPrimitive &&
group.getType("metadata").isPrimitive &&
group.getType("value").asPrimitiveType().getPrimitiveTypeName == PrimitiveType.PrimitiveTypeName.BINARY &&
group.getType("metadata").asPrimitiveType().getPrimitiveTypeName == PrimitiveType.PrimitiveTypeName.BINARY
}

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Yeah based on my understanding, in order to make sure a Flink-written variant field (or any variant field written without the parquet variant annotation — which is all current Hudi writes since the code has a TODO to add it like you pointed out) is re-read as variant in Flink, we would need a subsequent PR like #18539 that add HoodieSchema for detection or we need to actually attach the annotation at write time.
Maybe we can add a TODO comment here, just so that readers know this state of Flink variant integration (in case for some reason we don't fully "fix" this by 1.3)?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

yeah, add a TODO to add variant on writer side first.

DataTypes.FIELD("metadata", DataTypes.BYTES().notNull()),
DataTypes.FIELD("value", DataTypes.BYTES().notNull())
).notNull();
if (schema instanceof HoodieSchema.Variant
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

can be removed, since line 569 ensures the type is VARIANT.

@codecov-commenter
Copy link
Copy Markdown

Codecov Report

❌ Patch coverage is 0% with 37 lines in your changes missing coverage. Please review.
✅ Project coverage is 68.13%. Comparing base (47bf4e4) to head (b8a86a1).

Files with missing lines Patch % Lines
...io/storage/row/parquet/ParquetSchemaConverter.java 0.00% 14 Missing and 2 partials ⚠️
.../org/apache/hudi/util/RowDataToAvroConverters.java 0.00% 8 Missing ⚠️
...va/org/apache/hudi/util/HoodieSchemaConverter.java 0.00% 6 Missing ⚠️
.../org/apache/hudi/util/AvroToRowDataConverters.java 0.00% 5 Missing ⚠️
...pache/hudi/client/model/AbstractHoodieRowData.java 0.00% 1 Missing ⚠️
...org/apache/hudi/client/model/BootstrapRowData.java 0.00% 1 Missing ⚠️
Additional details and impacted files
@@             Coverage Diff              @@
##             master   #18702      +/-   ##
============================================
- Coverage     68.14%   68.13%   -0.02%     
+ Complexity    29077    29074       -3     
============================================
  Files          2522     2522              
  Lines        141177   141207      +30     
  Branches      17514    17522       +8     
============================================
- Hits          96208    96205       -3     
- Misses        37061    37092      +31     
- Partials       7908     7910       +2     
Flag Coverage Δ
common-and-other-modules 44.40% <0.00%> (-0.02%) ⬇️
hadoop-mr-java-client 44.99% <ø> (-0.02%) ⬇️
spark-client-hadoop-common 48.35% <ø> (-0.01%) ⬇️
spark-java-tests 49.02% <ø> (+0.02%) ⬆️
spark-scala-tests 44.90% <ø> (+<0.01%) ⬆️
utilities 37.63% <ø> (-0.01%) ⬇️

Flags with carried forward coverage won't be shown. Click here to find out more.

Files with missing lines Coverage Δ
...pache/hudi/client/model/AbstractHoodieRowData.java 77.50% <0.00%> (ø)
...org/apache/hudi/client/model/BootstrapRowData.java 50.00% <0.00%> (ø)
.../org/apache/hudi/util/AvroToRowDataConverters.java 49.24% <0.00%> (-1.94%) ⬇️
...va/org/apache/hudi/util/HoodieSchemaConverter.java 66.80% <0.00%> (-2.62%) ⬇️
.../org/apache/hudi/util/RowDataToAvroConverters.java 70.67% <0.00%> (-4.53%) ⬇️
...io/storage/row/parquet/ParquetSchemaConverter.java 71.42% <0.00%> (-7.33%) ⬇️

... and 7 files with indirect coverage changes

🚀 New features to boost your workflow:
  • ❄️ Test Analytics: Detect flaky tests, report on failures, and find test suite problems.

@hudi-bot
Copy link
Copy Markdown
Collaborator

hudi-bot commented May 9, 2026

CI report:

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

Copy link
Copy Markdown
Collaborator

@cshuo cshuo left a comment

Choose a reason for hiding this comment

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

+1

@voonhous voonhous merged commit 5e72c96 into apache:master May 11, 2026
63 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

size:L PR with lines of changes in (300, 1000]

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants