-
Notifications
You must be signed in to change notification settings - Fork 143
IGNITE-28304 Improve contention on waiters in lock manager #7970
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
base: main
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ | |
| import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.Arrays; | ||
| import java.util.Collection; | ||
| import java.util.Comparator; | ||
| import java.util.EnumMap; | ||
|
|
@@ -78,6 +79,8 @@ | |
| * <p>Additionally limits the lock map size. | ||
| */ | ||
| public class HeapLockManager extends AbstractEventProducer<LockEvent, LockEventParameters> implements LockManager { | ||
| private static final LockMode[] LOCK_MODES = LockMode.values(); | ||
|
|
||
| /** Table size. */ | ||
| public static final int DEFAULT_SLOTS = 1_048_576; | ||
|
|
||
|
|
@@ -761,6 +764,9 @@ public class LockState implements Releasable { | |
| /** Waiters. */ | ||
| private final TreeMap<UUID, WaiterImpl> waiters; | ||
|
|
||
| /** Number of waiters currently holding each lock mode. Indexed by {@link LockMode#ordinal()}. */ | ||
| private final int[] lockModeHeldCount = new int[LOCK_MODES.length]; | ||
|
|
||
| /** Lock key. */ | ||
| private volatile LockKey key; | ||
|
|
||
|
|
@@ -848,10 +854,14 @@ IgniteBiTuple<CompletableFuture<Void>, LockMode> tryAcquire(UUID txId, LockMode | |
| // Reenter | ||
| if (prev != null) { | ||
| if (prev.locked() && prev.lockMode().allowReenter(lockMode)) { | ||
| // No count update: prev was replaced (not added) and allowReenter | ||
| // guarantees the mode is unchanged, so lockModeHeldCount stays in sync. | ||
| waiter.lock(); | ||
|
|
||
| waiter.upgrade(prev); | ||
|
|
||
| assertLockModeHeldCount(); | ||
|
|
||
| return new IgniteBiTuple<>(nullCompletedFuture(), prev.lockMode()); | ||
| } else { | ||
| waiter.upgrade(prev); | ||
|
|
@@ -871,6 +881,8 @@ IgniteBiTuple<CompletableFuture<Void>, LockMode> tryAcquire(UUID txId, LockMode | |
| track(waiter.txId, this); | ||
| } | ||
|
|
||
| assertLockModeHeldCount(); | ||
|
|
||
| return new IgniteBiTuple<>(waiter.fut, waiter.lockMode()); | ||
| } | ||
|
|
||
|
|
@@ -884,6 +896,8 @@ IgniteBiTuple<CompletableFuture<Void>, LockMode> tryAcquire(UUID txId, LockMode | |
| track(waiter.txId, this); | ||
| } | ||
| } | ||
|
|
||
| assertLockModeHeldCount(); | ||
| } | ||
|
|
||
| // Notify outside the monitor. | ||
|
|
@@ -903,6 +917,52 @@ public int waitersCount() { | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * Checks if any currently held lock mode is incompatible with the intended mode. | ||
| * Runs in O(1) time (constant number of lock modes). Must be called under {@code synchronized (waiters)}. | ||
| */ | ||
| private boolean hasAnyIncompatibleHolder(LockMode intended) { | ||
| for (LockMode held : LOCK_MODES) { | ||
| if (lockModeHeldCount[held.ordinal()] > 0 && !held.isCompatible(intended)) { | ||
| return true; | ||
| } | ||
| } | ||
| return false; | ||
| } | ||
|
|
||
| /** Adjusts {@code lockModeHeldCount} for a mode transition. Must be called under {@code synchronized (waiters)}. */ | ||
| private void updateHeldCount(@Nullable LockMode oldMode, @Nullable LockMode newMode) { | ||
| if (oldMode == newMode) { | ||
| return; | ||
| } | ||
| if (oldMode != null) { | ||
| lockModeHeldCount[oldMode.ordinal()]--; | ||
| } | ||
| if (newMode != null) { | ||
| lockModeHeldCount[newMode.ordinal()]++; | ||
| } | ||
| } | ||
|
|
||
| /** Locks the waiter and updates {@code lockModeHeldCount}. Must be called under {@code synchronized (waiters)}. */ | ||
| private void lockOnWaiter(WaiterImpl waiter) { | ||
| LockMode oldMode = waiter.lockMode(); | ||
| waiter.lock(); | ||
| updateHeldCount(oldMode, waiter.lockMode()); | ||
| } | ||
|
|
||
| /** Validates lockModeHeldCount is consistent with actual waiter state. Must be called under {@code synchronized (waiters)}. */ | ||
| private void assertLockModeHeldCount() { | ||
| int[] expected = new int[LOCK_MODES.length]; | ||
| for (WaiterImpl w : waiters.values()) { | ||
| if (w.lockMode != null) { | ||
| expected[w.lockMode.ordinal()]++; | ||
| } | ||
| } | ||
|
Comment on lines
+956
to
+960
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Are you sure we want to check this every time? |
||
| assert Arrays.equals(expected, lockModeHeldCount) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. As I remember, we didn't want to use |
||
| : "lockModeHeldCount out of sync [expected=" + Arrays.toString(expected) | ||
| + ", actual=" + Arrays.toString(lockModeHeldCount) + ']'; | ||
| } | ||
|
|
||
| /** | ||
| * Checks current waiter. It can change the internal state of the waiter. | ||
| * | ||
|
|
@@ -914,6 +974,14 @@ private boolean isWaiterReadyToNotify(WaiterImpl waiter, boolean skipFail) { | |
|
|
||
| assert intendedLockMode != null : "Intended lock mode is null"; | ||
|
|
||
| // Fast path: no incompatible holders — grant immediately. | ||
| // Conservative for upgrades: the waiter's own mode is in the counts, causing a fallthrough to the slow path. | ||
| if (!hasAnyIncompatibleHolder(intendedLockMode)) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. is it correct that the main idea is adding fast path for |
||
| lockOnWaiter(waiter); | ||
|
|
||
| return true; | ||
| } | ||
|
|
||
| for (Entry<UUID, WaiterImpl> entry : waiters.tailMap(waiter.txId(), false).entrySet()) { | ||
| WaiterImpl tmp = entry.getValue(); | ||
| LockMode currentlyAcquiredLockMode = tmp.lockMode; | ||
|
|
@@ -983,7 +1051,7 @@ private boolean isWaiterReadyToNotify(WaiterImpl waiter, boolean skipFail) { | |
| } | ||
| } | ||
|
|
||
| waiter.lock(); | ||
| lockOnWaiter(waiter); | ||
|
|
||
| return true; | ||
| } | ||
|
|
@@ -1000,6 +1068,7 @@ public boolean tryRelease(UUID txId) { | |
|
|
||
| synchronized (waiters) { | ||
| toNotify = release(txId); | ||
| assertLockModeHeldCount(); | ||
| } | ||
|
|
||
| // Notify outside the monitor. | ||
|
|
@@ -1028,18 +1097,24 @@ boolean tryRelease(UUID txId, LockMode lockMode) { | |
| assert LockMode.supremum(lockMode, waiter.lockMode()) == waiter.lockMode() : | ||
| "The lock is not locked in specified mode [mode=" + lockMode + ", locked=" + waiter.lockMode() + ']'; | ||
|
|
||
| LockMode modeFromDowngrade = waiter.recalculateMode(lockMode); | ||
| LockMode oldMode = waiter.lockMode(); | ||
| waiter.recalculateMode(lockMode); | ||
| LockMode newMode = waiter.lockMode(); | ||
|
|
||
| updateHeldCount(oldMode, newMode); | ||
|
|
||
| if (!waiter.locked() && !waiter.hasLockIntent()) { | ||
| // All locks are revoked - deqeue waiter. | ||
| // All locks are revoked - dequeue waiter. | ||
| waiters.remove(txId); | ||
| if (!waiters.isEmpty()) { | ||
| toNotify = unlockCompatibleWaiters(); | ||
| } | ||
| } else if (modeFromDowngrade != waiter.lockMode()) { | ||
| } else if (oldMode != newMode) { | ||
| toNotify = unlockCompatibleWaiters(); | ||
| } | ||
| } | ||
|
|
||
| assertLockModeHeldCount(); | ||
| } | ||
|
|
||
| // Notify outside the monitor. | ||
|
|
@@ -1060,7 +1135,13 @@ private List<WaiterImpl> release(UUID txId) { | |
| WaiterImpl removed = waiters.remove(txId); | ||
|
|
||
| // Removing incomplete waiter doesn't affect lock state. | ||
| if (removed == null || waiters.isEmpty() || !removed.locked()) { | ||
| if (removed == null || !removed.locked()) { | ||
| return emptyList(); | ||
| } | ||
|
|
||
| updateHeldCount(removed.lockMode(), null); | ||
|
|
||
| if (waiters.isEmpty()) { | ||
| return emptyList(); | ||
| } | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
releaseflow seems to stay uncovered by this fix. Meanwhile, there is also a usage of heavy method under lock:unlockCompatibleWaiters