[HUDI-9247] Re-evaluate reuse of TimeGenerator instance#13077
[HUDI-9247] Re-evaluate reuse of TimeGenerator instance#13077nsivabalan merged 4 commits intoapache:masterfrom
Conversation
|
I have created https://issues.apache.org/jira/browse/HUDI-9254 for ensuring the lock providers which support thread safety are made thread safe. |
codope
left a comment
There was a problem hiding this comment.
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); |
There was a problem hiding this comment.
- Please add an inline comment to explain the rationale behind returning a new instance for non-thread safe providers.
- 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?
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
we would need to fix the close semantics for lock provider
+1.
nsivabalan
left a comment
There was a problem hiding this comment.
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.
nsivabalan
left a comment
There was a problem hiding this comment.
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?
|
We might need something like below 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. |
|
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. 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 |
|
I have disabled reuse for TimeGenerator by default. |
|
Why merge it without reaching consensus? Do we test the stability without the resuse. |
…ache#13077) (cherry picked from commit 08d534b)
…ache#13077) (cherry picked from commit 08d534b)
…ache#13077) (cherry picked from commit 08d534b)
…ache#13077) (cherry picked from commit 08d534b)
…ache#13077) (cherry picked from commit 08d534b)
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.
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