Skip to content

Conversation

@PavithranRick
Copy link
Collaborator

@PavithranRick PavithranRick commented Dec 16, 2025

Describe the issue this Pull Request addresses

This PR unifies Partial Update Modes defined via table properties with partial update encoding used by Spark SQL MERGE INTO, and cleans up inconsistencies in how partial updates are handled across the write path.

Specifically, prior to this change:

  • Partial update handling logic was fragmented.
  • Merge Into partial update encoding was only supported for Spark records.
  • Table property–driven partial update modes and Merge Into encoding were not unified.

This work was originally based on PR #13540 from Lin, with additional fixes and refinements applied.

Summary and Changelog

This PR introduces a revised and unified design for partial update handling across all record formats and write paths.

Key changes:

  • Unifying Partial Update Modes from table properties and Merge Into partial update encoding.
  • Fixed all partial update handling within BufferedRecordMergerFactory.
  • Extended Partial update encoding (Merge Into support) across all record formats (previously only Spark record was supported).

Revised PartialUpdateMode design:

Possible values for table property hoodie.table.partial.update.mode:

  • IGNORE_DEFAULTS
  • FILL_UNAVAILABLE
  • KEEP_VALUES

There is no default value for PartialUpdateMode. For tables without partial update requirements, this property may be absent.

Note on KEEP_VALUES:

  • This value may not be explicitly present in table properties.
  • It will take effect when partial columns are encoded via Spark SQL MERGE INTO.
  • When invoking BufferedRecordMerger for merging, this mode is expected to be set accordingly.

BufferedRecordMergerFactory changes:

BufferedRecordMergerFactory.create() now accepts:

  • enablePartialEncoding (boolean): indicates whether partial (vs full) record merging is used.
  • Option<PartialUpdateMode>: defines the merge semantics when partial encoding is enabled.

If enablePartialEncoding is true, the provided PartialUpdateMode will be honored.

Interaction between table property and Merge Into encoding:

Merge Mode Table Property: hoodie.table.partial.update.mode Writer BufferedRecordMerger: enablePartialEncoding BufferedRecordMerger: Option
event time not set spark-ds false Option.empty()
event time not set Merge Into true KEEP_VALUES
event time IGNORE_DEFAULTS spark-ds true IGNORE_DEFAULTS
event time IGNORE_DEFAULTS Merge Into true IGNORE_DEFAULTS

Pending:

  • Unless we land get the fix landed for https://issues.apache.org/jira/browse/HUDI-9638 these new PartialUpdateModes are not taking effect for end to end functional tests.
  • Also, we need to add more tests directly against BufferedRecordMerger class.

Impact

Cleaning up PartialUpdateMode feature and unifying MergeInto partial update encoding along with other PartialUpdateModes from table config.

Risk Level

medium

Documentation Update

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

@github-actions github-actions bot added the size:XL PR with lines of changes > 1000 label Dec 16, 2025
@PavithranRick PavithranRick marked this pull request as ready for review December 18, 2025 20:37
Copy link
Contributor

@yihua yihua left a comment

Choose a reason for hiding this comment

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

Have we done any performance benchmarks as I remember there's concern around the performance impact?

* Class to assist with merging two versions of the record that may contain partial updates using
* {@link org.apache.hudi.common.table.PartialUpdateMode#KEEP_VALUES} mode.
*/
public class KeepValuesPartialMergingUtils<T> {
Copy link
Contributor

Choose a reason for hiding this comment

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

Nitpick: Utils typically don't have any state. Let's name this something like PartialMergerWithKeepValues

Comment on lines +42 to +45
private static final Map<HoodieSchema, Map<String, Integer>>
FIELD_NAME_TO_ID_MAPPING_CACHE = new ConcurrentHashMap<>();
private static final Map<Pair<Pair<HoodieSchema, HoodieSchema>, HoodieSchema>, HoodieSchema>
MERGED_SCHEMA_CACHE = new ConcurrentHashMap<>();
Copy link
Contributor

Choose a reason for hiding this comment

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

This cache can just grow over the life of the application. I think we can create a single instance for each BufferedRecordMerger and then make this an instance variable

Object[] fieldVals = new Object[fields.size()];
int idx = 0;
List<HoodieSchemaField> mergedSchemaFields = mergedSchema.getFields();
for (HoodieSchemaField mergedSchemaField : mergedSchemaFields) {
Copy link
Contributor

Choose a reason for hiding this comment

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

What is the expected behavior for nested fields?

@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

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

size:XL PR with lines of changes > 1000

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants