feat(flink): Wire Flink 2.1 nested Parquet readers into the Hudi read path (FLINK-35702)#18700
Merged
danny0405 merged 1 commit intoMay 8, 2026
Conversation
… path (FLINK-35702)
Codecov Report❌ Patch coverage is
Additional details and impacted files@@ Coverage Diff @@
## master #18700 +/- ##
============================================
+ Coverage 67.90% 68.14% +0.23%
- Complexity 28958 29078 +120
============================================
Files 2521 2522 +1
Lines 141039 141177 +138
Branches 17480 17514 +34
============================================
+ Hits 95777 96209 +432
+ Misses 37401 37060 -341
- Partials 7861 7908 +47
Flags with carried forward coverage won't be shown. Click here to find out more.
🚀 New features to boost your workflow:
|
Collaborator
danny0405
approved these changes
May 8, 2026
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Describe the issue this Pull Request addresses
Hudi-flink's ParquetColumnarRowSplitReader read path still routes nested types (ARRAY, MAP, MULTISET, ROW) through the legacy ArrayColumnReader / MapColumnReader / RowColumnReader chain that PR 3 left untouched, so the ClassCastException class of bugs from FLINK-35702 (nested-in-nested ARRAY / MAP, ROW subtrees, legacy 2-level list encoding) is still reachable in production. PR 3 vendored the two Dremel-style readers but parked them as dead code; this PR rewires the read path to actually use them while preserving Hudi's schema-evolution handling and the Hudi-specific "row-collapse-on-all-null-children" semantic that the legacy RowColumnReader implemented.
This is the 4th PR in the series. The legacy ArrayColumnReader / MapColumnReader / RowColumnReader and the ColumnarGroup* data classes are no longer reachable from the read path after this PR, but are left in the tree for this change. Their removal is deferred to PR 5 so this PR stays narrowly scoped to the wiring switch.
Summary and Changelog
Faithful port from Apache Flink 2.1:
ParquetSplitReaderUtil is rewritten in line with the FLINK-35702 ParquetSplitReaderUtil:
ParquetColumnarRowSplitReader: builds the ParquetField tree once per split via new ColumnIOFactory().getColumnIO(requestedSchema), then feeds each per-column ParquetField into the new createColumnReader overload from readNextRowGroup. The pre-existing field-name / type bookkeeping (clipParquetSchema, checkSchema, patchedVector) is unchanged, as is the missing-top-level-column patching that was already in place.
HeapMapColumnVector.getMap(int): switched from the legacy ColumnarGroupMapData(keys, values, rowId) to the standard ColumnarMapData(keys, values, offset, length) wired through the offsets / lengths populated by NestedColumnReader (PR 3 added that state and the new accessors; this PR consumes them). Field types and the Lombok-generated getKeys() / getValues() accessors are kept as WritableColumnVector to avoid an API break on the surface PR 3 exposed.
Deliberate divergences from Flink 2.1:
ParquetSplitReaderUtil#constructField (ROW branch): a logical child may be absent from the Parquet file (Hudi schema evolution); this is handled via a case-insensitive lookupColumnByNameOrNull that yields a null ParquetField for the absent child. Upstream Flink throws on missing children. Companion behavior: createWritableColumnVector ROW branch slots a null-filled vector for the same child via createVectorFromConstant, so the Dremel assembler in NestedColumnReader.readRow passes the slot through unchanged when its ParquetField is null.
NestedColumnReader#readRow (Hudi backward-compat divergence): after the Dremel-driven null assignment via setFieldNullFlag, run an extra pass that collapses a present row whose every child is null into a null row. Flink 2.1 would surface row(null, ..., null) as Row(null, ..., null); the legacy Hudi RowColumnReader collapsed it to NULL, and existing Hudi users' SQL (the ITTestHoodieDataSource#testParquetNullChildColumnsRowTypes integration test in particular) depends on this. Documented inline at the divergence site.
Tests added:
None new in this PR. Hudi's existing integration tests in ITTestHoodieDataSource cover the rewired read path end-to-end at the SQL level (across insert / upsert / bulk_insert):
These are the same tests that exercised the legacy readers, so any regression in nesting / schema-evolution / null-handling between the legacy and the vendored Dremel path surfaces here. The upstream FLINK-35702 PR added a single unit test (testNestedRead in ParquetColumnarRowSplitReaderTest) which ports awkwardly into Hudi (Hudi has no upstream test class to extend, and the upstream test pulls a flink-shaded-guava31 transitive that Hudi's hudi-flink1.18.x test classpath doesn't ship); the IT coverage above subsumes its scope.
Part of #18491.
Impact
ARRAY / MAP / MULTISET / ROW reads on hudi-flink1.18.x now go through the vendored Dremel-style NestedColumnReader instead of the legacy ArrayColumnReader / MapColumnReader / RowColumnReader. The previously known ClassCastException surfaces from FLINK-35702 (nested-in-nested ARRAY, MAP-inside-ARRAY, ROW subtrees, legacy 2-level list encoding) are fixed.
Pre-existing Hudi-specific behaviors are preserved by the divergences listed above:
No public API is removed; the deprecated 5-arg createColumnReader entry point is preserved for primitive-only callers.
Risk Level
Medium-low.
Documentation Update
N/A
Contributor's checklist
Rollout plan
The backport into flink 1.18 will be split into 5 PRs for easier reviews. After they are all merged, we will backport to higher flink versions (1.19.x, 1.20.x, 2.0.x and 2.1.x) as well, and it will be 1 PR per version because it will be easy to diff between hudi-flink1.18.x and higher versions.