Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-8939] Fixing concurrency handling during upgrade #12737

Merged
merged 3 commits into from
Jan 30, 2025

Conversation

nsivabalan
Copy link
Contributor

@nsivabalan nsivabalan commented Jan 30, 2025

Change Logs

  • Fixing concurrency handling during upgrade and downgrade.

Problem scenario:

L1 df.write.format(hudi).save(path) -> say user configured zookeeper based lock provider. 
   .
L10   . writeClient.upsert 
L11         doInitTable
L12             lock using zookeeper based lock provider
L13                   upgrade
L14                            
L15                                 rollback failed writes
L16                                 full table compaction 
L17                                 both of the above operations will not try to re-acquire the lock again using user configured LP(i.e zookeeper based lock provider). this contends w/ the same lock taken at L12 and times out eventually. 

L18               unlock
L19        continue w/ upsert. 
L20       .
.

Root cause is the re-entrant locking.

We are making 3 fixes in this patch. w/ all of the fixes, the control flow is as follows:

L1 df.write.format(hudi).save(path) -> say user configured zookeeper based lock provider. 
   .
L10   . writeClient.upsert 
L11         doInitTable
L12             lock using zookeeper based lock provider
L13                   upgrade
L14                            override lock provider to NoopLockProvider, disable auto adjust lock configs, disable reuse time generator
L15                                 rollback failed writes
L16                                 full table compaction 
L17                                 both of the above operations will not try to re-acquire the lock again using user configured LP(i.e zookeeper based lock provider). here we use NoopLockProvider. 
L18               unlock
L19        continue w/ upsert. 
L20       .
.

Dissecting each fix:

  1. removing lock provider from upgrade code path.

after this fix, the control flow is as follows:

L1 df.write.format(hudi).save(path) -> say user configured zookeeper based lock provider. 
   .
L10   . writeClient.upsert 
L11         doInitTable
L12             lock using zookeeper based lock provider
L13                   upgrade
L14                            remove lock provider configs. // fix.
L15                                 rollback failed writes
L16                                 full table compaction 
L17                                 // but we did notice again re-entrancy was happening 
L18               unlock
L19        continue w/ upsert. 
L20       .
.
  1. We were automatically overriding the LockProvider to InProcessLock provider if its a single writer and all inline table services (at L15). So, essentially if we configure InProcessLockProvider from end user standpoint, we were hitting re-entrant locks. Fix: Made this logic to be guarded by the auto adjust lock config (hoodie.auto.adjust.lock.configs). So, no auto adjustment of lock configs will take place for straight forward out of the box use-case.

Again, we were still hitting the exception.

  1. TransactionManager is the entity used for regular locking mechanism (lock while scheduling compaction, locking while writing to MDT etc) in hudi. But the new instant time generation takes a diff route and has its down lock provider. So, both of them have different ways to deduce the default.

TransactionManager was explicitly setting the lock provider to InProcessLockProvider if there is no LockProvider configured. If user configures explicitly, txnManager re-uses the same.

Introducing NoopLockProvider which just allows anyone to acquire the lock (synonymous to single writer). So, for UpgradeHandler code blocks, we override the lock provider to use NoopLockProvider.

after this fix, the control flow is as follows:

L1 df.write.format(hudi).save(path) -> say user configured zookeeper based lock provider. 
   .
L10   . writeClient.upsert 
L11         doInitTable
L12             lock using zookeeper based lock provider
L13                   upgrade
L14                            override lock provider to NoopLockProvider
L15                                 rollback failed writes
L16                                 full table compaction 
L17                                 // we were again hitting the issue still.
L18               unlock
L19        continue w/ upsert. 
L20       .
.

Even w/ above fix, we were still hitting re-entrant locks.

  1. We use a cache for TimeGenerator instances, one per table path. Again, in the above flow of events, even if we override the LP to NoopLockProvider, new instant time generation was using the same lock provider as user configured one. Reason was, we re-use the same TimeGenerator which was generated before the upgrade call. So, introduced an internal config named "_hoodie.time.generator.reuse.enable". By default we enable it. In the upgrade flows, we override it to false. And so, we create new instance of TimeGenerator. This means that, the lock provider used by the TimeGenerator will use NoopLockProvider as configured.

So, w/ all of above 3 fixes, our solution is as follows

L1 df.write.format(hudi).save(path) -> say user configured zookeeper based lock provider. 
   .
L10   . writeClient.upsert 
L11         doInitTable
L12             lock using zookeeper based lock provider
L13                   upgrade
L14                            override lock provider to NoopLockProvider, disable auto adjust lock configs, disable reuse time generator
L15                                 rollback failed writes
L16                                 full table compaction 
L17                                 both of the above operations will not try to re-acquire the lock again using user configured LP(i.e zookeeper based lock provider). here we use NoopLockProvider. 
L18               unlock
L19        continue w/ upsert. 
L20       .
.

Impact

Seamless upgrade irrespective of lock provider used.

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

medium

Documentation Update

Describe any necessary documentation update if there is any new feature, config, or user-facing change. If not, put "none".

  • 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

@nsivabalan nsivabalan force-pushed the concurrencyFixUpgrade7To8 branch from 032cd44 to 0093e06 Compare January 30, 2025 01:40
@nsivabalan nsivabalan force-pushed the concurrencyFixUpgrade7To8 branch from 0093e06 to c0344ee Compare January 30, 2025 01:41
@github-actions github-actions bot added the size:L PR with lines of changes in (300, 1000] label Jan 30, 2025
Copy link
Member

@codope codope left a comment

Choose a reason for hiding this comment

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

Yet to go review the code. But, wondering if we need more complexty of new lock provider and new configs. I have a few high level questions:

  1. Why do we necessarily need a new NoopLockProvider if we are removing the lock configs during upgrade? Shouldn't the txn manager for the upgrade write client understand based on its write config that lock is not required? Conceptually, just removing lock configs and disabling auto adjustment should be enough.
  2. Why do we need a new config to decide whether or not to reuse time generator w/ or w/o lock? TimeGenerator API takes a flag to indicate whether locking is required or not. So, if the existing configs are being propagated properly and all callers of TimeGenerator API are passing the flag based on the config, then I don't think there is a need for another config.
  3. I think the goal was to identify the malicious caller, as we discussed, but we still don't know that right?

Copy link
Member

@codope codope left a comment

Choose a reason for hiding this comment

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

Discussed offline, and as such, the patch is good to unblock 1.0.1. But, there are still some open questions:

  1. Does TimeGenerator API always need a lock provider, even when there is no real lock requirement (say single writer, all inline table service)?
  2. Is upgrade (esp rollbackFailedWritesAndCompact) the only path where this issue happens? For COW tables wit explicit InProcessLockProvider configured, I have noticed that testPartitionFieldsWithUpgrade fails due to NPE after upgrade. This patch has somehow fixed it, but I don't have good understanding of what exactly was causing that NPE. For ref, draft patch based off of current master that repro the COW issue - [DO NOT MERGE] Investigate COW failure for null lock provider #12739

Let's revisit the above soon.

@hudi-bot
Copy link

CI report:

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

@codope codope merged commit b44e19c into apache:master Jan 30, 2025
43 checks passed
@yihua yihua changed the title [HUDI-8930] Fixing concurrency handling during upgrade [HUDI-8939] Fixing concurrency handling during upgrade Jan 31, 2025
linliu-code pushed a commit to linliu-code/hudi that referenced this pull request Jan 31, 2025
* minor fixes to upgrade path

* Fixes for concurrency handling during upgrade

* fix build failure

---------

Co-authored-by: Sagar Sumit <[email protected]>
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.

3 participants