perf(spark): Resolve drop-partition-columns projection once per write…#18972
Merged
danny0405 merged 1 commit intoJun 13, 2026
Merged
Conversation
…r instead of per row BulkInsertDataInternalWriterHelper#write redid constant work for every row when hoodie.datasource.write.drop.partition.columns is enabled: resolving the config flag, instantiating a key generator via constructor reflection through getPartitionPathCols, recomputing the partition-column ordinals into a fresh HashSet, and round-tripping the whole row through toSeq/fromSeq (boxing every column). The flag is now resolved once in the constructor, and the retained (non-partition) field ordinals and types are computed once on the first write(). The lazy initialization keeps the partition-column resolution unreachable for the bucket-index subclasses, which override write() and never drop columns, and for tasks that write no rows, matching the previous reachability exactly. write() copies the retained fields into a fresh GenericInternalRow, which is value-identical to the previous toSeq/filter/fromSeq output.
Codecov Report✅ All modified and coverable lines are covered by tests. Additional details and impacted files@@ Coverage Diff @@
## master #18972 +/- ##
============================================
+ Coverage 68.25% 68.26% +0.01%
- Complexity 29509 29531 +22
============================================
Files 2542 2542
Lines 142632 142697 +65
Branches 17789 17800 +11
============================================
+ Hits 97354 97415 +61
- Misses 37271 37275 +4
Partials 8007 8007
Flags with carried forward coverage won't be shown. Click here to find out more.
🚀 New features to boost your workflow:
|
Collaborator
wombatu-kun
approved these changes
Jun 13, 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.
…r instead of per row
Describe the issue this Pull Request addresses
Closes #18969
BulkInsertDataInternalWriterHelper#write(InternalRow)redoes constant work for every row whenhoodie.datasource.write.drop.partition.columns=true: it resolves the config flag, instantiates a key generator via constructor reflection throughHoodieDatasetBulkInsertHelper.getPartitionPathCols(ReflectionUtilscaches only the Class object, not instances), recomputes the partition-column ordinals into a fresh HashSet, and round-trips the whole row throughtoSeq/fromSeq, boxing every column. None of this depends on the row. The path runs per record in row-writer bulk insert and clustering rewrites whenever drop-partition-columns is enabled.Summary and Changelog
The projection is now resolved once per writer instead of once per row.
write()call.BucketBulkInsertDataInternalWriterHelper,ConsistentBucketBulkInsertDataInternalWriterHelper) overridewrite()and never drop partition columns, so the partition-column resolution was previously unreachable for them and must stay that way -- an eager precompute would turn configs that complete today (for example bucket index with drop enabled and aCustomKeyGenerator, whose partition fields keep thefield:typesuffix thatstructType.fieldIndexrejects) into deterministic construction failures. Lazy init also keeps zero-row tasks free of the resolution, matching the previous reachability exactly.write()copies the retained fields into a freshGenericInternalRowviarow.get(ordinal, type), which is value-identical to the previoustoSeq/filter/fromSeqround trip (toSeqitself reads each field withrow.get(i, dataType)), and a fresh row per record keeps the same aliasing behavior.Covered by the existing functional tests that exercise this branch:
TestGetPartitionValuesFromPath(bulk_insert with drop enabled) andTestHoodieSparkSqlWriter.Impact
Performance: removes per-row key-generator constructor reflection, config resolution, ordinal recomputation, and the full-row boxing round trip from row-writer bulk insert when drop-partition-columns is enabled. Output rows are value-identical. No public API change; the default flag-false path is unchanged.
Risk Level
Low. The transformation is a local row projection with no timeline or concurrency interaction; lazy initialization preserves the exact previous reachability for subclasses and zero-row tasks.
Documentation Update
None.
Contributor's checklist