Use per-table merge lock in Manager instead of single lock#6378
Open
dlmarion wants to merge 4 commits into
Open
Use per-table merge lock in Manager instead of single lock#6378dlmarion wants to merge 4 commits into
dlmarion wants to merge 4 commits into
Conversation
This change introduces a per-table merge lock instead of a single lock to protect reading/writing to ZooKeeper. Closes apache#6374
ctubbsii
requested changes
May 18, 2026
| Collections.synchronizedMap(new HashMap<>()); | ||
| final Set<TServerInstance> serversToShutdown = Collections.synchronizedSet(new HashSet<>()); | ||
| final Migrations migrations = new Migrations(); | ||
| private final MergeLocks mergeLocks = new MergeLocks(); |
Member
There was a problem hiding this comment.
Suggested change
| private final MergeLocks mergeLocks = new MergeLocks(); | |
| private final LoadingCache<TableId,ReentrantLock> mergeLocks = Caffeine.newBuilder().weakValues() | |
| .scheduler(Scheduler.systemScheduler()).build(k -> new ReentrantLock()); |
My main concern with this is that the TableId in the keys will prevent cleanup of the TableId.cache weak values. The scheduler is supposed to help with cleanup of the keys whose values have been garbage collected, but it may also be a good idea to use String (tableId.canonical()) for the key.
Comment on lines
+176
to
+199
| private final class MergeLocks { | ||
|
|
||
| private final Object lock = new Object(); | ||
| private Map<TableId,ReentrantLock> lockStorage = new HashMap<TableId,ReentrantLock>(); | ||
|
|
||
| private ReentrantLock getLock(TableId tid) { | ||
| synchronized (lock) { | ||
| return lockStorage.computeIfAbsent(tid, k -> new ReentrantLock(true)); | ||
| } | ||
| } | ||
|
|
||
| private void cleanup() { | ||
| synchronized (lock) { | ||
| Set<TableId> removals = new HashSet<>(); | ||
| for (Entry<TableId,ReentrantLock> e : lockStorage.entrySet()) { | ||
| if (!getContext().tableNodeExists(e.getKey()) && !e.getValue().isLocked()) { | ||
| removals.add(e.getKey()); | ||
| } | ||
| } | ||
| removals.forEach(lockStorage::remove); | ||
| } | ||
| } | ||
| } | ||
|
|
Member
There was a problem hiding this comment.
Suggested change
| private final class MergeLocks { | |
| private final Object lock = new Object(); | |
| private Map<TableId,ReentrantLock> lockStorage = new HashMap<TableId,ReentrantLock>(); | |
| private ReentrantLock getLock(TableId tid) { | |
| synchronized (lock) { | |
| return lockStorage.computeIfAbsent(tid, k -> new ReentrantLock(true)); | |
| } | |
| } | |
| private void cleanup() { | |
| synchronized (lock) { | |
| Set<TableId> removals = new HashSet<>(); | |
| for (Entry<TableId,ReentrantLock> e : lockStorage.entrySet()) { | |
| if (!getContext().tableNodeExists(e.getKey()) && !e.getValue().isLocked()) { | |
| removals.add(e.getKey()); | |
| } | |
| } | |
| removals.forEach(lockStorage::remove); | |
| } | |
| } | |
| } |
|
|
||
| public void clearMergeState(TableId tableId) throws KeeperException, InterruptedException { | ||
| synchronized (mergeLock) { | ||
| final ReentrantLock l = mergeLocks.getLock(tableId); |
Member
There was a problem hiding this comment.
Suggested change
| final ReentrantLock l = mergeLocks.getLock(tableId); | |
| final ReentrantLock l = mergeLocks.get(tableId); |
|
|
||
| ThreadPools.watchCriticalScheduledTask( | ||
| context.getScheduledExecutor().scheduleWithFixedDelay(mergeLocks::cleanup, 3, 3, HOURS)); | ||
|
|
Member
There was a problem hiding this comment.
No need for this cleanup code with the weakValues cache's scheduler.
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.
This change introduces a per-table merge lock instead of a single lock to protect reading/writing to ZooKeeper.
Closes #6374