Skip to content

[HUDI-9247] Re-evaluate reuse of TimeGenerator instance#13077

Merged
nsivabalan merged 4 commits intoapache:masterfrom
lokeshj1703:HUDI-9247
Apr 4, 2025
Merged

[HUDI-9247] Re-evaluate reuse of TimeGenerator instance#13077
nsivabalan merged 4 commits intoapache:masterfrom
lokeshj1703:HUDI-9247

Conversation

@lokeshj1703
Copy link
Collaborator

Change Logs

TimeGenerator currently reuses the lock provider while generating the timestamp. We need to account for thread safety of lock provider before reusing the lock provider by different threads. The PR currently fixes the logic to ensure lock provider is reused only if it is thread safe.

Only InProcessLockProvider is thread safe as of now. The other lock providers are not thread safe.

  • InProcessLockProvider uses ReentrantReadWriteLock which inherently is thread safe.
  • DynamoDBBasedLockProvider uses AmazonDynamoDBLockClient which is thread safe but it stores the lock item as an instance variable which can cause issues with multiple threads.
  • Similarly ZookeeperBasedLockProvider uses CuratorFramework which is thread safe but it stores InterProcessMutex as instance variable. The InterProcessMutex is then released during unlock.
  • HiveMetastoreBasedLockProvider uses IMetaStoreClient which inherently is not thread safe.
  • FileSystemBasedLockProvider is also not thread safe since it stores LockInfo and currentOwnerLockInfo as instance variables.

Created https://issues.apache.org/jira/browse/HUDI-9254 for tracking the issue

Impact

NA

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

low

Documentation Update

NA

Contributor's checklist

  • Read through contributor's guide
  • Change Logs and Impact were stated clearly
  • Adequate tests were added if applicable
  • CI passed

@lokeshj1703
Copy link
Collaborator Author

I have created https://issues.apache.org/jira/browse/HUDI-9254 for ensuring the lock providers which support thread safety are made thread safe.

@github-actions github-actions bot added the size:S PR with lines of changes in (10, 100] label Apr 3, 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.

Could you please explain more in the PR description what scenario actually led to the necessity of this fix? IIUC, this is just a band-aid solution. We are giving up reusability of lock provider which could have adverse implications (please see the second point in my comment below).

String lockProviderClass = lockConfiguration.getConfig().getString("hoodie.write.lock.provider");
LOG.info("LockProvider for TimeGenerator: {}", lockProviderClass);
if (!LockProvider.isThreadSafe(lockProviderClass)) {
return createLockProvider(lockProviderClass);
Copy link
Member

Choose a reason for hiding this comment

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

  1. Please add an inline comment to explain the rationale behind returning a new instance for non-thread safe providers.
  2. Creating a new instance for non-thread safe providers on every call might have performance and heap usage implications. Given that time generation has to happen on every start/end of operation, could this lead to poor driver memory utilization especially if multiple streams are multiplexed on the same driver?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Yes, it can cause issues with performance and heap usage. I think a better fix would be make it thread safe instead. We will not be able to make HiveMetastoreBasedLockProvider as thread safe though. So we would need a fix for HiveMetastoreBasedLockProvider atleast.

Copy link
Contributor

@danny0405 danny0405 Apr 3, 2025

Choose a reason for hiding this comment

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

Can we just make TimeGenerator#generateTime synchronized to fix the issue?

And even if we store the lock as member variable of the lock provider, the try-lock would trigger a lock acquisition exception then a retry until the lock is fetched successfully? So it is still thread-safe without any fix?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Yeah. That should also work. Only issue could be contention when multiple streams are multiplexed on same driver. But it should be better than having a separate lock provider I think.

Copy link
Collaborator Author

@lokeshj1703 lokeshj1703 Apr 3, 2025

Choose a reason for hiding this comment

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

I have pushed a change to make time generation API in SkewAdjustingTimeGenerator synchronized. Since the TimeGenerator is cached for every table base path, the lock contention due to time generation should be minimal for same table.
There is another issue around closing of lock provider. Currently lock provider is closed during unlock and if we are sharing the lock provider amongst multiple threads, we would need to fix the close semantics for lock provider. Will add it in HUDI-9254.

Copy link
Contributor

Choose a reason for hiding this comment

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

we would need to fix the close semantics for lock provider

+1.

@github-actions github-actions bot added size:XS PR with lines of changes in <= 10 and removed size:S PR with lines of changes in (10, 100] labels Apr 3, 2025
Copy link
Contributor

@nsivabalan nsivabalan left a comment

Choose a reason for hiding this comment

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

So, w/ this, even if the TimeGenerator is re-used across diff threads w/n same JVM, we will ensure only one thread can generate instant time at a given point in time. Others will have to wait until the other which could be waiting w/n LockProvider's tryLock method.

Copy link
Contributor

@nsivabalan nsivabalan left a comment

Choose a reason for hiding this comment

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

Curious as to why adding it only to concrete impl. Or do we need to have a wrapper for this which will make this within a synchronized call irrespective of the actual impl?

@nsivabalan
Copy link
Contributor

nsivabalan commented Apr 3, 2025

We might need something like below

public interface TimeGenerator {

  /**
   * Generates a globally monotonically increasing timestamp. The implementation must ensure that
   * a new generated timestamp T is guaranteed to be greater than
   * any timestamp generated by the preceding calls.
   *
   * @param skipLocking If this is triggered by another parent transaction, locking can be skipped.
   * @return Current TrueTime as milliseconds.
   */
  default long generateTime(boolean skipLocking) {
    synchronized (this) {
      return generateTimeInternal(skipLocking);
    }
  }

  long generateTimeInternal(boolean skipLocking);

  /**
   * Passes an auto generated timestamp to the given function {@code func}. The implementations
   * need to ensure timestamp generation and executing func are atomic.
   *
   * @param skipLocking If this is triggered by another parent transaction, locking can be skipped.
   * @param func   A consumer that takes a monotonically increasing timestamp.
   */
  default void consumeTime(boolean skipLocking, Consumer<Long> func) {
    synchronized (this) {
      consumeTimeInternal(skipLocking, func);
    }
  }

  void consumeTimeInternal(boolean skipLocking, Consumer<Long> func);

essentially, all callers will be calling generateTime and consumeTime. and all concrete impl of TimeGenerator will have to impl generateTimeInternal and consumeTimeInternal.

so that we are future proof. any new impl of TimeGenerator does not need to ensure they "synchronize" the impl methods.

@nsivabalan
Copy link
Contributor

but looking at your PR desc, if only InProcessLockProvider is eligible for re-use, I am wondering, why can't we disallow re-use only for any lock provider.

anyways incase of other lock providers, looks like re-use is not possible.
Incase of InProcess lock provider, for a given table, we might have max of 4 to 5 concurrent threads (ingestion writer , and 3 table services) which might be looking to generate instant time. And InProcessLockProvider internally ensures all of them will end up using the same instance of ReentrantReadWriteLock for a given table. So, we are not gaining much by allowing re-use for InProcessLockProvider and there is not a lot of heavy weight resources we are starting up.

Its going to be an issue w/ other lock provider where we might have client objects establishing connection to external endpoints, but looks like all the lock providers need to be worked upon to make it threadsafe.

Based on this, I feel we can disallow re-use only for now and think through methodically how to go about this and rush through 1.0.2

@lokeshj1703
Copy link
Collaborator Author

I have disabled reuse for TimeGenerator by default.

@github-actions github-actions bot added size:S PR with lines of changes in (10, 100] and removed size:XS PR with lines of changes in <= 10 labels Apr 4, 2025
@hudi-bot
Copy link
Collaborator

hudi-bot commented Apr 4, 2025

CI report:

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

@nsivabalan nsivabalan merged commit 08d534b into apache:master Apr 4, 2025
60 checks passed
@danny0405
Copy link
Contributor

Why merge it without reaching consensus? Do we test the stability without the resuse.

voonhous pushed a commit to voonhous/hudi that referenced this pull request Apr 8, 2025
voonhous pushed a commit to voonhous/hudi that referenced this pull request Apr 9, 2025
voonhous pushed a commit to voonhous/hudi that referenced this pull request Apr 11, 2025
voonhous pushed a commit to voonhous/hudi that referenced this pull request Apr 15, 2025
voonhous pushed a commit to voonhous/hudi that referenced this pull request Apr 16, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

release-1.0.2 size:S PR with lines of changes in (10, 100]

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants