From cc7cf981a0231dbad3009a0c849f7cfa2de6220d Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Wed, 18 Oct 2023 18:13:40 +0300 Subject: [PATCH 01/40] IGNITE-17811 Implement bounded lock table --- .../benchmark/LockManagerBenchmark.java | 104 +++++++++ ...ributedTestSingleNodeNoCleanupMessage.java | 2 +- .../distributed/raft/PartitionListener.java | 2 - .../replicator/PartitionReplicaListener.java | 2 +- .../replicator/TransactionStateResolver.java | 2 +- ...titionReplicaListenerIndexLockingTest.java | 2 +- .../PartitionReplicaListenerTest.java | 2 +- .../ignite/internal/table/TxAbstractTest.java | 7 +- .../ignite/internal/tx/LockManager.java | 23 +- .../internal/tx/impl/HeapLockManager.java | 205 +++++++++++------- .../internal/tx/AbstractLockManagerTest.java | 6 +- .../internal/tx/HeapLockManagerTest.java | 3 +- 12 files changed, 260 insertions(+), 100 deletions(-) create mode 100644 modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java new file mode 100644 index 00000000000..d33d2a804af --- /dev/null +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java @@ -0,0 +1,104 @@ +package org.apache.ignite.internal.benchmark; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.internal.TestHybridClock; +import org.apache.ignite.internal.hlc.HybridClock; +import org.apache.ignite.internal.storage.RowId; +import org.apache.ignite.internal.tx.Lock; +import org.apache.ignite.internal.tx.LockKey; +import org.apache.ignite.internal.tx.LockManager; +import org.apache.ignite.internal.tx.LockMode; +import org.apache.ignite.internal.tx.impl.HeapLockManager; +import org.apache.ignite.internal.tx.impl.TransactionIdGenerator; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +@State(Scope.Benchmark) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +public class LockManagerBenchmark { + private LockManager lockManager; + private TransactionIdGenerator generator; + private HybridClock clock; + + /** + * Initializes session and statement. + */ + @Setup + public void setUp() { + lockManager = new HeapLockManager(); + generator = new TransactionIdGenerator(0); + clock = new TestHybridClock(() -> 0L); + } + + /** + * Closes resources. + */ + @TearDown + public void tearDown() throws Exception { + assert lockManager.isEmpty(); + } + + /** + * Concurrent active transactions. + */ + @Param({"200"}) + private int concTxns; + + private int iter; + + @Benchmark + @Warmup(iterations = 1, time = 3) + @Measurement(iterations = 1, time = 10) + public void lockCommit() { + List ids = new ArrayList<>(concTxns); + + int c = 0; + + for (int i = 0; i < concTxns; i++) { + UUID txId = generator.transactionIdFor(clock.now()); + ids.add(txId); + CompletableFuture fut = lockManager.acquire(txId, new LockKey(0, new RowId(0, new UUID(0, c++))), LockMode.X); + fut.join(); + } + + for (UUID id : ids) { + lockManager.releaseAll(id); + } + +// if (!lockManager.isEmpty()) { +// throw new IllegalStateException(); +// } + } + + /** + * Benchmark's entry point. + */ + public static void main(String[] args) throws RunnerException { + // TODO JVM args + Options opt = new OptionsBuilder() + .include(".*" + LockManagerBenchmark.class.getSimpleName() + ".*") + .forks(0) + .threads(1) + .mode(Mode.AverageTime) + .build(); + + new Runner(opt).run(); + } +} diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNodeNoCleanupMessage.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNodeNoCleanupMessage.java index 08c7bcee7a6..ceeeb95a35d 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNodeNoCleanupMessage.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNodeNoCleanupMessage.java @@ -267,6 +267,6 @@ public void testTwoReadWriteTransactionsWaitForCleanup() throws TransactionExcep } private static void releaseTxLocks(UUID txId, LockManager lockManager) { - lockManager.locks(txId).forEachRemaining(lockManager::release); + lockManager.releaseAll(txId); } } diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionListener.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionListener.java index 6ed5a8464ba..13eba978e82 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionListener.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionListener.java @@ -148,8 +148,6 @@ public void onWrite(Iterator> iterator) { iterator.forEachRemaining((CommandClosure clo) -> { Command command = clo.command(); - // LOG.info("CMD {}", command.getClass().getName()); - long commandIndex = clo.index(); long commandTerm = clo.term(); diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java index ee6b2183a84..81cb59235b8 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java @@ -1418,7 +1418,7 @@ private static CompletableFuture allOffFuturesExceptionIgnored(List future) { future.thenAccept(txMeta -> { - if (txMeta != null && txMeta instanceof TxStateMeta) { + if (txMeta instanceof TxStateMeta) { txManager.updateTxMeta(txId, old -> (TxStateMeta) txMeta); } }); diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java index c972553a1d4..fc79f7911b6 100644 --- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java +++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java @@ -249,7 +249,7 @@ public void beforeTest() { ((TestHashIndexStorage) pkStorage.get().storage()).clear(); TEST_MV_PARTITION_STORAGE.clear(); - locks().forEach(LOCK_MANAGER::release); + LOCK_MANAGER.releaseAll(TRANSACTION_ID); } /** Verifies the mode in which the lock was acquired on the index key for a particular operation. */ diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java index 9985d158305..30ba839f113 100644 --- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java +++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java @@ -235,7 +235,7 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest { } } - lockManager.locks(cmd.txId()).forEachRemaining(lockManager::release); + lockManager.releaseAll(cmd.txId()); } else if (cmd instanceof UpdateCommand) { pendingRows.compute(cmd.txId(), (txId, v) -> { if (v == null) { diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java index 8213b926175..aa7414c99a8 100644 --- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java +++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java @@ -68,6 +68,7 @@ import org.apache.ignite.internal.tx.TxManager; import org.apache.ignite.internal.tx.TxState; import org.apache.ignite.internal.tx.TxStateMeta; +import org.apache.ignite.internal.tx.Waiter; import org.apache.ignite.internal.tx.impl.ReadWriteTransactionImpl; import org.apache.ignite.internal.util.CollectionUtils; import org.apache.ignite.internal.util.Pair; @@ -510,10 +511,10 @@ public void testBatchReadPutConcurrently() throws InterruptedException { assertTrue(IgniteTestUtils.waitForCondition(() -> { boolean lockUpgraded = false; - for (Iterator it = txManager(accounts).lockManager().locks(tx1.id()); it.hasNext(); ) { - Lock lock = it.next(); + for (Iterator it = txManager(accounts).lockManager().locks(tx1.id()); it.hasNext(); ) { + Waiter waiter = it.next(); - lockUpgraded = txManager(accounts).lockManager().waiter(lock.lockKey(), tx1.id()).intendedLockMode() == LockMode.X; + lockUpgraded = waiter.intendedLockMode() == LockMode.X; if (lockUpgraded) { break; diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java index 3ab221d8544..31351042ae8 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java @@ -25,7 +25,6 @@ /** Lock manager allows to acquire locks and release locks and supports deadlock prevention by transaction id ordering. */ public interface LockManager { - /** * Attempts to acquire a lock for the specified {@code lockKey} in specified {@code lockMode}. * @@ -34,14 +33,15 @@ public interface LockManager { * @param lockMode Lock mode, for example shared, exclusive, intention-shared etc. * @return The future with gained lock that will be completed when a lock is successfully acquired. */ - public CompletableFuture acquire(UUID txId, LockKey lockKey, LockMode lockMode); + CompletableFuture acquire(UUID txId, LockKey lockKey, LockMode lockMode); /** * Attempts to release the specified lock. * * @param lock Lock to release. */ - public void release(Lock lock); + @TestOnly + void release(Lock lock); /** * Release a lock that is held on the specific mode on the specific key. @@ -53,12 +53,19 @@ public interface LockManager { void release(UUID txId, LockKey lockKey, LockMode lockMode); /** - * Retrieves all locks for the specified transaction id. + * Retrieves all waiters for the specified transaction id. * * @param txId Transaction Id. * @return An iterator over a collection of locks. */ - public Iterator locks(UUID txId); + Iterator locks(UUID txId); + + /** + * Release all locks associated with a transaction. + * + * @param txId Tx id. + */ + void releaseAll(UUID txId); /** * Returns a collection of transaction ids that is associated with the specified {@code key}. @@ -67,7 +74,7 @@ public interface LockManager { * @return The waiters queue. */ @TestOnly - public Collection queue(LockKey key); + Collection queue(LockKey key); /** * Returns a waiter associated with the specified {@code key}. @@ -77,7 +84,7 @@ public interface LockManager { * @return The waiter. */ @TestOnly - public Waiter waiter(LockKey key, UUID txId); + Waiter waiter(LockKey key, UUID txId); /** * Returns {@code true} if no locks have been held. @@ -85,5 +92,5 @@ public interface LockManager { * @return {@code true} if no locks have been held. */ @TestOnly - public boolean isEmpty(); + boolean isEmpty(); } diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index 985ee08c135..4d978b673fa 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -25,7 +25,8 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; -import java.util.HashMap; +import java.util.EnumMap; +import java.util.EnumSet; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -35,6 +36,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import org.apache.ignite.internal.lang.IgniteBiTuple; @@ -60,18 +62,39 @@ * the queue). */ public class HeapLockManager implements LockManager { - private ConcurrentHashMap locks = new ConcurrentHashMap<>(); + private static final int HASH_BITS = 0x7fffffff; + private static final int SLOTS = 131059; + /** + * Lock queues. + */ + private final LockState[] slots; + + /** + * The policy. + */ private final DeadlockPreventionPolicy deadlockPreventionPolicy; /** Executor that is used to fail waiters after timeout. */ private final Executor delayedExecutor; + /** + * Enlisted transactions. + */ + private final ConcurrentHashMap> txMap = new ConcurrentHashMap<>(1024); + /** * Constructor. */ public HeapLockManager() { - this(new WaitDieDeadlockPreventionPolicy()); + this(new WaitDieDeadlockPreventionPolicy(), SLOTS); + } + + /** + * Constructor. + */ + public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy) { + this(deadlockPreventionPolicy, SLOTS); } /** @@ -79,64 +102,68 @@ public HeapLockManager() { * * @param deadlockPreventionPolicy Deadlock prevention policy. */ - public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy) { + public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy, int maxSize) { this.deadlockPreventionPolicy = deadlockPreventionPolicy; this.delayedExecutor = deadlockPreventionPolicy.waitTimeout() > 0 ? CompletableFuture.delayedExecutor(deadlockPreventionPolicy.waitTimeout(), TimeUnit.MILLISECONDS) : null; + + LockState[] tmp = new LockState[maxSize]; + for (int i = 0; i < tmp.length; i++) { + tmp[i] = new LockState(); + } + + slots = tmp; // Atomic init. } @Override public CompletableFuture acquire(UUID txId, LockKey lockKey, LockMode lockMode) { - while (true) { - LockState state = lockState(lockKey); - - IgniteBiTuple, LockMode> futureTuple = state.tryAcquire(txId, lockMode); + LockState state = lockState(lockKey); - if (futureTuple.get1() == null) { - continue; // Obsolete state. - } + IgniteBiTuple, LockMode> futureTuple = state.tryAcquire(txId, lockMode); - LockMode newLockMode = futureTuple.get2(); + LockMode newLockMode = futureTuple.get2(); - return futureTuple.get1().thenApply(res -> new Lock(lockKey, newLockMode, txId)); - } + return futureTuple.get1().thenApply(res -> new Lock(lockKey, newLockMode, txId)); } @Override public void release(Lock lock) { LockState state = lockState(lock.lockKey()); - if (state.tryRelease(lock.txId())) { - locks.remove(lock.lockKey(), state); - } + state.tryRelease(lock.txId()); } @Override public void release(UUID txId, LockKey lockKey, LockMode lockMode) { LockState state = lockState(lockKey); - if (state.tryRelease(txId, lockMode)) { - locks.remove(lockKey, state); + state.tryRelease(txId, lockMode); + } + + @Override + public void releaseAll(UUID txId) { + ConcurrentLinkedQueue states = this.txMap.remove(txId); + + if (states != null) { + for (LockState state : states) { + state.tryRelease(txId); + } } } @Override - public Iterator locks(UUID txId) { - // TODO: IGNITE-17811 Use index or similar instead of full locks set iteration. - List result = new ArrayList<>(); - - for (Map.Entry entry : locks.entrySet()) { - Waiter waiter = entry.getValue().waiter(txId); - - if (waiter != null) { - result.add( - new Lock( - entry.getKey(), - waiter.lockMode(), - txId - ) - ); + public Iterator locks(UUID txId) { + ConcurrentLinkedQueue lockStates = txMap.get(txId); + List result = new ArrayList<>(); + + if (lockStates != null) { + for (LockState lockState : lockStates) { + Waiter waiter = lockState.waiter(txId); + + if (waiter != null) { + result.add(waiter); + } } } @@ -149,7 +176,10 @@ public Iterator locks(UUID txId) { * @param key The key. */ private LockState lockState(LockKey key) { - return locks.computeIfAbsent(key, k -> new LockState(deadlockPreventionPolicy, delayedExecutor)); + int hash = key.hashCode(); + int h = hash & HASH_BITS; // spread(key.hashCode()); + int index = h % slots.length; // h & (size - 1); + return slots[index]; } /** {@inheritDoc} */ @@ -164,28 +194,29 @@ public Waiter waiter(LockKey key, UUID txId) { return lockState(key).waiter(txId); } + /** {@inheritDoc} */ + @Override + public boolean isEmpty() { + for (LockState slot : slots) { + if (!slot.waiters.isEmpty()) + return false; + } + + return true; + } + /** * A lock state. */ - private static class LockState { + protected class LockState { /** Waiters. */ private final TreeMap waiters; - private final DeadlockPreventionPolicy deadlockPreventionPolicy; - - /** Delayed executor for waiters timeout callback. */ - private final Executor delayedExecutor; - - /** Marked for removal flag. */ - private boolean markedForRemove = false; - - public LockState(DeadlockPreventionPolicy deadlockPreventionPolicy, Executor delayedExecutor) { + public LockState() { Comparator txComparator = deadlockPreventionPolicy.txIdComparator() != null ? deadlockPreventionPolicy.txIdComparator() : UUID::compareTo; this.waiters = new TreeMap<>(txComparator); - this.deadlockPreventionPolicy = deadlockPreventionPolicy; - this.delayedExecutor = delayedExecutor; } /** @@ -199,10 +230,6 @@ public LockState(DeadlockPreventionPolicy deadlockPreventionPolicy, Executor del WaiterImpl waiter = new WaiterImpl(txId, lockMode); synchronized (waiters) { - if (markedForRemove) { - return new IgniteBiTuple(null, lockMode); - } - // We always replace the previous waiter with the new one. If the previous waiter has lock intention then incomplete // lock future is copied to the new waiter. This guarantees that, if the previous waiter was locked concurrently, then // it doesn't have any lock intentions, and the future is not copied to the new waiter. Otherwise, if there is lock @@ -217,7 +244,7 @@ public LockState(DeadlockPreventionPolicy deadlockPreventionPolicy, Executor del waiter.upgrade(prev); - return new IgniteBiTuple(completedFuture(null), prev.lockMode()); + return new IgniteBiTuple<>(completedFuture(null), prev.lockMode()); } else { waiter.upgrade(prev); @@ -231,6 +258,11 @@ public LockState(DeadlockPreventionPolicy deadlockPreventionPolicy, Executor del setWaiterTimeout(waiter); } + // Put to wait queue, track. + if (prev == null) { + track(waiter.txId); + } + return new IgniteBiTuple<>(waiter.fut, waiter.lockMode()); } @@ -238,13 +270,16 @@ public LockState(DeadlockPreventionPolicy deadlockPreventionPolicy, Executor del waiters.remove(waiter.txId()); } else if (waiter.hasLockIntent()) { waiter.refuseIntent(); // Restore old lock. + } else { + // Lock granted, track. + track(waiter.txId); } } // Notify outside the monitor. waiter.notifyLocked(); - return new IgniteBiTuple(waiter.fut, waiter.lockMode()); + return new IgniteBiTuple<>(waiter.fut, waiter.lockMode()); } /** @@ -309,7 +344,7 @@ private LockException lockException(UUID txId, WaiterImpl conflictingWaiter) { * @param txId Transaction id. * @return {@code True} if the queue is empty. */ - public boolean tryRelease(UUID txId) { + void tryRelease(UUID txId) { Collection toNotify; synchronized (waiters) { @@ -320,25 +355,23 @@ public boolean tryRelease(UUID txId) { for (WaiterImpl waiter : toNotify) { waiter.notifyLocked(); } - - return markedForRemove; } /** - * Releases a specific lock of the key. + * Releases a specific lock of the key, if a key is locked in multiple modes by the same locker. * * @param txId Transaction id. * @param lockMode Lock mode. * @return If the value is true, no one waits of any lock of the key, false otherwise. */ - public boolean tryRelease(UUID txId, LockMode lockMode) { + void tryRelease(UUID txId, LockMode lockMode) { List toNotify = Collections.emptyList(); synchronized (waiters) { WaiterImpl waiter = waiters.get(txId); if (waiter != null) { - assert lockMode.supremum(lockMode, waiter.lockMode()) == waiter.lockMode() : - "The lock mode is not locked [mode=" + lockMode + ", locked=" + waiter.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); @@ -354,29 +387,23 @@ public boolean tryRelease(UUID txId, LockMode lockMode) { for (WaiterImpl waiter : toNotify) { waiter.notifyLocked(); } - - return markedForRemove; } /** - * Releases all locks are held by a specific transaction. - * This method should be invoked synchronously. + * Releases all locks are held by a specific transaction. This method should be invoked synchronously. * * @param txId Transaction id. * @return List of waiters to notify. */ private List release(UUID txId) { waiters.remove(txId); + untrack(txId); if (waiters.isEmpty()) { - markedForRemove = true; - return Collections.emptyList(); } - List toNotify = unlockCompatibleWaiters(); - - return toNotify; + return unlockCompatibleWaiters(); } /** @@ -481,6 +508,32 @@ public Waiter waiter(UUID txId) { return waiters.get(txId); } } + + private void track(UUID txId) { + txMap.compute(txId, (k, v) -> { + if (v == null) { + v = new ConcurrentLinkedQueue<>(); + } + + v.add(this); + + return v; + }); + } + + private void untrack(UUID txId) { + assert waiter(txId) == null; + + txMap.compute(txId, (k, v) -> { + if (v == null) { + return v; + } + + v.remove(this); + + return v; + }); + } } /** @@ -489,15 +542,13 @@ public Waiter waiter(UUID txId) { private static class WaiterImpl implements Comparable, Waiter { /** * Holding locks by type. - * TODO: IGNITE-18350 Abandon the collection in favor of BitSet. */ - private final Map locks = new HashMap<>(); + private final Map locks = new EnumMap<>(LockMode.class); /** * Lock modes are marked as intended, but have not taken yet. This is NOT specific to intention lock modes, such as IS and IX. - * TODO: IGNITE-18350 Abandon the collection in favor of BitSet. */ - private final Set intendedLocks = new HashSet<>(); + private final Set intendedLocks = EnumSet.noneOf(LockMode.class); /** Locked future. */ @IgniteToStringExclude @@ -546,7 +597,7 @@ void addLock(LockMode lockMode, int increment) { * Removes a lock mode. * * @param lockMode Lock mode. - * @return True if the lock mode was removed, false otherwise. + * @return True if the lock is not locked in the passed mode, false otherwise. */ private boolean removeLock(LockMode lockMode) { Integer counter = locks.get(lockMode); @@ -724,9 +775,7 @@ public String toString() { } } - /** {@inheritDoc} */ - @Override - public boolean isEmpty() { - return locks.isEmpty(); + private static int spread(int h) { + return (h ^ (h >>> 16)) & 0x7fffffff; } } diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/AbstractLockManagerTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/AbstractLockManagerTest.java index d719b6c2909..28f8c2194c9 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/AbstractLockManagerTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/AbstractLockManagerTest.java @@ -126,7 +126,7 @@ public void downgradeLockOutOfTurnTest() { LockKey key = new LockKey("test"); lockManager.acquire(txId0, key, S).join(); - lockManager.acquire(txId2, key, S).join(); + Lock lock = lockManager.acquire(txId2, key, S).join(); CompletableFuture fut0 = lockManager.acquire(txId0, key, X); assertFalse(fut0.isDone()); @@ -139,8 +139,8 @@ public void downgradeLockOutOfTurnTest() { assertFalse(fut0.isDone()); - lockManager.release(lockManager.locks(txId2).next()); - fut0.thenAccept(lock -> lockManager.release(lock)); + lockManager.release(lock); + fut0.thenAccept(l -> lockManager.release(l)); } @Test diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapLockManagerTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapLockManagerTest.java index 2d08c52bf0a..76b83571bc1 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapLockManagerTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapLockManagerTest.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.tx; import org.apache.ignite.internal.tx.impl.HeapLockManager; +import org.apache.ignite.internal.tx.impl.WaitDieDeadlockPreventionPolicy; /** * Test class for {@link HeapLockManager}. @@ -25,6 +26,6 @@ public class HeapLockManagerTest extends AbstractLockManagerTest { @Override protected LockManager newInstance() { - return new HeapLockManager(); + return new HeapLockManager(new WaitDieDeadlockPreventionPolicy()); } } From 3f57ac20c9d84517a0c93715ee425ad0557d5491 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Thu, 19 Oct 2023 14:43:27 +0300 Subject: [PATCH 02/40] IGNITE-17811 Fix tests --- ...titionReplicaListenerIndexLockingTest.java | 19 ++-------- .../ignite/internal/tx/LockManager.java | 17 +++++++++ .../internal/tx/impl/HeapLockManager.java | 38 ++++++++++++++++--- 3 files changed, 54 insertions(+), 20 deletions(-) diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java index fc79f7911b6..127dbc1b57c 100644 --- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java +++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java @@ -31,7 +31,6 @@ import static org.mockito.Mockito.when; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import java.util.Locale; import java.util.Map; @@ -300,12 +299,13 @@ void testReadWriteSingle(ReadWriteTestArg arg) throws MarshallerException { throw new AssertionError("Unexpected operation type: " + arg.type); } + LOCK_MANAGER.recordLocks(true); CompletableFuture fut = partitionReplicaListener.invoke(request, "local"); await(fut); assertThat( - locks(), + LOCK_MANAGER.recordedLocks(), allOf( hasItem(lockThat( arg.expectedLockOnUniqueHash + " on unique hash index", @@ -378,13 +378,14 @@ void testReadWriteMulti(ReadWriteTestArg arg) throws MarshallerException { throw new AssertionError("Unexpected operation type: " + arg.type); } + LOCK_MANAGER.recordLocks(true); CompletableFuture fut = partitionReplicaListener.invoke(request, "local"); await(fut); for (BinaryRow row : rows) { assertThat( - locks(), + LOCK_MANAGER.recordedLocks(), allOf( hasItem(lockThat( arg.expectedLockOnUniqueHash + " on unique hash index", @@ -432,18 +433,6 @@ private static Iterable readWriteMultiTestArguments() { ); } - private List locks() { - List locks = new ArrayList<>(); - - Iterator it = LOCK_MANAGER.locks(TRANSACTION_ID); - - while (it.hasNext()) { - locks.add(it.next()); - } - - return locks; - } - private void insertRows(List> rows, UUID txId) { HybridTimestamp commitTs = CLOCK.now(); diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java index 31351042ae8..266c2e9b33c 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java @@ -19,6 +19,7 @@ import java.util.Collection; import java.util.Iterator; +import java.util.List; import java.util.UUID; import java.util.concurrent.CompletableFuture; import org.jetbrains.annotations.TestOnly; @@ -93,4 +94,20 @@ public interface LockManager { */ @TestOnly boolean isEmpty(); + + /** + * Record acquired locks. + * + * @param mode Mode. + */ + @TestOnly + void recordLocks(boolean mode); + + /** + * Return recorded locks. Can clear collection to reset state. + * + * @return Recorded locks. + */ + @TestOnly + List recordedLocks(); } diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index 4d978b673fa..9440405e612 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -63,7 +63,8 @@ */ public class HeapLockManager implements LockManager { private static final int HASH_BITS = 0x7fffffff; - private static final int SLOTS = 131059; + + private static final int SLOTS = 131072; /** * Lock queues. @@ -83,6 +84,16 @@ public class HeapLockManager implements LockManager { */ private final ConcurrentHashMap> txMap = new ConcurrentHashMap<>(1024); + /** + * {@code True} to record acquired locks + */ + private boolean record; + + /** + * Recorded locks. + */ + private final List recordedLocks = new ArrayList<>(); + /** * Constructor. */ @@ -124,7 +135,15 @@ public CompletableFuture acquire(UUID txId, LockKey lockKey, LockMode lock LockMode newLockMode = futureTuple.get2(); - return futureTuple.get1().thenApply(res -> new Lock(lockKey, newLockMode, txId)); + return futureTuple.get1().thenApply(res -> { + Lock lock = new Lock(lockKey, newLockMode, txId); + + if (record) { + recordedLocks.add(lock); + } + + return lock; + }); } @Override @@ -176,9 +195,8 @@ public Iterator locks(UUID txId) { * @param key The key. */ private LockState lockState(LockKey key) { - int hash = key.hashCode(); - int h = hash & HASH_BITS; // spread(key.hashCode()); - int index = h % slots.length; // h & (size - 1); + int h = spread(key.hashCode()); + int index = h & (slots.length - 1); return slots[index]; } @@ -775,6 +793,16 @@ public String toString() { } } + @Override + public void recordLocks(boolean mode) { + record = mode; + } + + @Override + public List recordedLocks() { + return recordedLocks; + } + private static int spread(int h) { return (h ^ (h >>> 16)) & 0x7fffffff; } From 17dc1a34a3977c9c0659d045bcc6c1ee10d380e4 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Thu, 19 Oct 2023 15:04:26 +0300 Subject: [PATCH 03/40] IGNITE-17811 Cleanup --- .../org/apache/ignite/internal/tx/impl/HeapLockManager.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index 9440405e612..621a9128a00 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -62,8 +62,9 @@ * the queue). */ public class HeapLockManager implements LockManager { - private static final int HASH_BITS = 0x7fffffff; - + /** + * Table size. TODO make it configurable IGNITE-20694 + */ private static final int SLOTS = 131072; /** From 009c910fa2f6655c90162b658e9a650ef9e50b74 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Thu, 19 Oct 2023 15:30:01 +0300 Subject: [PATCH 04/40] IGNITE-17811 Removed untrack --- .../ignite/internal/tx/impl/HeapLockManager.java | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index 621a9128a00..b20e6201301 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -416,7 +416,6 @@ void tryRelease(UUID txId, LockMode lockMode) { */ private List release(UUID txId) { waiters.remove(txId); - untrack(txId); if (waiters.isEmpty()) { return Collections.emptyList(); @@ -539,20 +538,6 @@ private void track(UUID txId) { return v; }); } - - private void untrack(UUID txId) { - assert waiter(txId) == null; - - txMap.compute(txId, (k, v) -> { - if (v == null) { - return v; - } - - v.remove(this); - - return v; - }); - } } /** From 1a23d3c6c90b851556b46637f63c48bf237970f2 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Mon, 30 Oct 2023 13:41:42 +0300 Subject: [PATCH 05/40] IGNITE-17811 Fix comment --- .../org/apache/ignite/internal/tx/impl/HeapLockManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index b20e6201301..4d93f83b8e4 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -55,8 +55,8 @@ * A {@link LockManager} implementation which stores lock queues in the heap. * *

Lock waiters are placed in the queue, ordered according to comparator provided by {@link HeapLockManager#deadlockPreventionPolicy}. - * When a new waiter is placed in the queue, it's validated against current lock owner: if there is an owner with a higher transaction id - * lock request is denied. + * When a new waiter is placed in the queue, it's validated against current lock owner: if there is an owner with a higher priority + * (as defined by comparator) lock request is denied. * *

Read lock can be upgraded to write lock (only available for the lowest read-locked entry of * the queue). From 4207569a61101a684d3f10cbbb24efdd601076de Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Mon, 30 Oct 2023 16:47:24 +0300 Subject: [PATCH 06/40] IGNITE-17811 Temporary disable suspicious test --- .../replication/PartitionReplicaListenerIndexLockingTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java index 9953d012ed7..3f1e5cd3b6e 100644 --- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java +++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java @@ -110,12 +110,14 @@ import org.hamcrest.Matcher; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; /** There are tests for partition replica listener. */ @ExtendWith(ConfigurationExtension.class) +@Disabled public class PartitionReplicaListenerIndexLockingTest extends IgniteAbstractTest { private static final int PART_ID = 0; private static final int TABLE_ID = 1; @@ -283,6 +285,7 @@ public void beforeTest() { TEST_MV_PARTITION_STORAGE.clear(); LOCK_MANAGER.releaseAll(TRANSACTION_ID); + LOCK_MANAGER.recordedLocks().clear(); } /** Verifies the mode in which the lock was acquired on the index key for a particular operation. */ From a1c1b358208e3633584b8e3279a920635405aee9 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Mon, 30 Oct 2023 17:45:46 +0300 Subject: [PATCH 07/40] IGNITE-17811 Fix memory leak --- ...titionReplicaListenerIndexLockingTest.java | 1 - .../table/impl/DummyInternalTableImpl.java | 48 ++++++------------- 2 files changed, 15 insertions(+), 34 deletions(-) diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java index 3f1e5cd3b6e..cd414274f7a 100644 --- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java +++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java @@ -117,7 +117,6 @@ /** There are tests for partition replica listener. */ @ExtendWith(ConfigurationExtension.class) -@Disabled public class PartitionReplicaListenerIndexLockingTest extends IgniteAbstractTest { private static final int PART_ID = 0; private static final int TABLE_ID = 1; diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java index c3d191a5682..2cfd599ef5b 100644 --- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java +++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java @@ -46,6 +46,7 @@ import org.apache.ignite.internal.lang.IgniteInternalException; import org.apache.ignite.internal.logger.IgniteLogger; import org.apache.ignite.internal.logger.Loggers; +import org.apache.ignite.internal.placementdriver.PlacementDriver; import org.apache.ignite.internal.placementdriver.TestPlacementDriver; import org.apache.ignite.internal.raft.Command; import org.apache.ignite.internal.raft.Peer; @@ -101,10 +102,12 @@ import org.apache.ignite.network.NetworkAddress; import org.apache.ignite.tx.TransactionException; import org.jetbrains.annotations.Nullable; +import org.jetbrains.annotations.TestOnly; /** * Dummy table storage implementation. */ +@TestOnly public class DummyInternalTableImpl extends InternalTableImpl { public static final IgniteLogger LOG = Loggers.forClass(DummyInternalTableImpl.class); @@ -112,8 +115,6 @@ public class DummyInternalTableImpl extends InternalTableImpl { public static final ClusterNode LOCAL_NODE = new ClusterNodeImpl("id", "node", ADDR); - private static final TestPlacementDriver TEST_PLACEMENT_DRIVER = new TestPlacementDriver(LOCAL_NODE.name()); - // 2000 was picked to avoid negative time that we get when building read timestamp // in TxManagerImpl.currentReadTimestamp. // We subtract (ReplicaManager.IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS + HybridTimestamp.CLOCK_SKEW) = (1000 + 7) = 1007 @@ -160,29 +161,7 @@ public DummyInternalTableImpl(ReplicaService replicaSvc) { * @param schema Schema. */ public DummyInternalTableImpl(ReplicaService replicaSvc, SchemaDescriptor schema) { - this(replicaSvc, new TestMvPartitionStorage(0), schema); - } - - /** - * Creates a new local table. - * - * @param replicaSvc Replica service. - * @param txManager Transaction manager. - * @param crossTableUsage If this dummy table is going to be used in cross-table tests, it won't mock the calls of - * ReplicaService by itself. - * @param transactionStateResolver Transaction state resolver. - * @param schema Schema descriptor. - * @param tracker Observable timestamp tracker. - */ - public DummyInternalTableImpl( - ReplicaService replicaSvc, - TxManager txManager, - boolean crossTableUsage, - @Nullable TransactionStateResolver transactionStateResolver, - SchemaDescriptor schema, - HybridTimestampTracker tracker - ) { - this(replicaSvc, new TestMvPartitionStorage(0), txManager, crossTableUsage, transactionStateResolver, schema, tracker); + this(replicaSvc, new TestMvPartitionStorage(0), schema, new TestPlacementDriver(LOCAL_NODE.name())); } /** @@ -195,16 +174,18 @@ public DummyInternalTableImpl( public DummyInternalTableImpl( ReplicaService replicaSvc, MvPartitionStorage mvPartStorage, - SchemaDescriptor schema + SchemaDescriptor schema, + PlacementDriver placementDriver ) { this( replicaSvc, mvPartStorage, - txManager(replicaSvc), + txManager(replicaSvc, placementDriver), false, null, schema, - new HybridTimestampTracker() + new HybridTimestampTracker(), + placementDriver ); } @@ -227,7 +208,8 @@ public DummyInternalTableImpl( boolean crossTableUsage, @Nullable TransactionStateResolver transactionStateResolver, SchemaDescriptor schema, - HybridTimestampTracker tracker + HybridTimestampTracker tracker, + PlacementDriver placementDriver ) { super( "test", @@ -241,7 +223,7 @@ public DummyInternalTableImpl( replicaSvc, CLOCK, tracker, - TEST_PLACEMENT_DRIVER + placementDriver ); RaftGroupService svc = raftGroupServiceByPartitionId.get(PART_ID); @@ -387,7 +369,7 @@ public void result(@Nullable Serializable r) { LOCAL_NODE, new AlwaysSyncedSchemaSyncService(), catalogService, - TEST_PLACEMENT_DRIVER + new TestPlacementDriver(LOCAL_NODE.name()) ); partitionListener = new PartitionListener( @@ -432,14 +414,14 @@ public TxManager txManager() { * * @param replicaSvc Replica service to use. */ - public static TxManagerImpl txManager(ReplicaService replicaSvc) { + public static TxManagerImpl txManager(ReplicaService replicaSvc, PlacementDriver placementDriver) { return new TxManagerImpl( replicaSvc, new HeapLockManager(), CLOCK, new TransactionIdGenerator(0xdeadbeef), LOCAL_NODE::id, - TEST_PLACEMENT_DRIVER + placementDriver ); } From 34e328685ffc7a0ba44953265244b9583639e2ad Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Mon, 30 Oct 2023 18:41:59 +0300 Subject: [PATCH 08/40] IGNITE-17811 Fix compilation --- .../internal/table/impl/DummyInternalTableImpl.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java index 2cfd599ef5b..7c6e769c50d 100644 --- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java +++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java @@ -164,6 +164,18 @@ public DummyInternalTableImpl(ReplicaService replicaSvc, SchemaDescriptor schema this(replicaSvc, new TestMvPartitionStorage(0), schema, new TestPlacementDriver(LOCAL_NODE.name())); } + /** + * Creates a new local table. + * + * @param replicaSvc Replica service. + * @param mvPartStorage Multi version partition storage. + * @param schema Schema. + */ + public DummyInternalTableImpl(ReplicaService replicaSvc, MvPartitionStorage mvPartStorage, SchemaDescriptor schema) { + this(replicaSvc, mvPartStorage, schema, new TestPlacementDriver(LOCAL_NODE.name())); + } + + /** * Creates a new local table. * @@ -200,6 +212,7 @@ public DummyInternalTableImpl( * @param transactionStateResolver Transaction state resolver. * @param schema Schema descriptor. * @param tracker Observable timestamp tracker. + * @param placementDriver Placement driver. */ public DummyInternalTableImpl( ReplicaService replicaSvc, From 8ffd2fd53ec38dcd86ca50323928bc2c404d77f4 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Tue, 31 Oct 2023 15:55:53 +0300 Subject: [PATCH 09/40] IGNITE-17811 Leak fix attempt --- .../replication/PartitionReplicaListenerTest.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java index 58827cbb17e..6a5c85a6661 100644 --- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java +++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java @@ -182,6 +182,7 @@ import org.apache.ignite.tx.TransactionException; import org.hamcrest.Matcher; import org.jetbrains.annotations.Nullable; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; @@ -501,6 +502,11 @@ public void beforeTest( reset(); } + @AfterEach + public void cleanup() { + Mockito.framework().clearInlineMocks(); + } + private static SchemaDescriptor schemaDescriptorWith(int ver) { return new SchemaDescriptor(ver, new Column[]{ new Column("intKey".toUpperCase(Locale.ROOT), NativeTypes.INT32, false), From 5c363e89b283b35713afdf1616d721df074425d9 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Fri, 10 Nov 2023 18:14:07 +0300 Subject: [PATCH 10/40] IGNITE-17811 Bounded lock table --- .../benchmark/AbstractOneNodeBenchmark.java | 1 + .../benchmark/LockManagerBenchmark.java | 17 +- .../ignite/distributed/ItLockTableTest.java | 192 ++++++++++++++++++ .../internal/table/RecordBinaryViewImpl.java | 14 +- .../replicator/PartitionReplicaListener.java | 14 +- .../storage/InternalTableImpl.java | 129 +++++++----- ...titionReplicaListenerIndexLockingTest.java | 19 +- .../ignite/internal/table/TxAbstractTest.java | 6 +- .../apache/ignite/internal/tx/LockKey.java | 7 + .../ignite/internal/tx/LockManager.java | 2 +- .../internal/tx/impl/HeapLockManager.java | 177 +++++++++++++--- .../tx/impl/ReadWriteTransactionImpl.java | 3 - .../internal/tx/impl/TxManagerImpl.java | 62 +++--- 13 files changed, 504 insertions(+), 139 deletions(-) create mode 100644 modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/AbstractOneNodeBenchmark.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/AbstractOneNodeBenchmark.java index e2dbacc0aac..f3c6334f68a 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/AbstractOneNodeBenchmark.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/AbstractOneNodeBenchmark.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.Paths; import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.ignite.IgnitionManager; diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java index d33d2a804af..b23887ef493 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java @@ -31,7 +31,7 @@ import org.openjdk.jmh.runner.options.OptionsBuilder; @State(Scope.Benchmark) -@OutputTimeUnit(TimeUnit.MILLISECONDS) +@OutputTimeUnit(TimeUnit.MICROSECONDS) public class LockManagerBenchmark { private LockManager lockManager; private TransactionIdGenerator generator; @@ -52,7 +52,9 @@ public void setUp() { */ @TearDown public void tearDown() throws Exception { - assert lockManager.isEmpty(); + if (!lockManager.isEmpty()) { + throw new AssertionError("Invalid lock manager state"); + } } /** @@ -61,8 +63,6 @@ public void tearDown() throws Exception { @Param({"200"}) private int concTxns; - private int iter; - @Benchmark @Warmup(iterations = 1, time = 3) @Measurement(iterations = 1, time = 10) @@ -74,17 +74,12 @@ public void lockCommit() { for (int i = 0; i < concTxns; i++) { UUID txId = generator.transactionIdFor(clock.now()); ids.add(txId); - CompletableFuture fut = lockManager.acquire(txId, new LockKey(0, new RowId(0, new UUID(0, c++))), LockMode.X); - fut.join(); + lockManager.acquire(txId, new LockKey(0, new RowId(0, new UUID(0, c++))), LockMode.X).join(); } for (UUID id : ids) { lockManager.releaseAll(id); } - -// if (!lockManager.isEmpty()) { -// throw new IllegalStateException(); -// } } /** @@ -94,7 +89,7 @@ public static void main(String[] args) throws RunnerException { // TODO JVM args Options opt = new OptionsBuilder() .include(".*" + LockManagerBenchmark.class.getSimpleName() + ".*") - .forks(0) + .forks(1) .threads(1) .mode(Mode.AverageTime) .build(); diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java new file mode 100644 index 00000000000..d0d551ecc88 --- /dev/null +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -0,0 +1,192 @@ +package org.apache.ignite.distributed; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension; +import org.apache.ignite.internal.configuration.testframework.InjectConfiguration; +import org.apache.ignite.internal.hlc.HybridClock; +import org.apache.ignite.internal.logger.IgniteLogger; +import org.apache.ignite.internal.logger.Loggers; +import org.apache.ignite.internal.placementdriver.PlacementDriver; +import org.apache.ignite.internal.raft.configuration.RaftConfiguration; +import org.apache.ignite.internal.replicator.ReplicaService; +import org.apache.ignite.internal.schema.Column; +import org.apache.ignite.internal.schema.SchemaDescriptor; +import org.apache.ignite.internal.schema.configuration.GcConfiguration; +import org.apache.ignite.internal.table.TableImpl; +import org.apache.ignite.internal.testframework.IgniteAbstractTest; +import org.apache.ignite.internal.tx.DeadlockPreventionPolicy; +import org.apache.ignite.internal.tx.HybridTimestampTracker; +import org.apache.ignite.internal.tx.impl.HeapLockManager; +import org.apache.ignite.internal.tx.impl.HeapLockManager.LockState; +import org.apache.ignite.internal.tx.impl.TransactionIdGenerator; +import org.apache.ignite.internal.tx.impl.TxManagerImpl; +import org.apache.ignite.internal.type.NativeTypes; +import org.apache.ignite.network.ClusterNode; +import org.apache.ignite.raft.jraft.test.TestUtils; +import org.apache.ignite.table.RecordView; +import org.apache.ignite.table.Tuple; +import org.apache.ignite.tx.Transaction; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ConfigurationExtension.class) +public class ItLockTableTest extends IgniteAbstractTest { + private static final IgniteLogger LOG = Loggers.forClass(ItLockTableTest.class); + + private static int EMP_TABLE_ID = 2; + + private static final int CACHE_SIZE = 10; + + private static final String TABLE_NAME = "test"; + + private static SchemaDescriptor TABLE_SCHEMA = new SchemaDescriptor( + 1, + new Column[]{new Column("id".toUpperCase(), NativeTypes.INT32, false)}, + new Column[]{ + new Column("name".toUpperCase(), NativeTypes.STRING, true), + new Column("salary".toUpperCase(), NativeTypes.DOUBLE, true) + } + ); + + protected TableImpl testTable; + + protected final TestInfo testInfo; + + //TODO fsync can be turned on again after https://issues.apache.org/jira/browse/IGNITE-20195 + @InjectConfiguration("mock: { fsync: false }") + protected static RaftConfiguration raftConfiguration; + + @InjectConfiguration + protected static GcConfiguration gcConfig; + + private ItTxTestCluster txTestCluster; + + private HybridTimestampTracker timestampTracker = new HybridTimestampTracker(); + + /** + * The constructor. + * + * @param testInfo Test info. + */ + public ItLockTableTest(TestInfo testInfo) { + this.testInfo = testInfo; + } + + @BeforeEach + public void before() throws Exception { + txTestCluster = new ItTxTestCluster( + testInfo, + raftConfiguration, + gcConfig, + workDir, + 1, + 1, + false, + timestampTracker + ) { + @Override + protected TxManagerImpl newTxManager(ReplicaService replicaSvc, HybridClock clock, TransactionIdGenerator generator, + ClusterNode node, PlacementDriver placementDriver) { + return new TxManagerImpl( + replicaSvc, + new HeapLockManager(new DeadlockPreventionPolicy() { + }, HeapLockManager.SLOTS, CACHE_SIZE), + clock, + generator, + node::id, + placementDriver + ); + } + }; + txTestCluster.prepareCluster(); + + testTable = txTestCluster.startTable(TABLE_NAME, EMP_TABLE_ID, TABLE_SCHEMA); + + log.info("Tables have been started"); + } + + @AfterEach + public void after() throws Exception { + txTestCluster.shutdownCluster(); + } + + @Test + public void testCollision() { + RecordView view = testTable.recordView(); + + int i = 0; + final int count = 10_00; + List txns = new ArrayList<>(); + while (i++ < count) { + Transaction tx = txTestCluster.igniteTransactions().begin(); + view.insertAsync(tx, tuple(i, "x-" + i)); + txns.add(tx); + } + + assertTrue(TestUtils.waitForCondition(() -> { + int total = 0; + HeapLockManager lockManager = (HeapLockManager) txTestCluster.txManagers.get(txTestCluster.localNodeName).lockManager(); + for (int j = 0; j < lockManager.getSlots().length; j++) { + LockState slot = lockManager.getSlots()[j]; + total += slot.waitersCount(); + } + + return total == count && lockManager.available() == 0; + }, 10_000), "Some lockers are missing"); + + int empty = 0; + int coll = 0; + + HeapLockManager lm = (HeapLockManager) txTestCluster.txManagers.get(txTestCluster.localNodeName).lockManager(); + for (int j = 0; j < lm.getSlots().length; j++) { + LockState slot = lm.getSlots()[j]; + int cnt = slot.waitersCount(); + if (cnt == 0) { + empty++; + } + if (cnt > 1) { + coll += cnt; + } + } + + LOG.info("LockTable [emptySlots={} collisions={}]", empty, coll); + + List> finishFuts = new ArrayList<>(); + for (Transaction txn : txns) { + finishFuts.add(txn.commitAsync()); + } + + for (CompletableFuture finishFut : finishFuts) { + try { + finishFut.join(); + } catch (Exception e) { + // Ignore. + System.out.println(); + } + } + + assertTrue(TestUtils.waitForCondition(() -> { + int total = 0; + HeapLockManager lockManager = (HeapLockManager) txTestCluster.txManagers.get(txTestCluster.localNodeName).lockManager(); + for (int j = 0; j < lockManager.getSlots().length; j++) { + LockState slot = lockManager.getSlots()[j]; + total += slot.waitersCount(); + } + + return total == 0 && lockManager.available() == CACHE_SIZE; + }, 10_000), "Illegal lock manager state"); + } + + private static Tuple tuple(int id, String name) { + return Tuple.create() + .set("id", id) + .set("name", name); + } +} diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java index 7284c12a2b7..4b1befcab25 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java @@ -24,6 +24,7 @@ import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Flow.Publisher; +import java.util.function.Consumer; import org.apache.ignite.internal.schema.BinaryRow; import org.apache.ignite.internal.schema.BinaryRowEx; import org.apache.ignite.internal.schema.SchemaRegistry; @@ -432,7 +433,18 @@ public CompletableFuture streamData(Publisher publisher, @Nullable var partitioner = new TupleStreamerPartitionAwarenessProvider(rowConverter.registry(), tbl.partitions()); StreamerBatchSender batchSender = (partitionId, items) -> withSchemaSync(null, (schemaVersion) -> { - return this.tbl.upsertAll(mapToBinary(items, schemaVersion, false), partitionId); + System.out.println("BEGIN " + partitionId); + return this.tbl.upsertAll(mapToBinary(items, schemaVersion, false), partitionId).thenAccept(new Consumer() { + @Override + public void accept(Void unused) { + try { + Thread.sleep(500); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + System.out.println("END " + partitionId); + } + }); }); return DataStreamer.streamData(publisher, options, batchSender, partitioner); diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java index 667bf843766..c979cfb7b26 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java @@ -1396,8 +1396,9 @@ private CompletableFuture cleanup( UUID txId, int attemptsToCleanupReplica ) { + // Avoid invoking async chain in raft threads. CompletableFuture[] futures = enlistedPartitions.stream() - .map(partitionId -> changeStateFuture.thenCompose(ignored -> + .map(partitionId -> changeStateFuture.thenComposeAsync(ignored -> cleanupWithRetry(commit, commitTimestamp, txId, partitionId, attemptsToCleanupReplica))) .toArray(size -> new CompletableFuture[size]); @@ -1914,10 +1915,10 @@ private CompletableFuture> processReadOnlyDirectMultiEntryAction */ private CompletableFuture processMultiEntryAction(ReadWriteMultiRowReplicaRequest request, String txCoordinatorId) { UUID txId = request.transactionId(); - TablePartitionId committedPartitionId = request.commitPartitionId().asTablePartitionId(); + TablePartitionId commitdPartitionId = request.commitPartitionId().asTablePartitionId(); List searchRows = request.binaryRows(); - assert committedPartitionId != null : "Commit partition is null [type=" + request.requestType() + ']'; + assert commitdPartitionId != null : "Commit partition is null [type=" + request.requestType() + ']'; switch (request.requestType()) { case RW_DELETE_EXACT_ALL: { @@ -2918,8 +2919,11 @@ private CompletableFuture>> takeLocksForUp * @return Future completes with tuple {@link RowId} and collection of {@link Lock}. */ private CompletableFuture>> takeLocksForInsert(BinaryRow binaryRow, RowId rowId, UUID txId) { - return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table - .thenCompose(ignored -> takePutLockOnIndexes(binaryRow, rowId, txId)) +// return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table +// .thenCompose(ignored -> takePutLockOnIndexes(binaryRow, rowId, txId)) +// .thenApply(shortTermLocks -> new IgniteBiTuple<>(rowId, shortTermLocks)); + + return takePutLockOnIndexes(binaryRow, rowId, txId) .thenApply(shortTermLocks -> new IgniteBiTuple<>(rowId, shortTermLocks)); } diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java index 5a1eae715b4..44eb159a029 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java @@ -109,6 +109,7 @@ import org.apache.ignite.internal.tx.TxManager; import org.apache.ignite.internal.tx.storage.state.TxStateTableStorage; import org.apache.ignite.internal.util.CollectionUtils; +import org.apache.ignite.internal.util.ExceptionUtils; import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.PendingComparableValuesTracker; import org.apache.ignite.internal.utils.PrimaryReplica; @@ -252,13 +253,15 @@ public String name() { * @param tx The transaction, not null if explicit. * @param fac Replica requests factory. * @param noWriteChecker Used to handle operations producing no updates. + * @param retryOnLockConflict {@code True} to retry on lock conflict. * @return The future. */ private CompletableFuture enlistInTx( BinaryRowEx row, @Nullable InternalTransaction tx, IgniteTriFunction fac, - BiPredicate noWriteChecker + BiPredicate noWriteChecker, + boolean retryOnLockConflict ) { // Check whether proposed tx is read-only. Complete future exceptionally if true. // Attempting to enlist a read-only in a read-write transaction does not corrupt the transaction itself, thus read-write transaction @@ -287,10 +290,10 @@ private CompletableFuture enlistInTx( assert !implicit; fut = trackingInvoke(actualTx, partId, term -> fac.apply(actualTx, partGroupId, term), false, primaryReplicaAndTerm, - noWriteChecker); + noWriteChecker, retryOnLockConflict); } else { fut = enlistWithRetry(actualTx, partId, term -> fac.apply(actualTx, partGroupId, term), ATTEMPTS_TO_ENLIST_PARTITION, - implicit, noWriteChecker); + implicit, noWriteChecker, retryOnLockConflict); } return postEnlist(fut, false, actualTx, implicit); @@ -304,6 +307,7 @@ private CompletableFuture enlistInTx( * @param fac Replica requests factory. * @param reducer Transform reducer. * @param noOpChecker Used to handle no-op operations (producing no updates). + * @param retryOnLockConflict {@code True} to retry on lock conflict. * @return The future. */ private CompletableFuture enlistInTx( @@ -311,9 +315,10 @@ private CompletableFuture enlistInTx( @Nullable InternalTransaction tx, IgnitePentaFunction< Collection, InternalTransaction, ReplicationGroupId, Long, Boolean, ReplicaRequest - > fac, + > fac, Function, CompletableFuture> reducer, - BiPredicate noOpChecker + BiPredicate noOpChecker, + boolean retryOnLockConflict ) { // Check whether proposed tx is read-only. Complete future exceptionally if true. // Attempting to enlist a read-only in a read-write transaction does not corrupt the transaction itself, thus read-write transaction @@ -349,7 +354,7 @@ private CompletableFuture enlistInTx( assert !implicit; fut = trackingInvoke(actualTx, partitionId, term -> fac.apply(rowBatch.requestedRows, actualTx, partGroupId, term, false), - false, primaryReplicaAndTerm, noOpChecker); + false, primaryReplicaAndTerm, noOpChecker, retryOnLockConflict); } else { fut = enlistWithRetry( actualTx, @@ -357,7 +362,8 @@ private CompletableFuture enlistInTx( term -> fac.apply(rowBatch.requestedRows, actualTx, partGroupId, term, full), ATTEMPTS_TO_ENLIST_PARTITION, full, - noOpChecker + noOpChecker, + retryOnLockConflict ); } @@ -425,7 +431,7 @@ private CompletableFuture> enlistCursorInTx( ReadWriteScanRetrieveBatchReplicaRequest request = requestBuilder.term(primaryReplicaAndTerm.get2()).build(); fut = replicaSvc.invoke(primaryReplicaAndTerm.get1(), request); } else { - fut = enlistWithRetry(tx, partId, term -> requestBuilder.term(term).build(), ATTEMPTS_TO_ENLIST_PARTITION, false, null); + fut = enlistWithRetry(tx, partId, term -> requestBuilder.term(term).build(), ATTEMPTS_TO_ENLIST_PARTITION, false, null, false); } return postEnlist(fut, false, tx, false); @@ -451,6 +457,7 @@ private CompletableFuture> enlistCursorInTx( * @param attempts Number of attempts. * @param full {@code True} if is a full transaction. * @param noWriteChecker Used to handle operations producing no updates. + * @param retryOnLockConflict {@code True} to retry on lock conflict. * @return The future. */ private CompletableFuture enlistWithRetry( @@ -459,31 +466,28 @@ private CompletableFuture enlistWithRetry( Function mapFunc, int attempts, boolean full, - @Nullable BiPredicate noWriteChecker + @Nullable BiPredicate noWriteChecker, + boolean retryOnLockConflict ) { - CompletableFuture result = new CompletableFuture<>(); - - enlist(partId, tx).thenCompose( - primaryReplicaAndTerm -> trackingInvoke(tx, partId, mapFunc, full, primaryReplicaAndTerm, noWriteChecker)) + return (CompletableFuture) enlist(partId, tx).thenCompose( + primaryReplicaAndTerm -> trackingInvoke(tx, partId, mapFunc, full, primaryReplicaAndTerm, noWriteChecker, + retryOnLockConflict)) .handle((res0, e) -> { if (e != null) { + // We can safely retry indefinitely on deadlock prevention. + if (retryOnLockConflict && e.getCause() instanceof LockException) { + return enlistWithRetry(tx, partId, mapFunc, attempts, full, noWriteChecker, true); + } + if (e.getCause() instanceof PrimaryReplicaMissException && attempts > 0) { - return enlistWithRetry(tx, partId, mapFunc, attempts - 1, full, noWriteChecker).handle((r2, e2) -> { - if (e2 != null) { - return result.completeExceptionally(e2); - } else { - return result.complete(r2); - } - }); + return enlistWithRetry(tx, partId, mapFunc, attempts - 1, full, noWriteChecker, retryOnLockConflict); } - return result.completeExceptionally(e); + return failedFuture(e); } - return result.complete(res0); - }); - - return result; + return completedFuture(res0); + }).thenCompose(x -> x); } /** @@ -495,6 +499,7 @@ private CompletableFuture enlistWithRetry( * @param full {@code True} for a full transaction. * @param primaryReplicaAndTerm Replica and term. * @param noWriteChecker Used to handle operations producing no updates. + * @param retryOnLockConflict {@code True} to retry on lock conflics. * @return The future. */ private CompletableFuture trackingInvoke( @@ -503,7 +508,8 @@ private CompletableFuture trackingInvoke( Function mapFunc, boolean full, IgniteBiTuple primaryReplicaAndTerm, - @Nullable BiPredicate noWriteChecker + @Nullable BiPredicate noWriteChecker, + boolean retryOnLockConflict ) { ReplicaRequest request = mapFunc.apply(primaryReplicaAndTerm.get2()); @@ -535,6 +541,17 @@ private CompletableFuture trackingInvoke( } return res; + }).exceptionally(e -> { + if (retryOnLockConflict && e.getCause() instanceof LockException) { + txManager.removeInflight(tx.id()); // Will be retried. + } + + if (e.getCause() instanceof PrimaryReplicaMissException) { + txManager.removeInflight(tx.id()); // Will be retried. + } + + ExceptionUtils.sneakyThrow(e); + return null; // Unreachable. }); } else { return replicaSvc.invoke(primaryReplicaAndTerm.get1(), request); @@ -609,7 +626,7 @@ private CompletableFuture evaluateReadOnlyPrimaryNode( TimeUnit.SECONDS ); - CompletableFuture fut = primaryReplicaFuture.thenCompose(primaryReplica -> { + CompletableFuture fut = primaryReplicaFuture.thenCompose(primaryReplica -> { try { ClusterNode node = clusterNodeResolver.apply(primaryReplica.getLeaseholder()); @@ -660,7 +677,7 @@ private CompletableFuture evaluateReadOnlyPrimaryNode( TimeUnit.SECONDS ); - CompletableFuture fut = primaryReplicaFuture.thenCompose(primaryReplica -> { + CompletableFuture fut = primaryReplicaFuture.thenCompose(primaryReplica -> { try { ClusterNode node = clusterNodeResolver.apply(primaryReplica.getLeaseholder()); @@ -752,7 +769,8 @@ public CompletableFuture get(BinaryRowEx keyRow, InternalTransaction .timestampLong(clock.nowLong()) .full(tx == null) .build(), - (res, req) -> false + (res, req) -> false, + false ); } @@ -821,17 +839,18 @@ public CompletableFuture> getAll(Collection keyRows BinaryRowEx firstRow = keyRows.iterator().next(); return evaluateReadOnlyRecipientNode(partitionId(firstRow)) - .thenCompose(recipientNode -> getAll(keyRows, tx.readTimestamp(), recipientNode)); + .thenCompose(recipientNode -> getAll(keyRows, tx.readTimestamp(), recipientNode)); } return enlistInTx( keyRows, tx, - (keyRows0, txo, groupId, term, full) -> { - return readWriteMultiRowPkReplicaRequest(RW_GET_ALL, keyRows0, txo, groupId, term, full); - }, + (keyRows0, txo, groupId, term, full) -> { + return readWriteMultiRowPkReplicaRequest(RW_GET_ALL, keyRows0, txo, groupId, term, full); + }, InternalTableImpl::collectMultiRowsResponsesWithRestoreOrder, - (res, req) -> false + (res, req) -> false, + false ); } @@ -948,7 +967,8 @@ public CompletableFuture upsert(BinaryRowEx row, InternalTransaction tx) { .timestampLong(clock.nowLong()) .full(tx == null) .build(), - (res, req) -> false + (res, req) -> false, + false ); } @@ -960,7 +980,8 @@ public CompletableFuture upsertAll(Collection rows, InternalT tx, this::upsertAllInternal, RowBatch::allResultFutures, - (res, req) -> false + (res, req) -> false, + false ); } @@ -976,7 +997,8 @@ public CompletableFuture upsertAll(Collection rows, int parti term -> upsertAllInternal(rows, tx, partGroupId, term, true), ATTEMPTS_TO_ENLIST_PARTITION, true, - null + null, + true // Allow auto retries for data streamer. ); return postEnlist(fut, false, tx, true); // Will be committed in one RTT. @@ -999,7 +1021,8 @@ public CompletableFuture getAndUpsert(BinaryRowEx row, InternalTransa .timestampLong(clock.nowLong()) .full(tx == null) .build(), - (res, req) -> false + (res, req) -> false, + false ); } @@ -1020,7 +1043,8 @@ public CompletableFuture insert(BinaryRowEx row, InternalTransaction tx .timestampLong(clock.nowLong()) .full(tx == null) .build(), - (res, req) -> !res + (res, req) -> !res, + false ); } @@ -1043,7 +1067,8 @@ public CompletableFuture> insertAll(Collection rows // All values are null, this means nothing was deleted. return true; - } + }, + false ); } @@ -1087,7 +1112,8 @@ public CompletableFuture replace(BinaryRowEx row, InternalTransaction t .timestampLong(clock.nowLong()) .full(tx == null) .build(), - (res, req) -> !res + (res, req) -> !res, + false ); } @@ -1112,7 +1138,8 @@ public CompletableFuture replace(BinaryRowEx oldRow, BinaryRowEx newRow .timestampLong(clock.nowLong()) .full(tx == null) .build(), - (res, req) -> !res + (res, req) -> !res, + false ); } @@ -1133,7 +1160,8 @@ public CompletableFuture getAndReplace(BinaryRowEx row, InternalTrans .timestampLong(clock.nowLong()) .full(tx == null) .build(), - (res, req) -> res == null + (res, req) -> res == null, + false ); } @@ -1154,7 +1182,8 @@ public CompletableFuture delete(BinaryRowEx keyRow, InternalTransaction .timestampLong(clock.nowLong()) .full(tx == null) .build(), - (res, req) -> !res + (res, req) -> !res, + false ); } @@ -1175,7 +1204,8 @@ public CompletableFuture deleteExact(BinaryRowEx oldRow, InternalTransa .timestampLong(clock.nowLong()) .full(tx == null) .build(), - (res, req) -> !res + (res, req) -> !res, + false ); } @@ -1196,7 +1226,8 @@ public CompletableFuture getAndDelete(BinaryRowEx row, InternalTransa .timestampLong(clock.nowLong()) .full(tx == null) .build(), - (res, req) -> res == null + (res, req) -> res == null, + false ); } @@ -1219,7 +1250,8 @@ public CompletableFuture> deleteAll(Collection rows // All values are null, this means nothing was deleted. return true; - } + }, + false ); } @@ -1245,7 +1277,8 @@ public CompletableFuture> deleteAllExact( // All values are null, this means nothing was deleted. return true; - } + }, + false ); } diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java index cd414274f7a..16f3104e862 100644 --- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java +++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java @@ -34,6 +34,7 @@ import static org.mockito.Mockito.when; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Locale; import java.util.Map; @@ -284,7 +285,6 @@ public void beforeTest() { TEST_MV_PARTITION_STORAGE.clear(); LOCK_MANAGER.releaseAll(TRANSACTION_ID); - LOCK_MANAGER.recordedLocks().clear(); } /** Verifies the mode in which the lock was acquired on the index key for a particular operation. */ @@ -343,7 +343,7 @@ void testReadWriteSingle(ReadWriteTestArg arg) throws MarshallerException { await(fut); assertThat( - LOCK_MANAGER.recordedLocks(), + locks(), allOf( hasItem(lockThat( arg.expectedLockOnUniqueHash + " on unique hash index", @@ -418,14 +418,13 @@ void testReadWriteMulti(ReadWriteTestArg arg) throws MarshallerException { throw new AssertionError("Unexpected operation type: " + arg.type); } - LOCK_MANAGER.recordLocks(true); CompletableFuture fut = partitionReplicaListener.invoke(request, "local"); await(fut); for (BinaryRow row : rows) { assertThat( - LOCK_MANAGER.recordedLocks(), + locks(), allOf( hasItem(lockThat( arg.expectedLockOnUniqueHash + " on unique hash index", @@ -473,6 +472,18 @@ private static Iterable readWriteMultiTestArguments() { ); } + private List locks() { + List locks = new ArrayList<>(); + + Iterator it = LOCK_MANAGER.locks(TRANSACTION_ID); + + while (it.hasNext()) { + locks.add(it.next()); + } + + return locks; + } + private void insertRows(List> rows, UUID txId) { HybridTimestamp commitTs = CLOCK.now(); diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java index 03982109f6b..dcead8a4712 100644 --- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java +++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java @@ -510,10 +510,10 @@ public void testBatchReadPutConcurrently() throws InterruptedException { assertTrue(IgniteTestUtils.waitForCondition(() -> { boolean lockUpgraded = false; - for (Iterator it = txManager(accounts).lockManager().locks(tx1.id()); it.hasNext(); ) { - Waiter waiter = it.next(); + for (Iterator it = txManager(accounts).lockManager().locks(tx1.id()); it.hasNext(); ) { + Lock lock = it.next(); - lockUpgraded = waiter.intendedLockMode() == LockMode.X; + lockUpgraded = txManager(accounts).lockManager().waiter(lock.lockKey(), tx1.id()).intendedLockMode() == LockMode.X; if (lockUpgraded) { break; diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockKey.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockKey.java index 3ec60e170f7..0e68f51abb8 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockKey.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockKey.java @@ -17,7 +17,9 @@ package org.apache.ignite.internal.tx; +import java.nio.ByteBuffer; import org.apache.ignite.internal.tostring.S; +import org.apache.ignite.internal.util.HashUtils; /** Lock key. */ public class LockKey { @@ -73,6 +75,11 @@ public boolean equals(Object o) { @Override public int hashCode() { + if (key instanceof ByteBuffer) { + ByteBuffer key1 = (ByteBuffer) key; + return HashUtils.hash32(HashUtils.hash64(key1, 0, key1.capacity(), contextId.hashCode())); + } + int result = contextId != null ? contextId.hashCode() : 0; result = 31 * result + (key != null ? key.hashCode() : 0); return result; diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java index 266c2e9b33c..5f81ff133e5 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java @@ -59,7 +59,7 @@ public interface LockManager { * @param txId Transaction Id. * @return An iterator over a collection of locks. */ - Iterator locks(UUID txId); + Iterator locks(UUID txId); /** * Release all locks associated with a transaction. diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index 4d93f83b8e4..d0e79b603c3 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -50,6 +50,7 @@ import org.apache.ignite.internal.tx.LockMode; import org.apache.ignite.internal.tx.Waiter; import org.jetbrains.annotations.Nullable; +import org.jetbrains.annotations.TestOnly; /** * A {@link LockManager} implementation which stores lock queues in the heap. @@ -60,15 +61,27 @@ * *

Read lock can be upgraded to write lock (only available for the lowest read-locked entry of * the queue). + * + * Additionally limits the lock map size. */ public class HeapLockManager implements LockManager { /** * Table size. TODO make it configurable IGNITE-20694 */ - private static final int SLOTS = 131072; + public static final int SLOTS = 131072; //16536; + + /** + * Empty slots. + */ + private final ConcurrentLinkedQueue empty = new ConcurrentLinkedQueue<>(); /** - * Lock queues. + * Mapped slots. + */ + private final ConcurrentHashMap locks; + + /** + * Raw slots. */ private final LockState[] slots; @@ -77,7 +90,9 @@ public class HeapLockManager implements LockManager { */ private final DeadlockPreventionPolicy deadlockPreventionPolicy; - /** Executor that is used to fail waiters after timeout. */ + /** + * Executor that is used to fail waiters after timeout. + */ private final Executor delayedExecutor; /** @@ -99,30 +114,42 @@ public class HeapLockManager implements LockManager { * Constructor. */ public HeapLockManager() { - this(new WaitDieDeadlockPreventionPolicy(), SLOTS); + this(new WaitDieDeadlockPreventionPolicy(), SLOTS, SLOTS); } /** * Constructor. */ public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy) { - this(deadlockPreventionPolicy, SLOTS); + this(deadlockPreventionPolicy, SLOTS, SLOTS); } /** * Constructor. * * @param deadlockPreventionPolicy Deadlock prevention policy. + * @param maxSize Raw slots size. + * @param mapSize Lock map size. */ - public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy, int maxSize) { + public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy, int maxSize, int mapSize) { + if (mapSize > maxSize) { + throw new IllegalArgumentException("maxSize=" + maxSize + " < mapSize=" + mapSize); + } + this.deadlockPreventionPolicy = deadlockPreventionPolicy; this.delayedExecutor = deadlockPreventionPolicy.waitTimeout() > 0 ? CompletableFuture.delayedExecutor(deadlockPreventionPolicy.waitTimeout(), TimeUnit.MILLISECONDS) : null; + locks = new ConcurrentHashMap<>(mapSize); + LockState[] tmp = new LockState[maxSize]; for (int i = 0; i < tmp.length; i++) { - tmp[i] = new LockState(); + LockState lockState = new LockState(); + if (i < mapSize) { + empty.add(lockState); + } + tmp[i] = lockState; } slots = tmp; // Atomic init. @@ -130,35 +157,51 @@ public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy, int ma @Override public CompletableFuture acquire(UUID txId, LockKey lockKey, LockMode lockMode) { - LockState state = lockState(lockKey); + while (true) { + LockState state = lockState(lockKey); - IgniteBiTuple, LockMode> futureTuple = state.tryAcquire(txId, lockMode); + IgniteBiTuple, LockMode> futureTuple = state.tryAcquire(txId, lockMode); - LockMode newLockMode = futureTuple.get2(); + if (futureTuple.get1() == null) { + continue; // State is marked for remove, need retry. + } - return futureTuple.get1().thenApply(res -> { - Lock lock = new Lock(lockKey, newLockMode, txId); + LockMode newLockMode = futureTuple.get2(); - if (record) { - recordedLocks.add(lock); - } + return futureTuple.get1().thenApply(res -> { + Lock lock = new Lock(lockKey, newLockMode, txId); - return lock; - }); + if (record) { + recordedLocks.add(lock); + } + + return lock; + }); + } } @Override public void release(Lock lock) { LockState state = lockState(lock.lockKey()); - state.tryRelease(lock.txId()); + if (state.tryRelease(lock.txId())) { + if (locks.remove(lock.lockKey(), state)) { + state.reset(); + empty.add(state); + } + } } @Override public void release(UUID txId, LockKey lockKey, LockMode lockMode) { LockState state = lockState(lockKey); - state.tryRelease(txId, lockMode); + if (state.tryRelease(txId, lockMode)) { + if (locks.remove(lockKey, state)) { + state.reset(); + empty.add(state); + } + } } @Override @@ -167,23 +210,34 @@ public void releaseAll(UUID txId) { if (states != null) { for (LockState state : states) { - state.tryRelease(txId); + if (state.tryRelease(txId)) { + // Key may be null if a lock has bypassed the cache. + if (locks.remove(state.key, state)) { + state.reset(); + empty.add(state); + } + } } } } @Override - public Iterator locks(UUID txId) { + public Iterator locks(UUID txId) { ConcurrentLinkedQueue lockStates = txMap.get(txId); - List result = new ArrayList<>(); - if (lockStates != null) { - for (LockState lockState : lockStates) { - Waiter waiter = lockState.waiter(txId); + List result = new ArrayList<>(); - if (waiter != null) { - result.add(waiter); - } + for (LockState lockState : lockStates) { + Waiter waiter = lockState.waiter(txId); + + if (waiter != null) { + result.add( + new Lock( + lockState.key, + waiter.lockMode(), + txId + ) + ); } } @@ -198,7 +252,27 @@ public Iterator locks(UUID txId) { private LockState lockState(LockKey key) { int h = spread(key.hashCode()); int index = h & (slots.length - 1); - return slots[index]; + + LockState[] res = new LockState[1]; + + locks.compute(key, (k, v) -> { + if (v == null) { + if (empty.isEmpty()) { + res[0] = slots[index]; + } else { + v = empty.poll(); + v.key = key; + res[0] = v; + } + } else { + res[0] = v; + // assert v.waitersCount() == 1; + } + + return v; + }); + + return res[0]; } /** {@inheritDoc} */ @@ -227,10 +301,15 @@ public boolean isEmpty() { /** * A lock state. */ - protected class LockState { + public class LockState { /** Waiters. */ private final TreeMap waiters; + /** Marked for removal flag. */ + private volatile boolean markedForRemove = false; + + private LockKey key; + public LockState() { Comparator txComparator = deadlockPreventionPolicy.txIdComparator() != null ? deadlockPreventionPolicy.txIdComparator() : UUID::compareTo; @@ -249,6 +328,10 @@ public LockState() { WaiterImpl waiter = new WaiterImpl(txId, lockMode); synchronized (waiters) { + if (markedForRemove) { + return new IgniteBiTuple(null, lockMode); + } + // We always replace the previous waiter with the new one. If the previous waiter has lock intention then incomplete // lock future is copied to the new waiter. This guarantees that, if the previous waiter was locked concurrently, then // it doesn't have any lock intentions, and the future is not copied to the new waiter. Otherwise, if there is lock @@ -291,7 +374,9 @@ public LockState() { waiter.refuseIntent(); // Restore old lock. } else { // Lock granted, track. - track(waiter.txId); + if (prev == null) { + track(waiter.txId); + } } } @@ -301,6 +386,10 @@ public LockState() { return new IgniteBiTuple<>(waiter.fut, waiter.lockMode()); } + public synchronized int waitersCount() { + return waiters.size(); + } + /** * Checks current waiter. It can change the internal state of the waiter. * @@ -363,7 +452,7 @@ private LockException lockException(UUID txId, WaiterImpl conflictingWaiter) { * @param txId Transaction id. * @return {@code True} if the queue is empty. */ - void tryRelease(UUID txId) { + boolean tryRelease(UUID txId) { Collection toNotify; synchronized (waiters) { @@ -374,6 +463,8 @@ void tryRelease(UUID txId) { for (WaiterImpl waiter : toNotify) { waiter.notifyLocked(); } + + return markedForRemove; } /** @@ -383,7 +474,7 @@ void tryRelease(UUID txId) { * @param lockMode Lock mode. * @return If the value is true, no one waits of any lock of the key, false otherwise. */ - void tryRelease(UUID txId, LockMode lockMode) { + boolean tryRelease(UUID txId, LockMode lockMode) { List toNotify = Collections.emptyList(); synchronized (waiters) { WaiterImpl waiter = waiters.get(txId); @@ -406,6 +497,8 @@ void tryRelease(UUID txId, LockMode lockMode) { for (WaiterImpl waiter : toNotify) { waiter.notifyLocked(); } + + return markedForRemove; } /** @@ -418,6 +511,10 @@ private List release(UUID txId) { waiters.remove(txId); if (waiters.isEmpty()) { + if (key != null) { + markedForRemove = true; + } + return Collections.emptyList(); } @@ -538,6 +635,11 @@ private void track(UUID txId) { return v; }); } + + public void reset() { + key = null; + markedForRemove = false; + } } /** @@ -792,4 +894,13 @@ public List recordedLocks() { private static int spread(int h) { return (h ^ (h >>> 16)) & 0x7fffffff; } + + @TestOnly + public LockState[] getSlots() { + return slots; + } + + public int available() { + return empty.size(); + } } diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java index dc1e769c052..4325739949c 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java @@ -165,9 +165,6 @@ private CompletableFuture finishInternal(boolean commit) { ClusterNode recipientNode = nodeAndTerm.get1(); Long term = nodeAndTerm.get2(); - LOG.debug("Finish [recipientNode={}, term={} commit={}, txId={}, groups={}].", - recipientNode, term, commit, id(), enlistedGroups); - assert recipientNode != null; assert term != null; diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java index 422f5da91f4..1365587b809 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java @@ -305,6 +305,8 @@ public CompletableFuture finish( Map enlistedGroups, UUID txId ) { + LOG.debug("Finish [recipientNode={}, term={} commit={}, txId={}, groups={}].", recipientNode, term, commit, txId, enlistedGroups); + assert enlistedGroups != null; // Here we put finishing state meta into the local map, so that all concurrent operations trying to read tx state @@ -340,45 +342,45 @@ public CompletableFuture finish( commit ? verifyCommitTimestamp(enlistedGroups, commitTimestamp) : completedFuture(null); return verificationFuture.handle( - (unused, throwable) -> { - Collection replicationGroupIds = new HashSet<>(enlistedGroups.keySet()); + (unused, throwable) -> { + Collection replicationGroupIds = new HashSet<>(enlistedGroups.keySet()); - boolean verifiedCommit = throwable == null && commit; + boolean verifiedCommit = throwable == null && commit; - TxFinishReplicaRequest req = FACTORY.txFinishReplicaRequest() - .txId(txId) - .timestampLong(clock.nowLong()) - .groupId(commitPartition) - .groups(replicationGroupIds) - // In case of verification future failure transaction will be rolled back. - .commit(verifiedCommit) - .commitTimestampLong(hybridTimestampToLong(commitTimestamp)) - .term(term) - .build(); + TxFinishReplicaRequest req = FACTORY.txFinishReplicaRequest() + .txId(txId) + .timestampLong(clock.nowLong()) + .groupId(commitPartition) + .groups(replicationGroupIds) + // In case of verification future failure transaction will be rolled back. + .commit(verifiedCommit) + .commitTimestampLong(hybridTimestampToLong(commitTimestamp)) + .term(term) + .build(); - return replicaService.invoke(recipientNode, req).thenRun( - () -> { - updateTxMeta(txId, old -> { - if (isFinalState(old.txState())) { - finishingStateMeta.txFinishFuture().complete(old); + return replicaService.invoke(recipientNode, req).thenRun( + () -> { + updateTxMeta(txId, old -> { + if (isFinalState(old.txState())) { + finishingStateMeta.txFinishFuture().complete(old); - return old; - } + return old; + } - assert old instanceof TxStateMetaFinishing; + assert old instanceof TxStateMetaFinishing; - TxStateMeta finalTxStateMeta = coordinatorFinalTxStateMeta(verifiedCommit, commitTimestamp); + TxStateMeta finalTxStateMeta = coordinatorFinalTxStateMeta(verifiedCommit, commitTimestamp); - finishingStateMeta.txFinishFuture().complete(finalTxStateMeta); + finishingStateMeta.txFinishFuture().complete(finalTxStateMeta); - return finalTxStateMeta; - }); + return finalTxStateMeta; + }); - if (verifiedCommit) { - observableTimestampTracker.update(commitTimestamp); - } - }); - }) + if (verifiedCommit) { + observableTimestampTracker.update(commitTimestamp); + } + }); + }) .thenCompose(Function.identity()) // verification future is added in order to share proper exception with the client .thenCompose(r -> verificationFuture); From effb4b0c79a4b859db7a9f56c161ceb5e557f871 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Sun, 12 Nov 2023 12:26:46 +0300 Subject: [PATCH 11/40] IGNITE-17811 Fix NPE --- .../ignite/distributed/ItLockTableTest.java | 2 ++ .../replicator/PartitionReplicaListener.java | 2 +- .../ignite/internal/tx/LockManager.java | 18 +--------- .../internal/tx/impl/HeapLockManager.java | 35 +++++++------------ 4 files changed, 16 insertions(+), 41 deletions(-) diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java index d0d551ecc88..15cff3b7379 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -158,6 +158,8 @@ public void testCollision() { LOG.info("LockTable [emptySlots={} collisions={}]", empty, coll); + assertTrue(coll > 0); + List> finishFuts = new ArrayList<>(); for (Transaction txn : txns) { finishFuts.add(txn.commitAsync()); diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java index c979cfb7b26..a0617e5711d 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java @@ -1398,7 +1398,7 @@ private CompletableFuture cleanup( ) { // Avoid invoking async chain in raft threads. CompletableFuture[] futures = enlistedPartitions.stream() - .map(partitionId -> changeStateFuture.thenComposeAsync(ignored -> + .map(partitionId -> changeStateFuture.thenCompose(ignored -> cleanupWithRetry(commit, commitTimestamp, txId, partitionId, attemptsToCleanupReplica))) .toArray(size -> new CompletableFuture[size]); diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java index 5f81ff133e5..e005739dd79 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java @@ -19,7 +19,6 @@ import java.util.Collection; import java.util.Iterator; -import java.util.List; import java.util.UUID; import java.util.concurrent.CompletableFuture; import org.jetbrains.annotations.TestOnly; @@ -59,6 +58,7 @@ public interface LockManager { * @param txId Transaction Id. * @return An iterator over a collection of locks. */ + @TestOnly Iterator locks(UUID txId); /** @@ -94,20 +94,4 @@ public interface LockManager { */ @TestOnly boolean isEmpty(); - - /** - * Record acquired locks. - * - * @param mode Mode. - */ - @TestOnly - void recordLocks(boolean mode); - - /** - * Return recorded locks. Can clear collection to reset state. - * - * @return Recorded locks. - */ - @TestOnly - List recordedLocks(); } diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index d0e79b603c3..e599bbbf01e 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -17,13 +17,14 @@ package org.apache.ignite.internal.tx.impl; +import static java.util.Collections.emptyIterator; +import static java.util.Collections.emptyList; import static java.util.concurrent.CompletableFuture.completedFuture; import static org.apache.ignite.lang.ErrorGroups.Transactions.ACQUIRE_LOCK_ERR; import static org.apache.ignite.lang.ErrorGroups.Transactions.ACQUIRE_LOCK_TIMEOUT_ERR; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.EnumMap; import java.util.EnumSet; @@ -168,15 +169,7 @@ public CompletableFuture acquire(UUID txId, LockKey lockKey, LockMode lock LockMode newLockMode = futureTuple.get2(); - return futureTuple.get1().thenApply(res -> { - Lock lock = new Lock(lockKey, newLockMode, txId); - - if (record) { - recordedLocks.add(lock); - } - - return lock; - }); + return futureTuple.get1().thenApply(res -> new Lock(lockKey, newLockMode, txId)); } } @@ -225,6 +218,10 @@ public void releaseAll(UUID txId) { public Iterator locks(UUID txId) { ConcurrentLinkedQueue lockStates = txMap.get(txId); + if (lockStates == null) { + return emptyIterator(); + } + List result = new ArrayList<>(); for (LockState lockState : lockStates) { @@ -475,7 +472,7 @@ boolean tryRelease(UUID txId) { * @return If the value is true, no one waits of any lock of the key, false otherwise. */ boolean tryRelease(UUID txId, LockMode lockMode) { - List toNotify = Collections.emptyList(); + List toNotify = emptyList(); synchronized (waiters) { WaiterImpl waiter = waiters.get(txId); @@ -515,7 +512,7 @@ private List release(UUID txId) { markedForRemove = true; } - return Collections.emptyList(); + return emptyList(); } return unlockCompatibleWaiters(); @@ -528,7 +525,7 @@ private List release(UUID txId) { */ private List unlockCompatibleWaiters() { if (!deadlockPreventionPolicy.usePriority() && deadlockPreventionPolicy.waitTimeout() == 0) { - return Collections.emptyList(); + return emptyList(); } ArrayList toNotify = new ArrayList<>(); @@ -803,6 +800,8 @@ private void notifyLocked() { } else { assert lockMode != null; + // TODO FIXME complete async if waiters are in queue to prevent to prevent thread pool starvation. + // This method can be called from raft thread, for example. fut.complete(null); } } @@ -881,16 +880,6 @@ public String toString() { } } - @Override - public void recordLocks(boolean mode) { - record = mode; - } - - @Override - public List recordedLocks() { - return recordedLocks; - } - private static int spread(int h) { return (h ^ (h >>> 16)) & 0x7fffffff; } From 70e21b9e6317a19c6cc9afc6db86de2a6257bc72 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Mon, 13 Nov 2023 16:55:53 +0300 Subject: [PATCH 12/40] IGNITE-17811 Fix compilation --- .../replication/PartitionReplicaListenerIndexLockingTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java index 16f3104e862..2283011f4c4 100644 --- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java +++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java @@ -111,7 +111,6 @@ import org.hamcrest.Matcher; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; @@ -337,7 +336,6 @@ void testReadWriteSingle(ReadWriteTestArg arg) throws MarshallerException { throw new AssertionError("Unexpected operation type: " + arg.type); } - LOCK_MANAGER.recordLocks(true); CompletableFuture fut = partitionReplicaListener.invoke(request, "local"); await(fut); From 44111043d0e4624a0c1e3ebd337b9b1a315627af Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Tue, 14 Nov 2023 14:49:48 +0300 Subject: [PATCH 13/40] IGNITE-17811 Fix lock table and leak in tx test --- .../ignite/distributed/ItLockTableTest.java | 7 +- .../ItTxDistributedTestSingleNode.java | 9 ++- .../ignite/internal/table/TxAbstractTest.java | 2 + .../apache/ignite/internal/tx/LockKey.java | 3 +- .../internal/tx/impl/HeapLockManager.java | 71 +++++++++++-------- 5 files changed, 52 insertions(+), 40 deletions(-) diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java index 15cff3b7379..68a66bb1883 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -166,12 +166,7 @@ public void testCollision() { } for (CompletableFuture finishFut : finishFuts) { - try { - finishFut.join(); - } catch (Exception e) { - // Ignore. - System.out.println(); - } + finishFut.join(); } assertTrue(TestUtils.waitForCondition(() -> { diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java index 82bc20eb898..52d9621081e 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java @@ -75,10 +75,15 @@ public class ItTxDistributedTestSingleNode extends TxAbstractTest { //TODO fsync can be turned on again after https://issues.apache.org/jira/browse/IGNITE-20195 @InjectConfiguration("mock: { fsync: false }") - protected static RaftConfiguration raftConfiguration; + protected RaftConfiguration raftConfiguration; @InjectConfiguration - protected static GcConfiguration gcConfig; + protected GcConfiguration gcConfig; + + @AfterEach + public void cleanup() { + Mockito.framework().clearInlineMocks(); + } /** * Returns a count of nodes. diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java index dcead8a4712..eee9d313f3a 100644 --- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java +++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java @@ -83,9 +83,11 @@ import org.apache.ignite.tx.TransactionException; import org.apache.ignite.tx.TransactionOptions; import org.jetbrains.annotations.Nullable; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockKey.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockKey.java index 0e68f51abb8..9569c04459e 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockKey.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockKey.java @@ -75,9 +75,10 @@ public boolean equals(Object o) { @Override public int hashCode() { + // Apply more efficient hashing to byte buffers to decrease collisions if (key instanceof ByteBuffer) { ByteBuffer key1 = (ByteBuffer) key; - return HashUtils.hash32(HashUtils.hash64(key1, 0, key1.capacity(), contextId.hashCode())); + return HashUtils.hash32(HashUtils.hash64(key1, 0, key1.capacity(), contextId != null ? contextId.hashCode() : 0)); } int result = contextId != null ? contextId.hashCode() : 0; diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index e599bbbf01e..06391cd817b 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -57,12 +57,12 @@ * A {@link LockManager} implementation which stores lock queues in the heap. * *

Lock waiters are placed in the queue, ordered according to comparator provided by {@link HeapLockManager#deadlockPreventionPolicy}. - * When a new waiter is placed in the queue, it's validated against current lock owner: if there is an owner with a higher priority - * (as defined by comparator) lock request is denied. + * When a new waiter is placed in the queue, it's validated against current lock owner: if there is an owner with a higher priority (as + * defined by comparator) lock request is denied. * *

Read lock can be upgraded to write lock (only available for the lowest read-locked entry of * the queue). - * + *

* Additionally limits the lock map size. */ public class HeapLockManager implements LockManager { @@ -178,10 +178,17 @@ public void release(Lock lock) { LockState state = lockState(lock.lockKey()); if (state.tryRelease(lock.txId())) { - if (locks.remove(lock.lockKey(), state)) { - state.reset(); - empty.add(state); - } + locks.compute(lock.lockKey(), (k, v) -> { + // Mapping may already change. + if (v != state || !v.markedForRemove) { + return v; + } + + // markedForRemove state should be cleared on entry reuse to avoid race. + v.key = null; + empty.add(v); + return null; + }); } } @@ -190,10 +197,16 @@ public void release(UUID txId, LockKey lockKey, LockMode lockMode) { LockState state = lockState(lockKey); if (state.tryRelease(txId, lockMode)) { - if (locks.remove(lockKey, state)) { - state.reset(); - empty.add(state); - } + locks.compute(lockKey, (k, v) -> { + // Mapping may already change. + if (v != state || !v.markedForRemove) { + return v; + } + + v.key = null; + empty.add(v); + return null; + }); } } @@ -204,11 +217,16 @@ public void releaseAll(UUID txId) { if (states != null) { for (LockState state : states) { if (state.tryRelease(txId)) { - // Key may be null if a lock has bypassed the cache. - if (locks.remove(state.key, state)) { - state.reset(); - empty.add(state); - } + locks.compute(state.key, (k, v) -> { + // Mapping may already change. + if (v != state || !v.markedForRemove) { + return v; + } + + v.key = null; + empty.add(v); + return null; + }); } } } @@ -228,13 +246,7 @@ public Iterator locks(UUID txId) { Waiter waiter = lockState.waiter(txId); if (waiter != null) { - result.add( - new Lock( - lockState.key, - waiter.lockMode(), - txId - ) - ); + result.add(new Lock(lockState.key, waiter.lockMode(), txId)); } } @@ -258,7 +270,8 @@ private LockState lockState(LockKey key) { res[0] = slots[index]; } else { v = empty.poll(); - v.key = key; + v.markedForRemove = false; + v.key = k; res[0] = v; } } else { @@ -288,8 +301,9 @@ public Waiter waiter(LockKey key, UUID txId) { @Override public boolean isEmpty() { for (LockState slot : slots) { - if (!slot.waiters.isEmpty()) + if (!slot.waiters.isEmpty()) { return false; + } } return true; @@ -305,7 +319,7 @@ public class LockState { /** Marked for removal flag. */ private volatile boolean markedForRemove = false; - private LockKey key; + private volatile LockKey key; public LockState() { Comparator txComparator = @@ -632,11 +646,6 @@ private void track(UUID txId) { return v; }); } - - public void reset() { - key = null; - markedForRemove = false; - } } /** From a02185346931fec06c810147b31b7dc4cfc6d434 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Tue, 14 Nov 2023 16:26:11 +0300 Subject: [PATCH 14/40] IGNITE-17811 Fix NPE --- .../internal/tx/impl/HeapLockManager.java | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index 06391cd817b..b4d798efd7a 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -217,16 +217,19 @@ public void releaseAll(UUID txId) { if (states != null) { for (LockState state : states) { if (state.tryRelease(txId)) { - locks.compute(state.key, (k, v) -> { - // Mapping may already change. - if (v != state || !v.markedForRemove) { - return v; - } - - v.key = null; - empty.add(v); - return null; - }); + LockKey key = state.key; // State may be already invalidated. + if (key != null) { + locks.compute(key, (k, v) -> { + // Mapping may already change. + if (v != state || !v.markedForRemove) { + return v; + } + + v.key = null; + empty.add(v); + return null; + }); + } } } } From 65a608503d6697e9d1e2bc8fc71659f78688bd34 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Fri, 17 Nov 2023 13:04:23 +0300 Subject: [PATCH 15/40] IGNITE-17811 Fix test and formatting --- .../benchmark/AbstractOneNodeBenchmark.java | 13 +- .../benchmark/LockManagerBenchmark.java | 20 +- .../ignite/distributed/ItLockTableTest.java | 10 +- .../replicator/PartitionReplicaListener.java | 27 +- .../internal/tx/DeadlockPreventionPolicy.java | 5 + .../ignite/internal/tx/LockManager.java | 8 + .../internal/tx/impl/HeapLockManager.java | 39 +- .../tx/impl/HeapUnboundedLockManager.java | 744 ++++++++++++++++++ .../tx/impl/ReadWriteTransactionImpl.java | 6 - 9 files changed, 820 insertions(+), 52 deletions(-) create mode 100644 modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/AbstractOneNodeBenchmark.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/AbstractOneNodeBenchmark.java index f3c6334f68a..8874298b8ff 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/AbstractOneNodeBenchmark.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/AbstractOneNodeBenchmark.java @@ -24,7 +24,6 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.nio.file.Paths; import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.ignite.IgnitionManager; @@ -71,13 +70,13 @@ public final void nodeSetUp() throws IOException { @Language("HOCON") String config = "network: {\n" - + " nodeFinder:{\n" - + " netClusterNodes: [ \"localhost:" + PORT + "\"] \n" - + " }\n" - + "}," - + "raft.fsync = " + fsync; + + " nodeFinder:{\n" + + " netClusterNodes: [ \"localhost:" + PORT + "\"] \n" + + " }\n" + + "}," + + "raft.fsync = " + fsync; - var fut = TestIgnitionManager.start(NODE_NAME, config, workDir.resolve(NODE_NAME)); + var fut = TestIgnitionManager.start(NODE_NAME, config, workDir.resolve(NODE_NAME)); TestIgnitionManager.init(new InitParametersBuilder() .clusterName("cluster") diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java index b23887ef493..b63a0dc4aef 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java @@ -1,15 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.ignite.internal.benchmark; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import java.util.UUID; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import org.apache.ignite.internal.TestHybridClock; import org.apache.ignite.internal.hlc.HybridClock; import org.apache.ignite.internal.storage.RowId; -import org.apache.ignite.internal.tx.Lock; import org.apache.ignite.internal.tx.LockKey; import org.apache.ignite.internal.tx.LockManager; import org.apache.ignite.internal.tx.LockMode; diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java index 68a66bb1883..16b44aeb729 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -20,6 +20,7 @@ import org.apache.ignite.internal.testframework.IgniteAbstractTest; import org.apache.ignite.internal.tx.DeadlockPreventionPolicy; import org.apache.ignite.internal.tx.HybridTimestampTracker; +import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; import org.apache.ignite.internal.tx.impl.HeapLockManager; import org.apache.ignite.internal.tx.impl.HeapLockManager.LockState; import org.apache.ignite.internal.tx.impl.TransactionIdGenerator; @@ -96,8 +97,11 @@ protected TxManagerImpl newTxManager(ReplicaService replicaSvc, HybridClock cloc ClusterNode node, PlacementDriver placementDriver) { return new TxManagerImpl( replicaSvc, - new HeapLockManager(new DeadlockPreventionPolicy() { - }, HeapLockManager.SLOTS, CACHE_SIZE), + new HeapLockManager( + DeadlockPreventionPolicy.NO_OP, + HeapLockManager.SLOTS, + CACHE_SIZE, + new HeapUnboundedLockManager()), clock, generator, node::id, @@ -122,7 +126,7 @@ public void testCollision() { RecordView view = testTable.recordView(); int i = 0; - final int count = 10_00; + final int count = 1000; List txns = new ArrayList<>(); while (i++ < count) { Transaction tx = txTestCluster.igniteTransactions().begin(); diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java index a0617e5711d..c3cdc38b4fc 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java @@ -960,7 +960,7 @@ private CompletableFuture> processScanRetrieveBatchAction( IgniteUuid cursorId = new IgniteUuid(txId, request.scanId()); - return lockManager.acquire(txId, new LockKey(tableId()), LockMode.S) + return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.S) .thenCompose(tblLock -> retrieveExactEntriesUntilCursorEmpty(txId, cursorId, batchCount)); } @@ -1008,8 +1008,8 @@ private CompletableFuture> lookupIndex( BinaryTuple exactKey = request.exactKey().asBinaryTuple(); - return lockManager.acquire(txId, new LockKey(indexId), LockMode.IS).thenCompose(idxLock -> { // Index IS lock - return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IS).thenCompose(tblLock -> { // Table IS lock + return lockManager.parentLockManager().acquire(txId, new LockKey(indexId), LockMode.IS).thenCompose(idxLock -> { // Index IS lock + return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IS).thenCompose(tblLock -> { // Table IS lock return lockManager.acquire(txId, new LockKey(indexId, exactKey.byteBuffer()), LockMode.S) .thenCompose(indRowLock -> { // Hash index bucket S lock Cursor cursor = (Cursor) cursors.computeIfAbsent(cursorId, id -> indexStorage.get(exactKey)); @@ -1051,8 +1051,8 @@ private CompletableFuture> scanSortedIndex( int flags = request.flags(); - return lockManager.acquire(txId, new LockKey(indexId), LockMode.IS).thenCompose(idxLock -> { // Index IS lock - return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IS).thenCompose(tblLock -> { // Table IS lock + return lockManager.parentLockManager().acquire(txId, new LockKey(indexId), LockMode.IS).thenCompose(idxLock -> { // Index IS lock + return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IS).thenCompose(tblLock -> { // Table IS lock var comparator = new BinaryTupleComparator(indexStorage.indexDescriptor().columns()); Predicate isUpperBoundAchieved = indexRow -> { @@ -2905,7 +2905,7 @@ private Cursor getFromPkIndex(BinaryTuple key) { * @return Future completes with tuple {@link RowId} and collection of {@link Lock}. */ private CompletableFuture>> takeLocksForUpdate(BinaryRow binaryRow, RowId rowId, UUID txId) { - return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IX) + return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IX) .thenCompose(ignored -> lockManager.acquire(txId, new LockKey(tableId(), rowId), LockMode.X)) .thenCompose(ignored -> takePutLockOnIndexes(binaryRow, rowId, txId)) .thenApply(shortTermLocks -> new IgniteBiTuple<>(rowId, shortTermLocks)); @@ -2919,11 +2919,8 @@ private CompletableFuture>> takeLocksForUp * @return Future completes with tuple {@link RowId} and collection of {@link Lock}. */ private CompletableFuture>> takeLocksForInsert(BinaryRow binaryRow, RowId rowId, UUID txId) { -// return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table -// .thenCompose(ignored -> takePutLockOnIndexes(binaryRow, rowId, txId)) -// .thenApply(shortTermLocks -> new IgniteBiTuple<>(rowId, shortTermLocks)); - - return takePutLockOnIndexes(binaryRow, rowId, txId) + return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table + .thenCompose(ignored -> takePutLockOnIndexes(binaryRow, rowId, txId)) .thenApply(shortTermLocks -> new IgniteBiTuple<>(rowId, shortTermLocks)); } @@ -2980,7 +2977,7 @@ private CompletableFuture takeRemoveLockOnIndexes(BinaryRow binaryRow, RowId * @return Future completes with {@link RowId} or {@code null} if there is no value for remove. */ private CompletableFuture takeLocksForDeleteExact(BinaryRow expectedRow, RowId rowId, BinaryRow actualRow, UUID txId) { - return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table + return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table .thenCompose(ignored -> lockManager.acquire(txId, new LockKey(tableId(), rowId), LockMode.S)) // S lock on RowId .thenCompose(ignored -> { if (equalValues(actualRow, expectedRow)) { @@ -3000,7 +2997,7 @@ private CompletableFuture takeLocksForDeleteExact(BinaryRow expectedRow, * @return Future completes with {@link RowId} or {@code null} if there is no value for the key. */ private CompletableFuture takeLocksForDelete(BinaryRow binaryRow, RowId rowId, UUID txId) { - return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table + return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table .thenCompose(ignored -> lockManager.acquire(txId, new LockKey(tableId(), rowId), LockMode.X)) // X lock on RowId .thenCompose(ignored -> takeRemoveLockOnIndexes(binaryRow, rowId, txId)) .thenApply(ignored -> rowId); @@ -3013,7 +3010,7 @@ private CompletableFuture takeLocksForDelete(BinaryRow binaryRow, RowId r * @return Future completes with {@link RowId} or {@code null} if there is no value for the key. */ private CompletableFuture takeLocksForGet(RowId rowId, UUID txId) { - return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IS) // IS lock on table + return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IS) // IS lock on table .thenCompose(tblLock -> lockManager.acquire(txId, new LockKey(tableId(), rowId), LockMode.S)) // S lock on RowId .thenApply(ignored -> rowId); } @@ -3087,7 +3084,7 @@ private CompletableFuture processTwoEntriesAction( */ private CompletableFuture>> takeLocksForReplace(BinaryRow expectedRow, @Nullable BinaryRow oldRow, BinaryRow newRow, RowId rowId, UUID txId) { - return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IX) + return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IX) .thenCompose(ignored -> lockManager.acquire(txId, new LockKey(tableId(), rowId), LockMode.S)) .thenCompose(ignored -> { if (oldRow != null && equalValues(oldRow, expectedRow)) { diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/DeadlockPreventionPolicy.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/DeadlockPreventionPolicy.java index eaf376860e0..7b5b02ce316 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/DeadlockPreventionPolicy.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/DeadlockPreventionPolicy.java @@ -27,6 +27,11 @@ * See also {@link org.apache.ignite.internal.tx.impl.HeapLockManager}. */ public interface DeadlockPreventionPolicy { + /** + * No-op policy which does nothing to prevent deadlocks. + */ + DeadlockPreventionPolicy NO_OP = new DeadlockPreventionPolicy() {}; + /** * Comparator for transaction ids that allows to set transaction priority, if deadlock prevention policy requires this priority. * The transaction with higher id has lower priority. If this comparator is {@code null} then behavior of any transaction diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java index e005739dd79..03703bd8862 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java @@ -21,6 +21,7 @@ import java.util.Iterator; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import org.jetbrains.annotations.Nullable; import org.jetbrains.annotations.TestOnly; /** Lock manager allows to acquire locks and release locks and supports deadlock prevention by transaction id ordering. */ @@ -94,4 +95,11 @@ public interface LockManager { */ @TestOnly boolean isEmpty(); + + /** + * Get parent lock manager. + * @return Parent lock manager in lock hierarchy. + */ + @Nullable + LockManager parentLockManager(); } diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index b4d798efd7a..0b01c73a74d 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -32,6 +32,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.TreeMap; import java.util.UUID; @@ -102,27 +103,22 @@ public class HeapLockManager implements LockManager { private final ConcurrentHashMap> txMap = new ConcurrentHashMap<>(1024); /** - * {@code True} to record acquired locks + * Parent lock manager. */ - private boolean record; - - /** - * Recorded locks. - */ - private final List recordedLocks = new ArrayList<>(); + private final LockManager parentLockManager; /** * Constructor. */ public HeapLockManager() { - this(new WaitDieDeadlockPreventionPolicy(), SLOTS, SLOTS); + this(new WaitDieDeadlockPreventionPolicy(), SLOTS, SLOTS, new HeapUnboundedLockManager()); } /** * Constructor. */ public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy) { - this(deadlockPreventionPolicy, SLOTS, SLOTS); + this(deadlockPreventionPolicy, SLOTS, SLOTS, new HeapUnboundedLockManager()); } /** @@ -132,11 +128,12 @@ public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy) { * @param maxSize Raw slots size. * @param mapSize Lock map size. */ - public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy, int maxSize, int mapSize) { + public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy, int maxSize, int mapSize, LockManager parentLockManager) { if (mapSize > maxSize) { throw new IllegalArgumentException("maxSize=" + maxSize + " < mapSize=" + mapSize); } + this.parentLockManager = Objects.requireNonNull(parentLockManager); this.deadlockPreventionPolicy = deadlockPreventionPolicy; this.delayedExecutor = deadlockPreventionPolicy.waitTimeout() > 0 ? CompletableFuture.delayedExecutor(deadlockPreventionPolicy.waitTimeout(), TimeUnit.MILLISECONDS) @@ -312,6 +309,12 @@ public boolean isEmpty() { return true; } + /** {@inheritDoc} */ + @Override + public @Nullable LockManager parentLockManager() { + return parentLockManager; + } + /** * A lock state. */ @@ -417,7 +420,7 @@ private boolean isWaiterReadyToNotify(WaiterImpl waiter, boolean skipFail) { if (mode != null && !mode.isCompatible(waiter.intendedLockMode())) { if (!deadlockPreventionPolicy.usePriority() && deadlockPreventionPolicy.waitTimeout() == 0) { - waiter.fail(lockException(waiter.txId(), tmp)); + waiter.fail(lockException(waiter, tmp)); return true; } @@ -434,7 +437,7 @@ private boolean isWaiterReadyToNotify(WaiterImpl waiter, boolean skipFail) { if (skipFail) { return false; } else if (deadlockPreventionPolicy.waitTimeout() == 0) { - waiter.fail(lockException(waiter.txId(), tmp)); + waiter.fail(lockException(waiter, tmp)); return true; } else { @@ -451,13 +454,13 @@ private boolean isWaiterReadyToNotify(WaiterImpl waiter, boolean skipFail) { /** * Create lock exception with given parameters. * - * @param txId Transaction id. - * @param conflictingWaiter Conflicting waiter. + * @param locker Locker. + * @param holder Lock holder. * @return Lock exception. */ - private LockException lockException(UUID txId, WaiterImpl conflictingWaiter) { - return new LockException(ACQUIRE_LOCK_ERR, "Failed to acquire a lock due to a conflict [txId=" - + txId + ", conflictingWaiter=" + conflictingWaiter + ']'); + private LockException lockException(WaiterImpl locker, WaiterImpl holder) { + return new LockException(ACQUIRE_LOCK_ERR, + "Failed to acquire a lock due to a conflict [locker=" + locker + ", holder=" + holder + ']'); } /** @@ -888,7 +891,7 @@ public int hashCode() { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(WaiterImpl.class, this, "isDone", fut.isDone()); + return S.toString(WaiterImpl.class, this, "granted", fut.isDone()); } } diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java new file mode 100644 index 00000000000..70c0068b5d7 --- /dev/null +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java @@ -0,0 +1,744 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.tx.impl; + +import static java.util.concurrent.CompletableFuture.completedFuture; +import static org.apache.ignite.lang.ErrorGroups.Transactions.ACQUIRE_LOCK_ERR; +import static org.apache.ignite.lang.ErrorGroups.Transactions.ACQUIRE_LOCK_TIMEOUT_ERR; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.internal.lang.IgniteBiTuple; +import org.apache.ignite.internal.tostring.IgniteToStringExclude; +import org.apache.ignite.internal.tostring.S; +import org.apache.ignite.internal.tx.DeadlockPreventionPolicy; +import org.apache.ignite.internal.tx.Lock; +import org.apache.ignite.internal.tx.LockException; +import org.apache.ignite.internal.tx.LockKey; +import org.apache.ignite.internal.tx.LockManager; +import org.apache.ignite.internal.tx.LockMode; +import org.apache.ignite.internal.tx.Waiter; +import org.jetbrains.annotations.Nullable; + +/** + * A {@link LockManager} which uses unbounded hashtable implementation. + * + * Suitable for holding coarse-grained locks. + */ +public class HeapUnboundedLockManager implements LockManager { + /** Locks. */ + private final ConcurrentHashMap locks = new ConcurrentHashMap<>(); + + /** Prevention policy. */ + private final DeadlockPreventionPolicy deadlockPreventionPolicy; + + /** Executor that is used to fail waiters after timeout. */ + private final Executor delayedExecutor; + + /** + * Constructor. + */ + public HeapUnboundedLockManager() { + this(new WaitDieDeadlockPreventionPolicy()); + } + + /** + * Constructor. + * + * @param deadlockPreventionPolicy Deadlock prevention policy. + */ + public HeapUnboundedLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy) { + this.deadlockPreventionPolicy = deadlockPreventionPolicy; + this.delayedExecutor = deadlockPreventionPolicy.waitTimeout() > 0 + ? CompletableFuture.delayedExecutor(deadlockPreventionPolicy.waitTimeout(), TimeUnit.MILLISECONDS) + : null; + } + + @Override + public CompletableFuture acquire(UUID txId, LockKey lockKey, LockMode lockMode) { + while (true) { + LockState state = lockState(lockKey); + + IgniteBiTuple, LockMode> futureTuple = state.tryAcquire(txId, lockMode); + + if (futureTuple.get1() == null) { + continue; // Obsolete state. + } + + LockMode newLockMode = futureTuple.get2(); + + return futureTuple.get1().thenApply(res -> new Lock(lockKey, newLockMode, txId)); + } + } + + @Override + public void release(Lock lock) { + LockState state = lockState(lock.lockKey()); + + if (state.tryRelease(lock.txId())) { + locks.remove(lock.lockKey(), state); + } + } + + @Override + public void release(UUID txId, LockKey lockKey, LockMode lockMode) { + LockState state = lockState(lockKey); + + if (state.tryRelease(txId, lockMode)) { + locks.remove(lockKey, state); + } + } + + @Override + public void releaseAll(UUID txId) { + Iterator locks = locks(txId); + + while (locks.hasNext()) { + Lock lock = locks.next(); + release(lock); + } + } + + @Override + public Iterator locks(UUID txId) { + // Decently ok to do full scan here because coarse-grained locks are a few in quantity. + List result = new ArrayList<>(); + + for (Map.Entry entry : locks.entrySet()) { + Waiter waiter = entry.getValue().waiter(txId); + + if (waiter != null) { + result.add( + new Lock( + entry.getKey(), + waiter.lockMode(), + txId + ) + ); + } + } + + return result.iterator(); + } + + /** + * Returns the lock state for the key. + * + * @param key The key. + */ + private LockState lockState(LockKey key) { + return locks.computeIfAbsent(key, k -> new LockState(deadlockPreventionPolicy, delayedExecutor)); + } + + /** {@inheritDoc} */ + @Override + public Collection queue(LockKey key) { + return lockState(key).queue(); + } + + /** {@inheritDoc} */ + @Override + public Waiter waiter(LockKey key, UUID txId) { + return lockState(key).waiter(txId); + } + + /** + * A lock state. + */ + private static class LockState { + /** Waiters. */ + private final TreeMap waiters; + + private final DeadlockPreventionPolicy deadlockPreventionPolicy; + + /** Delayed executor for waiters timeout callback. */ + private final Executor delayedExecutor; + + /** Marked for removal flag. */ + private boolean markedForRemove = false; + + public LockState(DeadlockPreventionPolicy deadlockPreventionPolicy, Executor delayedExecutor) { + Comparator txComparator = + deadlockPreventionPolicy.txIdComparator() != null ? deadlockPreventionPolicy.txIdComparator() : UUID::compareTo; + + this.waiters = new TreeMap<>(txComparator); + this.deadlockPreventionPolicy = deadlockPreventionPolicy; + this.delayedExecutor = delayedExecutor; + } + + /** + * Attempts to acquire a lock for the specified {@code key} in specified lock mode. + * + * @param txId Transaction id. + * @param lockMode Lock mode. + * @return The future or null if state is marked for removal and acquired lock mode. + */ + public @Nullable IgniteBiTuple, LockMode> tryAcquire(UUID txId, LockMode lockMode) { + WaiterImpl waiter = new WaiterImpl(txId, lockMode); + + synchronized (waiters) { + if (markedForRemove) { + return new IgniteBiTuple(null, lockMode); + } + + // We always replace the previous waiter with the new one. If the previous waiter has lock intention then incomplete + // lock future is copied to the new waiter. This guarantees that, if the previous waiter was locked concurrently, then + // it doesn't have any lock intentions, and the future is not copied to the new waiter. Otherwise, if there is lock + // intention, this means that the lock future contained in previous waiter, is not going to be completed and can be + // copied safely. + WaiterImpl prev = waiters.put(txId, waiter); + + // Reenter + if (prev != null) { + if (prev.locked() && prev.lockMode().allowReenter(lockMode)) { + waiter.lock(); + + waiter.upgrade(prev); + + return new IgniteBiTuple(completedFuture(null), prev.lockMode()); + } else { + waiter.upgrade(prev); + + assert prev.lockMode() == waiter.lockMode() : + "Lock modes are incorrect [prev=" + prev.lockMode() + ", new=" + waiter.lockMode() + ']'; + } + } + + if (!isWaiterReadyToNotify(waiter, false)) { + if (deadlockPreventionPolicy.waitTimeout() > 0) { + setWaiterTimeout(waiter); + } + + return new IgniteBiTuple<>(waiter.fut, waiter.lockMode()); + } + + if (!waiter.locked()) { + waiters.remove(waiter.txId()); + } else if (waiter.hasLockIntent()) { + waiter.refuseIntent(); // Restore old lock. + } + } + + // Notify outside the monitor. + waiter.notifyLocked(); + + return new IgniteBiTuple(waiter.fut, waiter.lockMode()); + } + + /** + * Checks current waiter. It can change the internal state of the waiter. + * + * @param waiter Checked waiter. + * @return True if current waiter ready to notify, false otherwise. + */ + private boolean isWaiterReadyToNotify(WaiterImpl waiter, boolean skipFail) { + for (Map.Entry entry : waiters.tailMap(waiter.txId(), false).entrySet()) { + WaiterImpl tmp = entry.getValue(); + LockMode mode = lockedMode(tmp); + + if (mode != null && !mode.isCompatible(waiter.intendedLockMode())) { + if (!deadlockPreventionPolicy.usePriority() && deadlockPreventionPolicy.waitTimeout() == 0) { + waiter.fail(lockException(waiter.txId(), tmp)); + + return true; + } + + return false; + } + } + + for (Map.Entry entry : waiters.headMap(waiter.txId()).entrySet()) { + WaiterImpl tmp = entry.getValue(); + LockMode mode = lockedMode(tmp); + + if (mode != null && !mode.isCompatible(waiter.intendedLockMode())) { + if (skipFail) { + return false; + } else if (deadlockPreventionPolicy.waitTimeout() == 0) { + waiter.fail(lockException(waiter.txId(), tmp)); + + return true; + } else { + return false; + } + } + } + + waiter.lock(); + + return true; + } + + /** + * Create lock exception with given parameters. + * + * @param txId Transaction id. + * @param conflictingWaiter Conflicting waiter. + * @return Lock exception. + */ + private LockException lockException(UUID txId, WaiterImpl conflictingWaiter) { + return new LockException(ACQUIRE_LOCK_ERR, "Failed to acquire a lock due to a conflict [txId=" + + txId + ", conflictingWaiter=" + conflictingWaiter + ']'); + } + + /** + * Attempts to release a lock for the specified {@code key} in exclusive mode. + * + * @param txId Transaction id. + * @return {@code True} if the queue is empty. + */ + public boolean tryRelease(UUID txId) { + Collection toNotify; + + synchronized (waiters) { + toNotify = release(txId); + } + + // Notify outside the monitor. + for (WaiterImpl waiter : toNotify) { + waiter.notifyLocked(); + } + + return markedForRemove; + } + + /** + * Releases a specific lock of the key. + * + * @param txId Transaction id. + * @param lockMode Lock mode. + * @return If the value is true, no one waits of any lock of the key, false otherwise. + */ + public boolean tryRelease(UUID txId, LockMode lockMode) { + List toNotify = Collections.emptyList(); + synchronized (waiters) { + WaiterImpl waiter = waiters.get(txId); + + if (waiter != null) { + assert lockMode.supremum(lockMode, waiter.lockMode()) == waiter.lockMode() : + "The lock mode is not locked [mode=" + lockMode + ", locked=" + waiter.lockMode() + ']'; + + LockMode modeFromDowngrade = waiter.recalculateMode(lockMode); + + if (!waiter.locked() && !waiter.hasLockIntent()) { + toNotify = release(txId); + } else if (modeFromDowngrade != waiter.lockMode()) { + toNotify = unlockCompatibleWaiters(); + } + } + } + + // Notify outside the monitor. + for (WaiterImpl waiter : toNotify) { + waiter.notifyLocked(); + } + + return markedForRemove; + } + + /** + * Releases all locks are held by a specific transaction. + * This method should be invoked synchronously. + * + * @param txId Transaction id. + * @return List of waiters to notify. + */ + private List release(UUID txId) { + waiters.remove(txId); + + if (waiters.isEmpty()) { + markedForRemove = true; + + return Collections.emptyList(); + } + + List toNotify = unlockCompatibleWaiters(); + + return toNotify; + } + + /** + * Unlock compatible waiters. + * + * @return List of waiters to notify. + */ + private List unlockCompatibleWaiters() { + if (!deadlockPreventionPolicy.usePriority() && deadlockPreventionPolicy.waitTimeout() == 0) { + return Collections.emptyList(); + } + + ArrayList toNotify = new ArrayList<>(); + Set toFail = new HashSet<>(); + + for (Map.Entry entry : waiters.entrySet()) { + WaiterImpl tmp = entry.getValue(); + + if (tmp.hasLockIntent() && isWaiterReadyToNotify(tmp, true)) { + assert !tmp.hasLockIntent() : "This waiter in not locked for notification [waiter=" + tmp + ']'; + + toNotify.add(tmp); + } + } + + if (deadlockPreventionPolicy.usePriority() && deadlockPreventionPolicy.waitTimeout() >= 0) { + for (Map.Entry entry : waiters.entrySet()) { + WaiterImpl tmp = entry.getValue(); + + if (tmp.hasLockIntent() && isWaiterReadyToNotify(tmp, false)) { + assert tmp.hasLockIntent() : "Only failed waiter can be notified here [waiter=" + tmp + ']'; + + toNotify.add(tmp); + toFail.add(tmp.txId()); + } + } + + for (UUID failTx : toFail) { + var w = waiters.get(failTx); + + if (w.locked()) { + w.refuseIntent(); + } else { + waiters.remove(failTx); + } + } + } + + return toNotify; + } + + /** + * Makes the waiter fail after specified timeout (in milliseconds), if intended lock was not acquired within this timeout. + * + * @param waiter Waiter. + */ + private void setWaiterTimeout(WaiterImpl waiter) { + delayedExecutor.execute(() -> { + if (!waiter.fut.isDone()) { + waiter.fut.completeExceptionally(new LockException(ACQUIRE_LOCK_TIMEOUT_ERR, "Failed to acquire a lock due to " + + "timeout [txId=" + waiter.txId() + ", waiter=" + waiter + + ", timeout=" + deadlockPreventionPolicy.waitTimeout() + ']')); + } + }); + } + + /** + * Gets a lock mode for this waiter. + * + * @param waiter Waiter. + * @return Lock mode, which is held by the waiter or {@code null}, if the waiter holds nothing. + */ + private LockMode lockedMode(WaiterImpl waiter) { + LockMode mode = null; + + if (waiter.locked()) { + mode = waiter.lockMode(); + } + + return mode; + } + + /** + * Returns a collection of timestamps that is associated with the specified {@code key}. + * + * @return The waiters queue. + */ + public Collection queue() { + synchronized (waiters) { + return new ArrayList<>(waiters.keySet()); + } + } + + /** + * Returns a waiter for the specified {@code key}. + * + * @param txId Transaction id. + * @return The waiter. + */ + public Waiter waiter(UUID txId) { + synchronized (waiters) { + return waiters.get(txId); + } + } + } + + /** + * A waiter implementation. + */ + private static class WaiterImpl implements Comparable, Waiter { + /** + * Holding locks by type. + * TODO: IGNITE-18350 Abandon the collection in favor of BitSet. + */ + private final Map locks = new HashMap<>(); + + /** + * Lock modes are marked as intended, but have not taken yet. This is NOT specific to intention lock modes, such as IS and IX. + * TODO: IGNITE-18350 Abandon the collection in favor of BitSet. + */ + private final Set intendedLocks = new HashSet<>(); + + /** Locked future. */ + @IgniteToStringExclude + private CompletableFuture fut; + + /** Waiter transaction id. */ + private final UUID txId; + + /** The lock mode to intend to hold. This is NOT specific to intention lock modes, such as IS and IX. */ + private LockMode intendedLockMode; + + /** The lock mode. */ + private LockMode lockMode; + + /** + * The filed has a value when the waiter couldn't lock a key. + */ + private LockException ex; + + /** + * The constructor. + * + * @param txId Transaction id. + * @param lockMode Lock mode. + */ + WaiterImpl(UUID txId, LockMode lockMode) { + this.fut = new CompletableFuture<>(); + this.txId = txId; + this.intendedLockMode = lockMode; + + locks.put(lockMode, 1); + intendedLocks.add(lockMode); + } + + /** + * Adds a lock mode. + * + * @param lockMode Lock mode. + * @param increment Value to increment amount. + */ + void addLock(LockMode lockMode, int increment) { + locks.merge(lockMode, increment, Integer::sum); + } + + /** + * Removes a lock mode. + * + * @param lockMode Lock mode. + * @return True if the lock mode was removed, false otherwise. + */ + private boolean removeLock(LockMode lockMode) { + Integer counter = locks.get(lockMode); + + if (counter == null || counter < 2) { + locks.remove(lockMode); + + return true; + } else { + locks.put(lockMode, counter - 1); + + return false; + } + } + + /** + * Recalculates lock mode based of all locks which the waiter has taken. + * + * @param modeToRemove Mode without which, the recalculation will happen. + * @return Previous lock mode. + */ + LockMode recalculateMode(LockMode modeToRemove) { + if (!removeLock(modeToRemove)) { + return lockMode; + } + + return recalculate(); + } + + /** + * Recalculates lock supremums. + * + * @return Previous lock mode. + */ + private LockMode recalculate() { + LockMode newIntendedLockMode = null; + LockMode newLockMode = null; + + for (LockMode mode : locks.keySet()) { + assert locks.get(mode) > 0 : "Incorrect lock counter [txId=" + txId + ", mode=" + mode + "]"; + + if (intendedLocks.contains(mode)) { + newIntendedLockMode = newIntendedLockMode == null ? mode : LockMode.supremum(newIntendedLockMode, mode); + } else { + newLockMode = newLockMode == null ? mode : LockMode.supremum(newLockMode, mode); + } + } + + LockMode mode = lockMode; + + lockMode = newLockMode; + intendedLockMode = newLockMode != null && newIntendedLockMode != null ? LockMode.supremum(newLockMode, newIntendedLockMode) + : newIntendedLockMode; + + return mode; + } + + /** + * Merge all locks that were held by another waiter to the current one. + * + * @param other Other waiter. + */ + void upgrade(WaiterImpl other) { + intendedLocks.addAll(other.intendedLocks); + + other.locks.entrySet().forEach(entry -> addLock(entry.getKey(), entry.getValue())); + + recalculate(); + + if (other.hasLockIntent()) { + fut = other.fut; + } + } + + /** + * Removes all locks that were intended to hold. + */ + void refuseIntent() { + for (LockMode mode : intendedLocks) { + locks.remove(mode); + } + + intendedLocks.clear(); + intendedLockMode = null; + } + + /** {@inheritDoc} */ + @Override + public int compareTo(WaiterImpl o) { + return txId.compareTo(o.txId); + } + + /** Notifies a future listeners. */ + private void notifyLocked() { + if (ex != null) { + fut.completeExceptionally(ex); + } else { + assert lockMode != null; + + fut.complete(null); + } + } + + /** {@inheritDoc} */ + @Override + public boolean locked() { + return this.lockMode != null; + } + + /** + * Checks is the waiter has any intended to lock a key. + * + * @return True if the waiter has an intended lock, false otherwise. + */ + public boolean hasLockIntent() { + return this.intendedLockMode != null; + } + + /** {@inheritDoc} */ + @Override + public LockMode lockMode() { + return lockMode; + } + + /** {@inheritDoc} */ + @Override + public LockMode intendedLockMode() { + return intendedLockMode; + } + + /** Grant a lock. */ + private void lock() { + lockMode = intendedLockMode; + + intendedLockMode = null; + + intendedLocks.clear(); + } + + /** + * Fails the lock waiter. + * + * @param e Lock exception. + */ + private void fail(LockException e) { + ex = e; + } + + /** {@inheritDoc} */ + @Override + public UUID txId() { + return txId; + } + + /** {@inheritDoc} */ + @Override + public boolean equals(Object o) { + if (!(o instanceof WaiterImpl)) { + return false; + } + + return compareTo((WaiterImpl) o) == 0; + } + + /** {@inheritDoc} */ + @Override + public int hashCode() { + return txId.hashCode(); + } + + /** {@inheritDoc} */ + @Override + public String toString() { + return S.toString(WaiterImpl.class, this, "isDone", fut.isDone()); + } + } + + /** {@inheritDoc} */ + @Override + public boolean isEmpty() { + return locks.isEmpty(); + } + + @Override + public LockManager parentLockManager() { + return null; + } +} diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java index 4325739949c..498d0f195ba 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java @@ -32,11 +32,8 @@ import java.util.stream.Collectors; import org.apache.ignite.internal.hlc.HybridTimestamp; import org.apache.ignite.internal.lang.IgniteBiTuple; -import org.apache.ignite.internal.logger.IgniteLogger; -import org.apache.ignite.internal.logger.Loggers; import org.apache.ignite.internal.replicator.TablePartitionId; import org.apache.ignite.internal.tx.HybridTimestampTracker; -import org.apache.ignite.internal.tx.InternalTransaction; import org.apache.ignite.internal.tx.TransactionIds; import org.apache.ignite.internal.tx.TxManager; import org.apache.ignite.network.ClusterNode; @@ -46,9 +43,6 @@ * The read-write implementation of an internal transaction. */ public class ReadWriteTransactionImpl extends IgniteAbstractTransactionImpl { - /** The logger. */ - private static final IgniteLogger LOG = Loggers.forClass(InternalTransaction.class); - /** Commit partition updater. */ private static final AtomicReferenceFieldUpdater COMMIT_PART_UPDATER = AtomicReferenceFieldUpdater.newUpdater(ReadWriteTransactionImpl.class, TablePartitionId.class, "commitPart"); From 459597b054be3d1330c6acc536c854e4d0dfe2c2 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Fri, 17 Nov 2023 14:28:53 +0300 Subject: [PATCH 16/40] IGNITE-17811 Fix unlock issue --- .../internal/benchmark/LockManagerBenchmark.java | 6 ++++++ .../ignite/distributed/ItLockTableTest.java | 6 ++++++ .../ignite/internal/tx/impl/HeapLockManager.java | 5 ++++- .../tx/impl/HeapUnboundedLockManager.java | 16 ++++++++-------- 4 files changed, 24 insertions(+), 9 deletions(-) diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java index b63a0dc4aef..1680ff2c34f 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/LockManagerBenchmark.java @@ -44,6 +44,9 @@ import org.openjdk.jmh.runner.options.Options; import org.openjdk.jmh.runner.options.OptionsBuilder; +/** + * Benchmark lock manager. + */ @State(Scope.Benchmark) @OutputTimeUnit(TimeUnit.MICROSECONDS) public class LockManagerBenchmark { @@ -77,6 +80,9 @@ public void tearDown() throws Exception { @Param({"200"}) private int concTxns; + /** + * Take and release some locks. + */ @Benchmark @Warmup(iterations = 1, time = 3) @Measurement(iterations = 1, time = 10) diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java index 16b44aeb729..2722bce7590 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -37,6 +37,9 @@ import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; +/** + * Test lock table. + */ @ExtendWith(ConfigurationExtension.class) public class ItLockTableTest extends IgniteAbstractTest { private static final IgniteLogger LOG = Loggers.forClass(ItLockTableTest.class); @@ -121,6 +124,9 @@ public void after() throws Exception { txTestCluster.shutdownCluster(); } + /** + * Test that a lock table behaves correctly in case of lock cache overflow. + */ @Test public void testCollision() { RecordView view = testTable.recordView(); diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index 0b01c73a74d..a117bb96e81 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -171,6 +171,7 @@ public CompletableFuture acquire(UUID txId, LockKey lockKey, LockMode lock } @Override + @TestOnly public void release(Lock lock) { LockState state = lockState(lock.lockKey()); @@ -230,6 +231,8 @@ public void releaseAll(UUID txId) { } } } + + parentLockManager.releaseAll(txId); } @Override @@ -460,7 +463,7 @@ private boolean isWaiterReadyToNotify(WaiterImpl waiter, boolean skipFail) { */ private LockException lockException(WaiterImpl locker, WaiterImpl holder) { return new LockException(ACQUIRE_LOCK_ERR, - "Failed to acquire a lock due to a conflict [locker=" + locker + ", holder=" + holder + ']'); + "Failed to acquire a lock due to a possible deadlock [locker=" + locker + ", holder=" + holder + ']'); } /** diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java index 70c0068b5d7..099d67c4954 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java @@ -184,7 +184,7 @@ private static class LockState { private final Executor delayedExecutor; /** Marked for removal flag. */ - private boolean markedForRemove = false; + private volatile boolean markedForRemove = false; public LockState(DeadlockPreventionPolicy deadlockPreventionPolicy, Executor delayedExecutor) { Comparator txComparator = @@ -267,7 +267,7 @@ private boolean isWaiterReadyToNotify(WaiterImpl waiter, boolean skipFail) { if (mode != null && !mode.isCompatible(waiter.intendedLockMode())) { if (!deadlockPreventionPolicy.usePriority() && deadlockPreventionPolicy.waitTimeout() == 0) { - waiter.fail(lockException(waiter.txId(), tmp)); + waiter.fail(lockException(waiter, tmp)); return true; } @@ -284,7 +284,7 @@ private boolean isWaiterReadyToNotify(WaiterImpl waiter, boolean skipFail) { if (skipFail) { return false; } else if (deadlockPreventionPolicy.waitTimeout() == 0) { - waiter.fail(lockException(waiter.txId(), tmp)); + waiter.fail(lockException(waiter, tmp)); return true; } else { @@ -301,13 +301,13 @@ private boolean isWaiterReadyToNotify(WaiterImpl waiter, boolean skipFail) { /** * Create lock exception with given parameters. * - * @param txId Transaction id. - * @param conflictingWaiter Conflicting waiter. + * @param locker Locker. + * @param holder Lock holder. * @return Lock exception. */ - private LockException lockException(UUID txId, WaiterImpl conflictingWaiter) { - return new LockException(ACQUIRE_LOCK_ERR, "Failed to acquire a lock due to a conflict [txId=" - + txId + ", conflictingWaiter=" + conflictingWaiter + ']'); + private LockException lockException(Waiter locker, Waiter holder) { + return new LockException(ACQUIRE_LOCK_ERR, + "Failed to acquire a lock due to a possible deadlock [locker=" + locker + ", holder=" + holder + ']'); } /** From 1b9a0fcdae66bac78b8758760a1cdb13635e2c52 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Mon, 20 Nov 2023 12:14:29 +0300 Subject: [PATCH 17/40] IGNITE-17811 Fix formatting --- .../ignite/distributed/ItLockTableTest.java | 48 +++++++++++ .../replicator/PartitionReplicaListener.java | 80 ++++++++++--------- 2 files changed, 90 insertions(+), 38 deletions(-) diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java index 2722bce7590..a4ab443dd8b 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -1,5 +1,6 @@ package org.apache.ignite.distributed; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.ArrayList; @@ -20,6 +21,7 @@ import org.apache.ignite.internal.testframework.IgniteAbstractTest; import org.apache.ignite.internal.tx.DeadlockPreventionPolicy; import org.apache.ignite.internal.tx.HybridTimestampTracker; +import org.apache.ignite.internal.tx.InternalTransaction; import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; import org.apache.ignite.internal.tx.impl.HeapLockManager; import org.apache.ignite.internal.tx.impl.HeapLockManager.LockState; @@ -124,6 +126,47 @@ public void after() throws Exception { txTestCluster.shutdownCluster(); } + @Test + public void testDeadlockRecovery() { + RecordView view = testTable.recordView(); + Tuple t1 = tuple(0, "0"); + assertTrue(view.insert(null, t1)); + + Tuple t2 = tuple(1, "1"); + assertTrue(view.insert(null, t2)); + + InternalTransaction tx1 = (InternalTransaction) txTestCluster.igniteTransactions().begin(); + InternalTransaction tx2 = (InternalTransaction) txTestCluster.igniteTransactions().begin(); + + LOG.info("id1={}", tx1.id()); + LOG.info("id2={}", tx2.id()); + + assertTrue(tx2.id().compareTo(tx1.id()) > 0); + + Tuple r1_0 = view.get(tx1, keyTuple(0)); + Tuple r2_1 = view.get(tx2, keyTuple(1)); + + assertEquals(t1.stringValue("name"), r1_0.stringValue("name")); + assertEquals(t2.stringValue("name"), r2_1.stringValue("name")); + + view.upsertAsync(tx1, tuple(1, "11")); + view.upsertAsync(tx2, tuple(0, "00")); + + assertTrue(TestUtils.waitForCondition(() -> { + int total = 0; + HeapLockManager lockManager = (HeapLockManager) txTestCluster.txManagers.get(txTestCluster.localNodeName).lockManager(); + for (int j = 0; j < lockManager.getSlots().length; j++) { + LockState slot = lockManager.getSlots()[j]; + + total += slot.waitersCount(); + } + + return total == 8; + }, 10_000), "Some lockers are missing"); + + tx1.commit(); + } + /** * Test that a lock table behaves correctly in case of lock cache overflow. */ @@ -196,4 +239,9 @@ private static Tuple tuple(int id, String name) { .set("id", id) .set("name", name); } + + private static Tuple keyTuple(int id) { + return Tuple.create() + .set("id", id); + } } diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java index c3cdc38b4fc..f61c7c59c7e 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java @@ -1009,17 +1009,19 @@ private CompletableFuture> lookupIndex( BinaryTuple exactKey = request.exactKey().asBinaryTuple(); return lockManager.parentLockManager().acquire(txId, new LockKey(indexId), LockMode.IS).thenCompose(idxLock -> { // Index IS lock - return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IS).thenCompose(tblLock -> { // Table IS lock - return lockManager.acquire(txId, new LockKey(indexId, exactKey.byteBuffer()), LockMode.S) - .thenCompose(indRowLock -> { // Hash index bucket S lock - Cursor cursor = (Cursor) cursors.computeIfAbsent(cursorId, id -> indexStorage.get(exactKey)); + return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IS) // Table IS lock + .thenCompose(tblLock -> { + return lockManager.acquire(txId, new LockKey(indexId, exactKey.byteBuffer()), LockMode.S) + .thenCompose(indRowLock -> { // Hash index bucket S lock + Cursor cursor = (Cursor) cursors.computeIfAbsent(cursorId, + id -> indexStorage.get(exactKey)); - var result = new ArrayList(batchCount); + var result = new ArrayList(batchCount); - return continueIndexLookup(txId, cursor, batchCount, result) - .thenApply(ignore -> result); - }); - }); + return continueIndexLookup(txId, cursor, batchCount, result) + .thenApply(ignore -> result); + }); + }); }); } @@ -1052,45 +1054,47 @@ private CompletableFuture> scanSortedIndex( int flags = request.flags(); return lockManager.parentLockManager().acquire(txId, new LockKey(indexId), LockMode.IS).thenCompose(idxLock -> { // Index IS lock - return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IS).thenCompose(tblLock -> { // Table IS lock - var comparator = new BinaryTupleComparator(indexStorage.indexDescriptor().columns()); + return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IS) // Table IS lock + .thenCompose(tblLock -> { + var comparator = new BinaryTupleComparator(indexStorage.indexDescriptor().columns()); - Predicate isUpperBoundAchieved = indexRow -> { - if (indexRow == null) { - return true; - } + Predicate isUpperBoundAchieved = indexRow -> { + if (indexRow == null) { + return true; + } - if (upperBound == null) { - return false; - } + if (upperBound == null) { + return false; + } - ByteBuffer buffer = upperBound.byteBuffer(); + ByteBuffer buffer = upperBound.byteBuffer(); - if ((flags & SortedIndexStorage.LESS_OR_EQUAL) != 0) { - byte boundFlags = buffer.get(0); + if ((flags & SortedIndexStorage.LESS_OR_EQUAL) != 0) { + byte boundFlags = buffer.get(0); - buffer.put(0, (byte) (boundFlags | BinaryTupleCommon.EQUALITY_FLAG)); - } + buffer.put(0, (byte) (boundFlags | BinaryTupleCommon.EQUALITY_FLAG)); + } - return comparator.compare(indexRow.indexColumns().byteBuffer(), buffer) >= 0; - }; + return comparator.compare(indexRow.indexColumns().byteBuffer(), buffer) >= 0; + }; - Cursor cursor = (Cursor) cursors.computeIfAbsent(cursorId, - id -> indexStorage.scan( - lowerBound, - // We have to handle upperBound on a level of replication listener, - // for correctness of taking of a range lock. - null, - flags - )); + Cursor cursor = (Cursor) cursors.computeIfAbsent(cursorId, + id -> indexStorage.scan( + lowerBound, + // We have to handle upperBound on a level of replication listener, + // for correctness of taking of a range lock. + null, + flags + )); - SortedIndexLocker indexLocker = (SortedIndexLocker) indexesLockers.get().get(indexId); + SortedIndexLocker indexLocker = (SortedIndexLocker) indexesLockers.get().get(indexId); - var result = new ArrayList(batchCount); + var result = new ArrayList(batchCount); - return continueIndexScan(txId, schemaAwareIndexStorage, indexLocker, cursor, batchCount, result, isUpperBoundAchieved) - .thenApply(ignore -> result); - }); + return continueIndexScan(txId, schemaAwareIndexStorage, indexLocker, cursor, batchCount, result, + isUpperBoundAchieved) + .thenApply(ignore -> result); + }); }); } From 34b2aa831e2480bc38b88eef3b77cb0d1905c345 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Mon, 20 Nov 2023 14:17:11 +0300 Subject: [PATCH 18/40] IGNITE-17811 Merge with master --- .../org/apache/ignite/distributed/ItLockTableTest.java | 8 +++++--- .../internal/table/impl/DummyInternalTableImpl.java | 6 +++--- .../apache/ignite/internal/tx/impl/HeapLockManager.java | 1 + .../org/apache/ignite/internal/tx/impl/TxManagerImpl.java | 2 +- 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java index a4ab443dd8b..f78a70d03b6 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -17,14 +17,14 @@ import org.apache.ignite.internal.schema.Column; import org.apache.ignite.internal.schema.SchemaDescriptor; import org.apache.ignite.internal.schema.configuration.GcConfiguration; -import org.apache.ignite.internal.table.TableImpl; +import org.apache.ignite.internal.table.TableViewInternal; import org.apache.ignite.internal.testframework.IgniteAbstractTest; import org.apache.ignite.internal.tx.DeadlockPreventionPolicy; import org.apache.ignite.internal.tx.HybridTimestampTracker; import org.apache.ignite.internal.tx.InternalTransaction; -import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; import org.apache.ignite.internal.tx.impl.HeapLockManager; import org.apache.ignite.internal.tx.impl.HeapLockManager.LockState; +import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; import org.apache.ignite.internal.tx.impl.TransactionIdGenerator; import org.apache.ignite.internal.tx.impl.TxManagerImpl; import org.apache.ignite.internal.type.NativeTypes; @@ -35,6 +35,7 @@ import org.apache.ignite.tx.Transaction; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; @@ -61,7 +62,7 @@ public class ItLockTableTest extends IgniteAbstractTest { } ); - protected TableImpl testTable; + protected TableViewInternal testTable; protected final TestInfo testInfo; @@ -127,6 +128,7 @@ public void after() throws Exception { } @Test + @Disabled("https://issues.apache.org/jira/browse/IGNITE-20894") public void testDeadlockRecovery() { RecordView view = testTable.recordView(); Tuple t1 = tuple(0, "0"); diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java index 07a9d2e6bfc..1f2bdb30282 100644 --- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java +++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java @@ -161,7 +161,7 @@ public DummyInternalTableImpl(ReplicaService replicaSvc) { * @param schema Schema. */ public DummyInternalTableImpl(ReplicaService replicaSvc, SchemaDescriptor schema) { - this(replicaSvc, new TestMvPartitionStorage(0), schema, new TestPlacementDriver(LOCAL_NODE.name())); + this(replicaSvc, new TestMvPartitionStorage(0), schema, new TestPlacementDriver(LOCAL_NODE)); } /** @@ -172,7 +172,7 @@ public DummyInternalTableImpl(ReplicaService replicaSvc, SchemaDescriptor schema * @param schema Schema. */ public DummyInternalTableImpl(ReplicaService replicaSvc, MvPartitionStorage mvPartStorage, SchemaDescriptor schema) { - this(replicaSvc, mvPartStorage, schema, new TestPlacementDriver(LOCAL_NODE.name())); + this(replicaSvc, mvPartStorage, schema, new TestPlacementDriver(LOCAL_NODE)); } @@ -382,7 +382,7 @@ public void result(@Nullable Serializable r) { LOCAL_NODE, new AlwaysSyncedSchemaSyncService(), catalogService, - new TestPlacementDriver(LOCAL_NODE.name()) + new TestPlacementDriver(LOCAL_NODE) ); partitionListener = new PartitionListener( diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index a117bb96e81..c06d42381ea 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -104,6 +104,7 @@ public class HeapLockManager implements LockManager { /** * Parent lock manager. + * TODO asch Needs optimization https://issues.apache.org/jira/browse/IGNITE-20895 */ private final LockManager parentLockManager; diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java index fc00f43790b..491d041cf6f 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java @@ -334,7 +334,7 @@ public CompletableFuture finish( Map enlistedGroups, UUID txId ) { - LOG.debug("Finish [recipientNode={}, term={} commit={}, txId={}, groups={}].", recipientNode, term, commit, txId, enlistedGroups); + LOG.debug("Finish [commit={}, txId={}, groups={}].", commit, txId, enlistedGroups); assert enlistedGroups != null; From c48662a7c12f6825c4ab02e4e8f2d43edbbaa17e Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Mon, 20 Nov 2023 17:05:39 +0300 Subject: [PATCH 19/40] IGNITE-17811 Fix test --- modules/platforms/cpp/tests/odbc-test/transaction_test.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/platforms/cpp/tests/odbc-test/transaction_test.cpp b/modules/platforms/cpp/tests/odbc-test/transaction_test.cpp index 7b9a4052627..16dace0f3e5 100644 --- a/modules/platforms/cpp/tests/odbc-test/transaction_test.cpp +++ b/modules/platforms/cpp/tests/odbc-test/transaction_test.cpp @@ -575,7 +575,7 @@ TEST_F(transaction_test, transaction_error) { try { insert_test_value(conn2.m_statement, 2, "test_2"); } catch (const odbc_exception &err) { - EXPECT_THAT(err.message, testing::HasSubstr("Failed to acquire a lock due to a conflict")); + EXPECT_THAT(err.message, testing::HasSubstr("Failed to acquire a lock due to a possible deadlock")); // TODO: IGNITE-19944 Propagate SQL errors from engine to driver EXPECT_EQ(err.sql_state, "HY000"); throw; From fc4a79ca5a225088eb26cacfab27fdb00b29661d Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Mon, 20 Nov 2023 17:45:05 +0300 Subject: [PATCH 20/40] IGNITE-17811 Fix javadoc --- .../java/org/apache/ignite/internal/tx/LockManager.java | 1 + .../apache/ignite/internal/tx/impl/HeapLockManager.java | 9 +++++---- .../internal/tx/impl/HeapUnboundedLockManager.java | 4 +--- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java index 03703bd8862..aec75f75fa6 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java @@ -98,6 +98,7 @@ public interface LockManager { /** * Get parent lock manager. + * * @return Parent lock manager in lock hierarchy. */ @Nullable diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index c06d42381ea..1a22c6503ae 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -63,8 +63,8 @@ * *

Read lock can be upgraded to write lock (only available for the lowest read-locked entry of * the queue). - *

- * Additionally limits the lock map size. + * + *

Additionally limits the lock map size. */ public class HeapLockManager implements LockManager { /** @@ -329,9 +329,10 @@ public class LockState { /** Marked for removal flag. */ private volatile boolean markedForRemove = false; + /** Lock key. */ private volatile LockKey key; - public LockState() { + LockState() { Comparator txComparator = deadlockPreventionPolicy.txIdComparator() != null ? deadlockPreventionPolicy.txIdComparator() : UUID::compareTo; @@ -345,7 +346,7 @@ public LockState() { * @param lockMode Lock mode. * @return The future or null if state is marked for removal and acquired lock mode. */ - public @Nullable IgniteBiTuple, LockMode> tryAcquire(UUID txId, LockMode lockMode) { + @Nullable IgniteBiTuple, LockMode> tryAcquire(UUID txId, LockMode lockMode) { WaiterImpl waiter = new WaiterImpl(txId, lockMode); synchronized (waiters) { diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java index 099d67c4954..629cc0b2525 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java @@ -50,9 +50,7 @@ import org.jetbrains.annotations.Nullable; /** - * A {@link LockManager} which uses unbounded hashtable implementation. - * - * Suitable for holding coarse-grained locks. + * A {@link LockManager} which uses unbounded hashtable implementation. Suitable for holding coarse-grained locks. */ public class HeapUnboundedLockManager implements LockManager { /** Locks. */ From 9532861d0363c2be93dd351d70cf7adb3d463cb3 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Thu, 23 Nov 2023 17:50:48 +0300 Subject: [PATCH 21/40] IGNITE-17811 Add test for unbounded heap lock manager, fix compilation --- .../ignite/distributed/ItLockTableTest.java | 17 +++++++--- .../tx/HeapUnboundedLockManagerTest.java | 31 +++++++++++++++++++ 2 files changed, 44 insertions(+), 4 deletions(-) create mode 100644 modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapUnboundedLockManagerTest.java diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java index f78a70d03b6..fe835c016e5 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -1,5 +1,6 @@ package org.apache.ignite.distributed; +import static org.apache.ignite.internal.replicator.ReplicaManager.DEFAULT_IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -29,6 +30,7 @@ import org.apache.ignite.internal.tx.impl.TxManagerImpl; import org.apache.ignite.internal.type.NativeTypes; import org.apache.ignite.network.ClusterNode; +import org.apache.ignite.network.ClusterService; import org.apache.ignite.raft.jraft.test.TestUtils; import org.apache.ignite.table.RecordView; import org.apache.ignite.table.Tuple; @@ -99,9 +101,16 @@ public void before() throws Exception { timestampTracker ) { @Override - protected TxManagerImpl newTxManager(ReplicaService replicaSvc, HybridClock clock, TransactionIdGenerator generator, - ClusterNode node, PlacementDriver placementDriver) { + protected TxManagerImpl newTxManager( + ClusterService clusterService, + ReplicaService replicaSvc, + HybridClock clock, + TransactionIdGenerator generator, + ClusterNode node, + PlacementDriver placementDriver + ) { return new TxManagerImpl( + clusterService, replicaSvc, new HeapLockManager( DeadlockPreventionPolicy.NO_OP, @@ -110,8 +119,8 @@ protected TxManagerImpl newTxManager(ReplicaService replicaSvc, HybridClock cloc new HeapUnboundedLockManager()), clock, generator, - node::id, - placementDriver + placementDriver, + () -> DEFAULT_IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS ); } }; diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapUnboundedLockManagerTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapUnboundedLockManagerTest.java new file mode 100644 index 00000000000..258d0160d91 --- /dev/null +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapUnboundedLockManagerTest.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.tx; + +import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; +import org.apache.ignite.internal.tx.impl.WaitDieDeadlockPreventionPolicy; + +/** + * Test class for {@link HeapUnboundedLockManager}. + */ +public class HeapUnboundedLockManagerTest extends AbstractLockManagerTest { + @Override + protected LockManager newInstance() { + return new HeapUnboundedLockManager(new WaitDieDeadlockPreventionPolicy()); + } +} From e5ad5518d1d2fef962f9e042d4c476156b30d6d4 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Tue, 28 Nov 2023 15:26:01 +0300 Subject: [PATCH 22/40] IGNITE-17811 Review fixes --- .../internal/table/RecordBinaryViewImpl.java | 2 -- .../replicator/PartitionReplicaListener.java | 22 +++++++++---------- .../ignite/internal/table/TxAbstractTest.java | 3 --- .../ignite/internal/tx/LockManager.java | 8 ------- .../internal/tx/impl/HeapLockManager.java | 13 +++++------ ...NoWaitDeadlockPreventionUnboundedTest.java | 11 ++++++++++ .../NoneDeadlockPreventionUnboundedTest.java | 11 ++++++++++ ...versedDeadlockPreventionUnboundedTest.java | 11 ++++++++++ ...imeoutDeadlockPreventionUnboundedTest.java | 11 ++++++++++ 9 files changed, 60 insertions(+), 32 deletions(-) create mode 100644 modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java create mode 100644 modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java create mode 100644 modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java create mode 100644 modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java index 369b2ddf6e8..9cd62d57034 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java @@ -431,7 +431,6 @@ public CompletableFuture streamData(Publisher publisher, @Nullable var partitioner = new TupleStreamerPartitionAwarenessProvider(rowConverter.registry(), tbl.partitions()); StreamerBatchSender batchSender = (partitionId, items) -> withSchemaSync(null, (schemaVersion) -> { - System.out.println("BEGIN " + partitionId); return this.tbl.upsertAll(mapToBinary(items, schemaVersion, false), partitionId).thenAccept(new Consumer() { @Override public void accept(Void unused) { @@ -440,7 +439,6 @@ public void accept(Void unused) { } catch (InterruptedException e) { throw new RuntimeException(e); } - System.out.println("END " + partitionId); } }); }); diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java index e44fe43ac32..b19df03aff5 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java @@ -1078,7 +1078,7 @@ private CompletableFuture> processScanRetrieveBatchAction( IgniteUuid cursorId = new IgniteUuid(txId, request.scanId()); - return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.S) + return lockManager.acquire(txId, new LockKey(tableId()), LockMode.S) .thenCompose(tblLock -> retrieveExactEntriesUntilCursorEmpty(txId, cursorId, batchCount)); } @@ -1126,8 +1126,8 @@ private CompletableFuture> lookupIndex( BinaryTuple exactKey = request.exactKey().asBinaryTuple(); - return lockManager.parentLockManager().acquire(txId, new LockKey(indexId), LockMode.IS).thenCompose(idxLock -> { // Index IS lock - return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IS) // Table IS lock + return lockManager.acquire(txId, new LockKey(indexId), LockMode.IS).thenCompose(idxLock -> { // Index IS lock + return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IS) // Table IS lock .thenCompose(tblLock -> { return lockManager.acquire(txId, new LockKey(indexId, exactKey.byteBuffer()), LockMode.S) .thenCompose(indRowLock -> { // Hash index bucket S lock @@ -1171,8 +1171,8 @@ private CompletableFuture> scanSortedIndex( int flags = request.flags(); - return lockManager.parentLockManager().acquire(txId, new LockKey(indexId), LockMode.IS).thenCompose(idxLock -> { // Index IS lock - return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IS) // Table IS lock + return lockManager.acquire(txId, new LockKey(indexId), LockMode.IS).thenCompose(idxLock -> { // Index IS lock + return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IS) // Table IS lock .thenCompose(tblLock -> { var comparator = new BinaryTupleComparator(indexStorage.indexDescriptor().columns()); @@ -3157,7 +3157,7 @@ private Cursor getFromPkIndex(BinaryTuple key) { * @return Future completes with tuple {@link RowId} and collection of {@link Lock}. */ private CompletableFuture>> takeLocksForUpdate(BinaryRow binaryRow, RowId rowId, UUID txId) { - return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IX) + return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IX) .thenCompose(ignored -> lockManager.acquire(txId, new LockKey(tableId(), rowId), LockMode.X)) .thenCompose(ignored -> takePutLockOnIndexes(binaryRow, rowId, txId)) .thenApply(shortTermLocks -> new IgniteBiTuple<>(rowId, shortTermLocks)); @@ -3171,7 +3171,7 @@ private CompletableFuture>> takeLocksForUp * @return Future completes with tuple {@link RowId} and collection of {@link Lock}. */ private CompletableFuture>> takeLocksForInsert(BinaryRow binaryRow, RowId rowId, UUID txId) { - return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table + return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table .thenCompose(ignored -> takePutLockOnIndexes(binaryRow, rowId, txId)) .thenApply(shortTermLocks -> new IgniteBiTuple<>(rowId, shortTermLocks)); } @@ -3229,7 +3229,7 @@ private CompletableFuture takeRemoveLockOnIndexes(BinaryRow binaryRow, RowId * @return Future completes with {@link RowId} or {@code null} if there is no value for remove. */ private CompletableFuture takeLocksForDeleteExact(BinaryRow expectedRow, RowId rowId, BinaryRow actualRow, UUID txId) { - return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table + return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table .thenCompose(ignored -> lockManager.acquire(txId, new LockKey(tableId(), rowId), LockMode.S)) // S lock on RowId .thenCompose(ignored -> { if (equalValues(actualRow, expectedRow)) { @@ -3249,7 +3249,7 @@ private CompletableFuture takeLocksForDeleteExact(BinaryRow expectedRow, * @return Future completes with {@link RowId} or {@code null} if there is no value for the key. */ private CompletableFuture takeLocksForDelete(BinaryRow binaryRow, RowId rowId, UUID txId) { - return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table + return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IX) // IX lock on table .thenCompose(ignored -> lockManager.acquire(txId, new LockKey(tableId(), rowId), LockMode.X)) // X lock on RowId .thenCompose(ignored -> takeRemoveLockOnIndexes(binaryRow, rowId, txId)) .thenApply(ignored -> rowId); @@ -3262,7 +3262,7 @@ private CompletableFuture takeLocksForDelete(BinaryRow binaryRow, RowId r * @return Future completes with {@link RowId} or {@code null} if there is no value for the key. */ private CompletableFuture takeLocksForGet(RowId rowId, UUID txId) { - return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IS) // IS lock on table + return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IS) // IS lock on table .thenCompose(tblLock -> lockManager.acquire(txId, new LockKey(tableId(), rowId), LockMode.S)) // S lock on RowId .thenApply(ignored -> rowId); } @@ -3336,7 +3336,7 @@ private CompletableFuture processTwoEntriesAction( */ private CompletableFuture>> takeLocksForReplace(BinaryRow expectedRow, @Nullable BinaryRow oldRow, BinaryRow newRow, RowId rowId, UUID txId) { - return lockManager.parentLockManager().acquire(txId, new LockKey(tableId()), LockMode.IX) + return lockManager.acquire(txId, new LockKey(tableId()), LockMode.IX) .thenCompose(ignored -> lockManager.acquire(txId, new LockKey(tableId(), rowId), LockMode.S)) .thenCompose(ignored -> { if (oldRow != null && equalValues(oldRow, expectedRow)) { diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java index 05b59995e1d..4e2040f7b2d 100644 --- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java +++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java @@ -69,7 +69,6 @@ import org.apache.ignite.internal.tx.TxManager; import org.apache.ignite.internal.tx.TxState; import org.apache.ignite.internal.tx.TxStateMeta; -import org.apache.ignite.internal.tx.Waiter; import org.apache.ignite.internal.tx.impl.ReadWriteTransactionImpl; import org.apache.ignite.internal.type.NativeTypes; import org.apache.ignite.internal.util.CollectionUtils; @@ -83,11 +82,9 @@ import org.apache.ignite.tx.TransactionException; import org.apache.ignite.tx.TransactionOptions; import org.jetbrains.annotations.Nullable; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java index aec75f75fa6..92736f0ca86 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java @@ -95,12 +95,4 @@ public interface LockManager { */ @TestOnly boolean isEmpty(); - - /** - * Get parent lock manager. - * - * @return Parent lock manager in lock hierarchy. - */ - @Nullable - LockManager parentLockManager(); } diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index 1a22c6503ae..c0a93cd6cf0 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -156,6 +156,10 @@ public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy, int ma @Override public CompletableFuture acquire(UUID txId, LockKey lockKey, LockMode lockMode) { + if (lockKey.contextId() == null) { + return parentLockManager.acquire(txId, lockKey, lockMode); + } + while (true) { LockState state = lockState(lockKey); @@ -313,12 +317,6 @@ public boolean isEmpty() { return true; } - /** {@inheritDoc} */ - @Override - public @Nullable LockManager parentLockManager() { - return parentLockManager; - } - /** * A lock state. */ @@ -820,8 +818,7 @@ private void notifyLocked() { } else { assert lockMode != null; - // TODO FIXME complete async if waiters are in queue to prevent to prevent thread pool starvation. - // This method can be called from raft thread, for example. + // TODO FIXME https://issues.apache.org/jira/browse/IGNITE-20985 fut.complete(null); } } diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java new file mode 100644 index 00000000000..ea5c4e105fc --- /dev/null +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java @@ -0,0 +1,11 @@ +package org.apache.ignite.internal.tx; + +import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; +import org.apache.ignite.internal.tx.impl.WaitDieDeadlockPreventionPolicy; + +public class NoWaitDeadlockPreventionUnboundedTest extends NoWaitDeadlockPreventionTest { + @Override + protected LockManager lockManager() { + return new HeapUnboundedLockManager(new WaitDieDeadlockPreventionPolicy()); + } +} diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java new file mode 100644 index 00000000000..4ad538599bc --- /dev/null +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java @@ -0,0 +1,11 @@ +package org.apache.ignite.internal.tx; + +import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; +import org.apache.ignite.internal.tx.impl.WaitDieDeadlockPreventionPolicy; + +public class NoneDeadlockPreventionUnboundedTest extends NoneDeadlockPreventionTest { + @Override + protected LockManager lockManager() { + return new HeapUnboundedLockManager(new WaitDieDeadlockPreventionPolicy()); + } +} diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java new file mode 100644 index 00000000000..a0137e219ab --- /dev/null +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java @@ -0,0 +1,11 @@ +package org.apache.ignite.internal.tx; + +import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; +import org.apache.ignite.internal.tx.impl.WaitDieDeadlockPreventionPolicy; + +public class ReversedDeadlockPreventionUnboundedTest extends ReversedDeadlockPreventionTest { + @Override + protected LockManager lockManager() { + return new HeapUnboundedLockManager(new WaitDieDeadlockPreventionPolicy()); + } +} diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java new file mode 100644 index 00000000000..fe7fbab962c --- /dev/null +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java @@ -0,0 +1,11 @@ +package org.apache.ignite.internal.tx; + +import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; +import org.apache.ignite.internal.tx.impl.WaitDieDeadlockPreventionPolicy; + +public class TimeoutDeadlockPreventionUnboundedTest extends TimeoutDeadlockPreventionTest { + @Override + protected LockManager lockManager() { + return new HeapUnboundedLockManager(new WaitDieDeadlockPreventionPolicy()); + } +} From 6d087d82a7fda63583b7ef0e6591c5db55351ed4 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Tue, 28 Nov 2023 16:15:52 +0300 Subject: [PATCH 23/40] IGNITE-17811 Fix compilation --- .../ignite/internal/tx/impl/HeapUnboundedLockManager.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java index 629cc0b2525..48d5864c9a8 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java @@ -734,9 +734,4 @@ public String toString() { public boolean isEmpty() { return locks.isEmpty(); } - - @Override - public LockManager parentLockManager() { - return null; - } } From dcecaa076d108d8354ab82cddd3b034fffc95395 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Wed, 29 Nov 2023 12:22:35 +0300 Subject: [PATCH 24/40] IGNITE-17811 Fix compilation again --- .../java/org/apache/ignite/distributed/ItLockTableTest.java | 1 - .../org/apache/ignite/internal/tx/impl/HeapLockManager.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java index fe835c016e5..cda62da9f2b 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -93,7 +93,6 @@ public void before() throws Exception { txTestCluster = new ItTxTestCluster( testInfo, raftConfiguration, - gcConfig, workDir, 1, 1, diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index c0a93cd6cf0..f13c8b2791d 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -156,7 +156,7 @@ public HeapLockManager(DeadlockPreventionPolicy deadlockPreventionPolicy, int ma @Override public CompletableFuture acquire(UUID txId, LockKey lockKey, LockMode lockMode) { - if (lockKey.contextId() == null) { + if (lockKey.contextId() == null) { // Treat this lock as a hierarchy lock. return parentLockManager.acquire(txId, lockKey, lockMode); } From 6bf6764489fed24ff6f73262e80fa5636dfc31c5 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Wed, 29 Nov 2023 13:23:17 +0300 Subject: [PATCH 25/40] IGNITE-17811 Fix tests --- .../internal/tx/AbstractLockManagerTest.java | 72 ++++++++++--------- .../internal/tx/AbstractLockingTest.java | 2 +- .../internal/tx/HeapLockManagerTest.java | 5 ++ .../tx/HeapUnboundedLockManagerTest.java | 5 ++ 4 files changed, 48 insertions(+), 36 deletions(-) diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/AbstractLockManagerTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/AbstractLockManagerTest.java index 28f8c2194c9..9e4d4367a20 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/AbstractLockManagerTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/AbstractLockManagerTest.java @@ -63,11 +63,13 @@ public void before() { protected abstract LockManager newInstance(); + protected abstract LockKey lockKey(); + @Test public void testSingleKeyWrite() { UUID txId1 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); CompletableFuture fut0 = lockManager.acquire(txId1, key, X); @@ -89,7 +91,7 @@ public void testSingleKeyWriteLock() { UUID txId1 = TestTransactionIds.newTransactionId(); UUID txId2 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); CompletableFuture fut0 = lockManager.acquire(txId2, key, X); @@ -123,7 +125,7 @@ public void downgradeLockOutOfTurnTest() { UUID txId1 = TestTransactionIds.newTransactionId(); UUID txId2 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); lockManager.acquire(txId0, key, S).join(); Lock lock = lockManager.acquire(txId2, key, S).join(); @@ -149,7 +151,7 @@ public void upgradeLockImmediatelyTest() { UUID txId1 = TestTransactionIds.newTransactionId(); UUID txId2 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); CompletableFuture fut = lockManager.acquire(txId0, key, IS); assertTrue(fut.isDone()); @@ -175,7 +177,7 @@ public void testSingleKeyReadWriteLock() { assertTrue(txId3.compareTo(txId2) > 0); assertTrue(txId2.compareTo(txId1) > 0); assertTrue(txId1.compareTo(txId0) > 0); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); CompletableFuture fut3 = lockManager.acquire(txId3, key, S); assertTrue(fut3.isDone()); @@ -220,7 +222,7 @@ public void testSingleKeyReadWriteLock() { public void testSingleKeyReadWriteConflict() { UUID txId0 = TestTransactionIds.newTransactionId(); UUID txId1 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); // Lock in order CompletableFuture fut0 = lockManager.acquire(txId1, key, S); @@ -252,7 +254,7 @@ public void testSingleKeyReadWriteConflict() { @Test public void testSingleKeyReadWriteConflict2() { UUID[] txId = generate(3); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); // Lock in order CompletableFuture fut0 = lockManager.acquire(txId[1], key, S); @@ -275,7 +277,7 @@ public void testSingleKeyReadWriteConflict3() { UUID txId0 = TestTransactionIds.newTransactionId(); UUID txId1 = TestTransactionIds.newTransactionId(); UUID txId2 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); // Lock in order CompletableFuture fut0 = lockManager.acquire(txId1, key, S); @@ -301,7 +303,7 @@ public void testSingleKeyReadWriteConflict4() { final UUID txId2 = TestTransactionIds.newTransactionId(); UUID txId3 = TestTransactionIds.newTransactionId(); UUID txId4 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); CompletableFuture fut4 = lockManager.acquire(txId4, key, S); assertTrue(fut4.isDone()); @@ -320,7 +322,7 @@ public void testSingleKeyReadWriteConflict4() { public void testSingleKeyReadWriteConflict5() { UUID txId0 = TestTransactionIds.newTransactionId(); UUID txId1 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); lockManager.acquire(txId0, key, X).join(); @@ -332,7 +334,7 @@ public void testConflicts() { UUID txId0 = TestTransactionIds.newTransactionId(); UUID txId1 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); List> lockModes = new ArrayList<>(); @@ -375,7 +377,7 @@ public void testSingleKeyWriteWriteConflict() { UUID txId0 = TestTransactionIds.newTransactionId(); UUID txId1 = TestTransactionIds.newTransactionId(); UUID txId2 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); // Lock in order CompletableFuture fut0 = lockManager.acquire(txId1, key, X); @@ -398,7 +400,7 @@ public void testSingleKeyWriteWriteConflict2() { UUID txId0 = TestTransactionIds.newTransactionId(); UUID txId1 = TestTransactionIds.newTransactionId(); UUID txId2 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); // Lock in order CompletableFuture fut2 = lockManager.acquire(txId2, key, X); @@ -447,7 +449,7 @@ public void testSingleKeyMultithreadedRandom() throws InterruptedException { public void testLockUpgrade() { UUID txId0 = TestTransactionIds.newTransactionId(); UUID txId1 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); lockManager.acquire(txId0, key, S).join(); @@ -469,7 +471,7 @@ public void testLockUpgrade() { public void testLockUpgrade2() { UUID txId0 = TestTransactionIds.newTransactionId(); UUID txId1 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); lockManager.acquire(txId0, key, S).join(); @@ -483,7 +485,7 @@ public void testLockUpgrade3() { UUID txId0 = TestTransactionIds.newTransactionId(); UUID txId1 = TestTransactionIds.newTransactionId(); UUID txId2 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); lockManager.acquire(txId1, key, S).join(); @@ -504,7 +506,7 @@ public void testLockUpgrade3() { public void testLockUpgrade4() { UUID txId0 = TestTransactionIds.newTransactionId(); UUID txId1 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); lockManager.acquire(txId0, key, S).join(); @@ -525,7 +527,7 @@ public void testLockUpgrade4() { public void testLockUpgrade5() { UUID txId0 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); for (LockMode lockMode : List.of(IS, IX, SIX, X)) { lockManager.acquire(txId0, key, lockMode).join(); @@ -561,7 +563,7 @@ public void testLockUpgrade5() { @Test public void testReenter() { UUID txId = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); CompletableFuture fut = lockManager.acquire(txId, key, X); assertTrue(fut.isDone()); @@ -593,7 +595,7 @@ public void testAcquireReleasedLock() { UUID txId0 = TestTransactionIds.newTransactionId(); UUID txId1 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); List> lockModes = new ArrayList<>(); @@ -638,7 +640,7 @@ public void testCompatibleLockModes() { UUID txId0 = TestTransactionIds.newTransactionId(); UUID txId1 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); List> lockModes = new ArrayList<>(); @@ -673,7 +675,7 @@ public void testCompatibleLockModes() { public void testPossibleDowngradeLockModes() { UUID txId0 = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); for (LockMode lockMode : List.of(SIX, S, IS, IX)) { CompletableFuture fut0 = lockManager.acquire(txId0, key, X); @@ -703,7 +705,7 @@ public void testPossibleDowngradeLockModes() { @Test public void testAcquireRelease() { UUID txId = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); for (LockMode lockMode : LockMode.values()) { lockManager.acquire(txId, key, lockMode); @@ -718,7 +720,7 @@ public void testAcquireRelease() { @Test public void testAcquireReleaseWhenHoldOther() { UUID txId = TestTransactionIds.newTransactionId(); - LockKey key = new LockKey("test"); + LockKey key = lockKey(); for (LockMode holdLockMode : LockMode.values()) { lockManager.acquire(txId, key, holdLockMode); @@ -744,7 +746,7 @@ public void testAcquireReleaseWhenHoldOther() { @Test public void testReleaseThenReleaseWeakerInHierarchy() { - LockKey key = new LockKey("test"); + LockKey key = lockKey(); UUID txId1 = TestTransactionIds.newTransactionId(); UUID txId2 = TestTransactionIds.newTransactionId(); @@ -774,7 +776,7 @@ public void testReleaseThenReleaseWeakerInHierarchy() { @Test public void testReleaseThenNoReleaseWeakerInHierarchy() { - LockKey key = new LockKey("test"); + LockKey key = lockKey(); UUID txId1 = TestTransactionIds.newTransactionId(); UUID txId2 = TestTransactionIds.newTransactionId(); @@ -804,7 +806,7 @@ public void testReleaseThenNoReleaseWeakerInHierarchy() { @Test public void testLockingOverloadAndUpgrade() { - LockKey key = new LockKey("test"); + LockKey key = lockKey(); UUID tx1 = TestTransactionIds.newTransactionId(); UUID tx2 = TestTransactionIds.newTransactionId(); @@ -829,7 +831,7 @@ public void testLockingOverloadAndUpgrade() { @Test public void testLockingOverload() { - LockKey key = new LockKey("test"); + LockKey key = lockKey(); UUID tx1 = TestTransactionIds.newTransactionId(); UUID tx2 = TestTransactionIds.newTransactionId(); @@ -857,7 +859,7 @@ public void testLockingOverload() { @Test public void testFailUpgrade() { - LockKey key = new LockKey("test"); + LockKey key = lockKey(); UUID tx1 = TestTransactionIds.newTransactionId(); UUID tx2 = TestTransactionIds.newTransactionId(); @@ -891,7 +893,7 @@ public void testFailUpgrade() { @Test public void testDowngradeTargetLock() { - LockKey key = new LockKey("test"); + LockKey key = lockKey(); UUID tx1 = TestTransactionIds.newTransactionId(); UUID tx2 = TestTransactionIds.newTransactionId(); @@ -920,7 +922,7 @@ public void testDowngradeTargetLock() { @Test public void testFailWait() { - LockKey key = new LockKey("test"); + LockKey key = lockKey(); UUID tx1 = TestTransactionIds.newTransactionId(); UUID tx2 = TestTransactionIds.newTransactionId(); @@ -947,7 +949,7 @@ public void testFailWait() { @Test public void testWaitInOrder() { - LockKey key = new LockKey("test"); + LockKey key = lockKey(); UUID tx1 = TestTransactionIds.newTransactionId(); UUID tx2 = TestTransactionIds.newTransactionId(); @@ -979,7 +981,7 @@ public void testWaitInOrder() { @Test public void testWaitNotInOrder() { - LockKey key = new LockKey("test"); + LockKey key = lockKey(); UUID tx1 = TestTransactionIds.newTransactionId(); UUID tx2 = TestTransactionIds.newTransactionId(); @@ -1011,7 +1013,7 @@ public void testWaitNotInOrder() { @Test public void testWaitFailNotInOrder() { - LockKey key = new LockKey("test"); + LockKey key = lockKey(); UUID tx1 = TestTransactionIds.newTransactionId(); UUID tx2 = TestTransactionIds.newTransactionId(); @@ -1058,7 +1060,7 @@ private void doTestSingleKeyMultithreaded( LongAdder failedLocks, int mode ) throws InterruptedException { - LockKey key = new LockKey("test"); + LockKey key = lockKey(); Thread[] threads = new Thread[Runtime.getRuntime().availableProcessors() * 2]; diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/AbstractLockingTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/AbstractLockingTest.java index e70565bc76d..89c32432e44 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/AbstractLockingTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/AbstractLockingTest.java @@ -49,7 +49,7 @@ protected LockKey key(Object key) { b.putInt(key.hashCode()); b.position(0); - return new LockKey(b); + return new LockKey(0, b); } protected CompletableFuture xlock(UUID tx, LockKey key) { diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapLockManagerTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapLockManagerTest.java index 76b83571bc1..43b2baacf67 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapLockManagerTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapLockManagerTest.java @@ -28,4 +28,9 @@ public class HeapLockManagerTest extends AbstractLockManagerTest { protected LockManager newInstance() { return new HeapLockManager(new WaitDieDeadlockPreventionPolicy()); } + + @Override + protected LockKey lockKey() { + return new LockKey(0, "test"); + } } diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapUnboundedLockManagerTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapUnboundedLockManagerTest.java index 258d0160d91..24744f75049 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapUnboundedLockManagerTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/HeapUnboundedLockManagerTest.java @@ -28,4 +28,9 @@ public class HeapUnboundedLockManagerTest extends AbstractLockManagerTest { protected LockManager newInstance() { return new HeapUnboundedLockManager(new WaitDieDeadlockPreventionPolicy()); } + + @Override + protected LockKey lockKey() { + return new LockKey("test"); + } } From d39d76e0b38b68264ad81b9d2a0ce2913b856a7a Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Wed, 29 Nov 2023 14:12:49 +0300 Subject: [PATCH 26/40] IGNITE-17811 Fix tests again --- .../main/java/org/apache/ignite/internal/tx/LockManager.java | 1 - .../ignite/internal/tx/NoWaitDeadlockPreventionTest.java | 2 +- .../internal/tx/NoWaitDeadlockPreventionUnboundedTest.java | 3 +-- .../internal/tx/NoneDeadlockPreventionUnboundedTest.java | 3 +-- .../internal/tx/ReversedDeadlockPreventionUnboundedTest.java | 3 +-- .../internal/tx/TimeoutDeadlockPreventionUnboundedTest.java | 3 +-- 6 files changed, 5 insertions(+), 10 deletions(-) diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java index 92736f0ca86..e005739dd79 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java @@ -21,7 +21,6 @@ import java.util.Iterator; import java.util.UUID; import java.util.concurrent.CompletableFuture; -import org.jetbrains.annotations.Nullable; import org.jetbrains.annotations.TestOnly; /** Lock manager allows to acquire locks and release locks and supports deadlock prevention by transaction id ordering. */ diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionTest.java index 52f258d456e..6319536ee7c 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionTest.java @@ -31,7 +31,7 @@ * another transaction. */ public class NoWaitDeadlockPreventionTest extends AbstractLockingTest { - private DeadlockPreventionPolicy deadlockPreventionPolicy() { + DeadlockPreventionPolicy deadlockPreventionPolicy() { return new DeadlockPreventionPolicy() { @Override public long waitTimeout() { diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java index ea5c4e105fc..2af185b63b8 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java @@ -1,11 +1,10 @@ package org.apache.ignite.internal.tx; import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; -import org.apache.ignite.internal.tx.impl.WaitDieDeadlockPreventionPolicy; public class NoWaitDeadlockPreventionUnboundedTest extends NoWaitDeadlockPreventionTest { @Override protected LockManager lockManager() { - return new HeapUnboundedLockManager(new WaitDieDeadlockPreventionPolicy()); + return new HeapUnboundedLockManager(deadlockPreventionPolicy()); } } diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java index 4ad538599bc..45b06c5cbf3 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java @@ -1,11 +1,10 @@ package org.apache.ignite.internal.tx; import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; -import org.apache.ignite.internal.tx.impl.WaitDieDeadlockPreventionPolicy; public class NoneDeadlockPreventionUnboundedTest extends NoneDeadlockPreventionTest { @Override protected LockManager lockManager() { - return new HeapUnboundedLockManager(new WaitDieDeadlockPreventionPolicy()); + return new HeapUnboundedLockManager(deadlockPreventionPolicy()); } } diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java index a0137e219ab..9d51eaa6727 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java @@ -1,11 +1,10 @@ package org.apache.ignite.internal.tx; import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; -import org.apache.ignite.internal.tx.impl.WaitDieDeadlockPreventionPolicy; public class ReversedDeadlockPreventionUnboundedTest extends ReversedDeadlockPreventionTest { @Override protected LockManager lockManager() { - return new HeapUnboundedLockManager(new WaitDieDeadlockPreventionPolicy()); + return new HeapUnboundedLockManager(deadlockPreventionPolicy()); } } diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java index fe7fbab962c..f1e51ba82bf 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java @@ -1,11 +1,10 @@ package org.apache.ignite.internal.tx; import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; -import org.apache.ignite.internal.tx.impl.WaitDieDeadlockPreventionPolicy; public class TimeoutDeadlockPreventionUnboundedTest extends TimeoutDeadlockPreventionTest { @Override protected LockManager lockManager() { - return new HeapUnboundedLockManager(new WaitDieDeadlockPreventionPolicy()); + return new HeapUnboundedLockManager(deadlockPreventionPolicy()); } } From bd5bbac3cf6dd4cf3a5a7e57051595ad7174f4a5 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Wed, 29 Nov 2023 14:57:55 +0300 Subject: [PATCH 27/40] IGNITE-17811 Suppress warnings --- .../ItTxDistributedTestSingleNode.java | 2 +- .../PartitionReplicaListenerTest.java | 2 +- .../NoWaitDeadlockPreventionUnboundedTest.java | 17 +++++++++++++++++ .../tx/NoneDeadlockPreventionUnboundedTest.java | 17 +++++++++++++++++ ...ReversedDeadlockPreventionUnboundedTest.java | 17 +++++++++++++++++ .../TimeoutDeadlockPreventionUnboundedTest.java | 17 +++++++++++++++++ 6 files changed, 70 insertions(+), 2 deletions(-) diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java index c4ef57dfba1..2b634b6421f 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java @@ -77,7 +77,7 @@ public class ItTxDistributedTestSingleNode extends TxAbstractTest { protected RaftConfiguration raftConfiguration; @AfterEach - public void cleanup() { + public void clearMocks() { Mockito.framework().clearInlineMocks(); } diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java index 117b413e642..7aee88c3ba1 100644 --- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java +++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java @@ -495,7 +495,7 @@ public void beforeTest() { } @AfterEach - public void cleanup() { + public void clearMocks() { Mockito.framework().clearInlineMocks(); } diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java index 2af185b63b8..d72407b9ae2 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.ignite.internal.tx; import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java index 45b06c5cbf3..c7529ed9da3 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.ignite.internal.tx; import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java index 9d51eaa6727..03debb47b37 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.ignite.internal.tx; import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java index f1e51ba82bf..9209d0026c7 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.ignite.internal.tx; import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; From c37ce6ae049b5579b859ad7d5723e47a03eee94e Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Wed, 29 Nov 2023 16:10:34 +0300 Subject: [PATCH 28/40] IGNITE-17811 Suppress warnings --- .../internal/tx/NoWaitDeadlockPreventionUnboundedTest.java | 3 +++ .../internal/tx/NoneDeadlockPreventionUnboundedTest.java | 3 +++ .../internal/tx/ReversedDeadlockPreventionUnboundedTest.java | 3 +++ .../internal/tx/TimeoutDeadlockPreventionUnboundedTest.java | 3 +++ 4 files changed, 12 insertions(+) diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java index d72407b9ae2..c335950e910 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoWaitDeadlockPreventionUnboundedTest.java @@ -19,6 +19,9 @@ import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; +/** + * NoWaitDeadlockPreventionUnboundedTest. + */ public class NoWaitDeadlockPreventionUnboundedTest extends NoWaitDeadlockPreventionTest { @Override protected LockManager lockManager() { diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java index c7529ed9da3..4dcdfadd007 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/NoneDeadlockPreventionUnboundedTest.java @@ -19,6 +19,9 @@ import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; +/** + * NoneDeadlockPreventionUnboundedTest. + */ public class NoneDeadlockPreventionUnboundedTest extends NoneDeadlockPreventionTest { @Override protected LockManager lockManager() { diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java index 03debb47b37..f0f66bb8ea5 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/ReversedDeadlockPreventionUnboundedTest.java @@ -19,6 +19,9 @@ import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; +/** + * ReversedDeadlockPreventionUnboundedTest. + */ public class ReversedDeadlockPreventionUnboundedTest extends ReversedDeadlockPreventionTest { @Override protected LockManager lockManager() { diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java index 9209d0026c7..3bd121abd5d 100644 --- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java +++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TimeoutDeadlockPreventionUnboundedTest.java @@ -19,6 +19,9 @@ import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; +/** + * TimeoutDeadlockPreventionUnboundedTest. + */ public class TimeoutDeadlockPreventionUnboundedTest extends TimeoutDeadlockPreventionTest { @Override protected LockManager lockManager() { From e00a2e3dc61183c98652ee17d7219073332a947e Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Thu, 30 Nov 2023 11:12:47 +0300 Subject: [PATCH 29/40] IGNITE-17811 Suppress warnings --- .../ignite/distributed/ItLockTableTest.java | 25 ++++++++++++++++--- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java index cda62da9f2b..f5ed30e7316 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.ignite.distributed; import static org.apache.ignite.internal.replicator.ReplicaManager.DEFAULT_IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS; @@ -153,11 +170,11 @@ public void testDeadlockRecovery() { assertTrue(tx2.id().compareTo(tx1.id()) > 0); - Tuple r1_0 = view.get(tx1, keyTuple(0)); - Tuple r2_1 = view.get(tx2, keyTuple(1)); + Tuple t10 = view.get(tx1, keyTuple(0)); + Tuple t21 = view.get(tx2, keyTuple(1)); - assertEquals(t1.stringValue("name"), r1_0.stringValue("name")); - assertEquals(t2.stringValue("name"), r2_1.stringValue("name")); + assertEquals(t1.stringValue("name"), t10.stringValue("name")); + assertEquals(t2.stringValue("name"), t21.stringValue("name")); view.upsertAsync(tx1, tuple(1, "11")); view.upsertAsync(tx2, tuple(0, "00")); From 5667c940fb3578fe0e2d973abc6309a9984648cd Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Thu, 30 Nov 2023 12:52:35 +0300 Subject: [PATCH 30/40] IGNITE-17811 Move mock clearing to the safe place --- .../ignite/distributed/ItTxDistributedTestSingleNode.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java index 2b634b6421f..dc5423b4064 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java @@ -76,11 +76,6 @@ public class ItTxDistributedTestSingleNode extends TxAbstractTest { @InjectConfiguration("mock: { fsync: false }") protected RaftConfiguration raftConfiguration; - @AfterEach - public void clearMocks() { - Mockito.framework().clearInlineMocks(); - } - /** * Returns a count of nodes. * @@ -153,6 +148,7 @@ public void before() throws Exception { @AfterEach public void after() throws Exception { txTestCluster.shutdownCluster(); + Mockito.framework().clearInlineMocks(); } /** From 54324b057f1e1f2e7a41646d86d8e5f3d76bee73 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Thu, 30 Nov 2023 18:26:33 +0300 Subject: [PATCH 31/40] IGNITE-17811 Remove commented code --- .../java/org/apache/ignite/internal/tx/impl/HeapLockManager.java | 1 - 1 file changed, 1 deletion(-) diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index f13c8b2791d..e0d5bad58b8 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -284,7 +284,6 @@ private LockState lockState(LockKey key) { } } else { res[0] = v; - // assert v.waitersCount() == 1; } return v; From d42f20f32d74d3e99f49db7d9220206258dbad3e Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Fri, 1 Dec 2023 11:57:37 +0300 Subject: [PATCH 32/40] IGNITE-17811 Add event support to unbounded lock manager --- .../internal/tx/impl/HeapLockManager.java | 2 +- .../tx/impl/HeapUnboundedLockManager.java | 21 +++++++++++++++++-- 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index 08b87f67a7d..37a60d7f817 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -322,7 +322,7 @@ public boolean isEmpty() { /** * A lock state. */ - private static class LockState { + private class LockState { /** Waiters. */ private final TreeMap waiters; diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java index 48d5864c9a8..2aebc53a93d 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapUnboundedLockManager.java @@ -37,6 +37,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; +import org.apache.ignite.internal.event.AbstractEventProducer; import org.apache.ignite.internal.lang.IgniteBiTuple; import org.apache.ignite.internal.tostring.IgniteToStringExclude; import org.apache.ignite.internal.tostring.S; @@ -47,12 +48,14 @@ import org.apache.ignite.internal.tx.LockManager; import org.apache.ignite.internal.tx.LockMode; import org.apache.ignite.internal.tx.Waiter; +import org.apache.ignite.internal.tx.event.LockEvent; +import org.apache.ignite.internal.tx.event.LockEventParameters; import org.jetbrains.annotations.Nullable; /** * A {@link LockManager} which uses unbounded hashtable implementation. Suitable for holding coarse-grained locks. */ -public class HeapUnboundedLockManager implements LockManager { +public class HeapUnboundedLockManager extends AbstractEventProducer implements LockManager { /** Locks. */ private final ConcurrentHashMap locks = new ConcurrentHashMap<>(); @@ -172,7 +175,7 @@ public Waiter waiter(LockKey key, UUID txId) { /** * A lock state. */ - private static class LockState { + private class LockState { /** Waiters. */ private final TreeMap waiters; @@ -264,6 +267,8 @@ private boolean isWaiterReadyToNotify(WaiterImpl waiter, boolean skipFail) { LockMode mode = lockedMode(tmp); if (mode != null && !mode.isCompatible(waiter.intendedLockMode())) { + conflictFound(waiter.txId(), tmp.txId()); + if (!deadlockPreventionPolicy.usePriority() && deadlockPreventionPolicy.waitTimeout() == 0) { waiter.fail(lockException(waiter, tmp)); @@ -279,6 +284,8 @@ private boolean isWaiterReadyToNotify(WaiterImpl waiter, boolean skipFail) { LockMode mode = lockedMode(tmp); if (mode != null && !mode.isCompatible(waiter.intendedLockMode())) { + conflictFound(waiter.txId(), tmp.txId()); + if (skipFail) { return false; } else if (deadlockPreventionPolicy.waitTimeout() == 0) { @@ -486,6 +493,16 @@ public Waiter waiter(UUID txId) { return waiters.get(txId); } } + + /** + * Notifies about the lock conflict found between transactions. + * + * @param acquirerTx Transaction which tries to acquire the lock. + * @param holderTx Transaction which holds the lock. + */ + private void conflictFound(UUID acquirerTx, UUID holderTx) { + fireEvent(LockEvent.LOCK_CONFLICT, new LockEventParameters(acquirerTx, holderTx)); + } } /** From cce073b993ea8819b482643bf5aa1f07a8864fd1 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Fri, 1 Dec 2023 12:54:55 +0300 Subject: [PATCH 33/40] IGNITE-17811 Fix compilation --- .../org/apache/ignite/internal/tx/impl/HeapLockManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index 37a60d7f817..78b87a9f3dc 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -322,7 +322,7 @@ public boolean isEmpty() { /** * A lock state. */ - private class LockState { + public class LockState { /** Waiters. */ private final TreeMap waiters; From eb23f6ef84368f8d8514b4b1dfc655e5b940e085 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Fri, 8 Dec 2023 15:34:14 +0300 Subject: [PATCH 34/40] IGNITE-17811 Fix review comments --- .../internal/table/RecordBinaryViewImpl.java | 15 ++------------- .../replicator/PartitionReplicaListener.java | 4 ++-- .../table/impl/DummyInternalTableImpl.java | 2 -- .../apache/ignite/internal/tx/LockManager.java | 2 +- .../ignite/internal/tx/impl/HeapLockManager.java | 2 +- .../internal/tx/NoneDeadlockPreventionTest.java | 2 +- 6 files changed, 7 insertions(+), 20 deletions(-) diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java index 9cd62d57034..f1c943de946 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java @@ -24,7 +24,6 @@ import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Flow.Publisher; -import java.util.function.Consumer; import org.apache.ignite.internal.schema.BinaryRow; import org.apache.ignite.internal.schema.BinaryRowEx; import org.apache.ignite.internal.schema.SchemaRegistry; @@ -430,18 +429,8 @@ public CompletableFuture streamData(Publisher publisher, @Nullable Objects.requireNonNull(publisher); var partitioner = new TupleStreamerPartitionAwarenessProvider(rowConverter.registry(), tbl.partitions()); - StreamerBatchSender batchSender = (partitionId, items) -> withSchemaSync(null, (schemaVersion) -> { - return this.tbl.upsertAll(mapToBinary(items, schemaVersion, false), partitionId).thenAccept(new Consumer() { - @Override - public void accept(Void unused) { - try { - Thread.sleep(500); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - }); - }); + StreamerBatchSender batchSender = (partitionId, items) -> withSchemaSync(null, + (schemaVersion) -> this.tbl.upsertAll(mapToBinary(items, schemaVersion, false), partitionId)); return DataStreamer.streamData(publisher, options, batchSender, partitioner); } diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java index f862ffff2f3..8232b686e69 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java @@ -2123,10 +2123,10 @@ private CompletableFuture> processReadOnlyDirectMultiEntryAction */ private CompletableFuture processMultiEntryAction(ReadWriteMultiRowReplicaRequest request, String txCoordinatorId) { UUID txId = request.transactionId(); - TablePartitionId commitdPartitionId = request.commitPartitionId().asTablePartitionId(); + TablePartitionId commitPartitionId = request.commitPartitionId().asTablePartitionId(); List searchRows = request.binaryRows(); - assert commitdPartitionId != null : "Commit partition is null [type=" + request.requestType() + ']'; + assert commitPartitionId != null : "Commit partition is null [type=" + request.requestType() + ']'; switch (request.requestType()) { case RW_DELETE_EXACT_ALL: { diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java index 9db18a8dffc..8160c625d9d 100644 --- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java +++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java @@ -102,12 +102,10 @@ import org.apache.ignite.network.TopologyService; import org.apache.ignite.tx.TransactionException; import org.jetbrains.annotations.Nullable; -import org.jetbrains.annotations.TestOnly; /** * Dummy table storage implementation. */ -@TestOnly public class DummyInternalTableImpl extends InternalTableImpl { public static final IgniteLogger LOG = Loggers.forClass(DummyInternalTableImpl.class); diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java index 587db2e0c89..65e6132cd62 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockManager.java @@ -56,7 +56,7 @@ public interface LockManager extends EventProducer Date: Fri, 8 Dec 2023 16:00:46 +0300 Subject: [PATCH 35/40] IGNITE-17811 Fix review comments --- .../internal/table/impl/DummyInternalTableImpl.java | 12 ------------ .../ignite/internal/tx/impl/HeapLockManager.java | 3 +-- 2 files changed, 1 insertion(+), 14 deletions(-) diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java index 1f74307919c..2ebda108464 100644 --- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java +++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java @@ -155,18 +155,6 @@ public DummyInternalTableImpl(ReplicaService replicaSvc, SchemaDescriptor schema this(replicaSvc, new TestMvPartitionStorage(0), schema, new TestPlacementDriver(LOCAL_NODE), txConfiguration); } - /** - * Creates a new local table. - * - * @param replicaSvc Replica service. - * @param mvPartStorage Multi version partition storage. - * @param schema Schema. - */ - public DummyInternalTableImpl(ReplicaService replicaSvc, MvPartitionStorage mvPartStorage, SchemaDescriptor schema) { - this(replicaSvc, mvPartStorage, schema, new TestPlacementDriver(LOCAL_NODE)); - } - - /** * Creates a new local table. * diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index 1f460c274fa..7888fc44db3 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -19,7 +19,6 @@ import static java.util.Collections.emptyIterator; import static java.util.Collections.emptyList; -import static java.util.concurrent.CompletableFuture.completedFuture; import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture; import static org.apache.ignite.lang.ErrorGroups.Transactions.ACQUIRE_LOCK_ERR; import static org.apache.ignite.lang.ErrorGroups.Transactions.ACQUIRE_LOCK_TIMEOUT_ERR; @@ -369,7 +368,7 @@ public class LockState { waiter.upgrade(prev); - return new IgniteBiTuple(nullCompletedFuture(null), prev.lockMode()); + return new IgniteBiTuple(nullCompletedFuture(), prev.lockMode()); } else { waiter.upgrade(prev); From 2563dedd05a83ff21d412647399b755624413b12 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Fri, 8 Dec 2023 18:12:43 +0300 Subject: [PATCH 36/40] IGNITE-17811 Fix compilation --- .../ignite/distributed/ItLockTableTest.java | 6 ++++++ .../ItTxDistributedTestSingleNode.java | 2 +- .../table/impl/DummyInternalTableImpl.java | 19 +++++++++++++++++++ 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java index f5ed30e7316..23923287deb 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -40,6 +40,7 @@ import org.apache.ignite.internal.tx.DeadlockPreventionPolicy; import org.apache.ignite.internal.tx.HybridTimestampTracker; import org.apache.ignite.internal.tx.InternalTransaction; +import org.apache.ignite.internal.tx.configuration.TransactionConfiguration; import org.apache.ignite.internal.tx.impl.HeapLockManager; import org.apache.ignite.internal.tx.impl.HeapLockManager.LockState; import org.apache.ignite.internal.tx.impl.HeapUnboundedLockManager; @@ -92,6 +93,9 @@ public class ItLockTableTest extends IgniteAbstractTest { @InjectConfiguration protected static GcConfiguration gcConfig; + @InjectConfiguration + protected static TransactionConfiguration txConfiguration; + private ItTxTestCluster txTestCluster; private HybridTimestampTracker timestampTracker = new HybridTimestampTracker(); @@ -110,6 +114,7 @@ public void before() throws Exception { txTestCluster = new ItTxTestCluster( testInfo, raftConfiguration, + txConfiguration, workDir, 1, 1, @@ -126,6 +131,7 @@ protected TxManagerImpl newTxManager( PlacementDriver placementDriver ) { return new TxManagerImpl( + txConfiguration, clusterService, replicaSvc, new HeapLockManager( diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java index ae2bfd340c6..2b4f99ce358 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java @@ -78,7 +78,7 @@ public class ItTxDistributedTestSingleNode extends TxAbstractTest { protected RaftConfiguration raftConfiguration; @InjectConfiguration - protected static TransactionConfiguration txConfiguration; + protected TransactionConfiguration txConfiguration; /** * Returns a count of nodes. diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java index 2ebda108464..32dd822de89 100644 --- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java +++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java @@ -103,10 +103,12 @@ import org.apache.ignite.network.TopologyService; import org.apache.ignite.tx.TransactionException; import org.jetbrains.annotations.Nullable; +import org.jetbrains.annotations.TestOnly; /** * Dummy table storage implementation. */ +@TestOnly public class DummyInternalTableImpl extends InternalTableImpl { public static final IgniteLogger LOG = Loggers.forClass(DummyInternalTableImpl.class); @@ -155,6 +157,23 @@ public DummyInternalTableImpl(ReplicaService replicaSvc, SchemaDescriptor schema this(replicaSvc, new TestMvPartitionStorage(0), schema, new TestPlacementDriver(LOCAL_NODE), txConfiguration); } + /** + * Creates a new local table. + * + * @param replicaSvc Replica service. + * @param storage Storage. + * @param schema Schema. + * @param txConfiguration Transaction configuration. + */ + public DummyInternalTableImpl( + ReplicaService replicaSvc, + MvPartitionStorage storage, + SchemaDescriptor schema, + TransactionConfiguration txConfiguration + ) { + this(replicaSvc, storage, schema, new TestPlacementDriver(LOCAL_NODE), txConfiguration); + } + /** * Creates a new local table. * From 833a8d11a373e320ff0784fa2e2a635f4f3a37df Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Sun, 10 Dec 2023 13:32:33 +0300 Subject: [PATCH 37/40] IGNITE-17811 Cleanup --- .../java/org/apache/ignite/distributed/ItLockTableTest.java | 3 +++ .../table/distributed/replicator/PartitionReplicaListener.java | 1 - 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java index 23923287deb..269e9092c19 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -194,6 +194,9 @@ public void testDeadlockRecovery() { total += slot.waitersCount(); } + // 2 S_lock on key=0 + 2 S_lock on key=1 - resolve by PK + // 1 S lock + 1 X lock on rowId(0) - read and update row0 + // 1 S lock + 1 X lock on rowId(1) - read and update row1 return total == 8; }, 10_000), "Some lockers are missing"); diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java index 5fd6c002272..62f5433e346 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java @@ -1562,7 +1562,6 @@ private CompletableFuture cleanup( UUID txId, int attemptsToCleanupReplica ) { - // Avoid invoking async chain in raft threads. CompletableFuture[] futures = enlistedPartitions.stream() .map(partitionId -> changeStateFuture.thenCompose(ignored -> // TODO: IGNITE-20874 Use the node cleanup procedure instead of the replication group cleanup one. From d2de79097b5ddd102dea7e7a62375196cf555777 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Wed, 13 Dec 2023 12:22:59 +0300 Subject: [PATCH 38/40] IGNITE-17811 Removed test --- .../ignite/distributed/ItLockTableTest.java | 48 ------------------- .../table/impl/DummyInternalTableImpl.java | 2 +- 2 files changed, 1 insertion(+), 49 deletions(-) diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java index 269e9092c19..ab97070520f 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItLockTableTest.java @@ -18,7 +18,6 @@ package org.apache.ignite.distributed; import static org.apache.ignite.internal.replicator.ReplicaManager.DEFAULT_IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS; -import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.ArrayList; @@ -39,7 +38,6 @@ import org.apache.ignite.internal.testframework.IgniteAbstractTest; import org.apache.ignite.internal.tx.DeadlockPreventionPolicy; import org.apache.ignite.internal.tx.HybridTimestampTracker; -import org.apache.ignite.internal.tx.InternalTransaction; import org.apache.ignite.internal.tx.configuration.TransactionConfiguration; import org.apache.ignite.internal.tx.impl.HeapLockManager; import org.apache.ignite.internal.tx.impl.HeapLockManager.LockState; @@ -55,7 +53,6 @@ import org.apache.ignite.tx.Transaction; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; @@ -158,51 +155,6 @@ public void after() throws Exception { txTestCluster.shutdownCluster(); } - @Test - @Disabled("https://issues.apache.org/jira/browse/IGNITE-20894") - public void testDeadlockRecovery() { - RecordView view = testTable.recordView(); - Tuple t1 = tuple(0, "0"); - assertTrue(view.insert(null, t1)); - - Tuple t2 = tuple(1, "1"); - assertTrue(view.insert(null, t2)); - - InternalTransaction tx1 = (InternalTransaction) txTestCluster.igniteTransactions().begin(); - InternalTransaction tx2 = (InternalTransaction) txTestCluster.igniteTransactions().begin(); - - LOG.info("id1={}", tx1.id()); - LOG.info("id2={}", tx2.id()); - - assertTrue(tx2.id().compareTo(tx1.id()) > 0); - - Tuple t10 = view.get(tx1, keyTuple(0)); - Tuple t21 = view.get(tx2, keyTuple(1)); - - assertEquals(t1.stringValue("name"), t10.stringValue("name")); - assertEquals(t2.stringValue("name"), t21.stringValue("name")); - - view.upsertAsync(tx1, tuple(1, "11")); - view.upsertAsync(tx2, tuple(0, "00")); - - assertTrue(TestUtils.waitForCondition(() -> { - int total = 0; - HeapLockManager lockManager = (HeapLockManager) txTestCluster.txManagers.get(txTestCluster.localNodeName).lockManager(); - for (int j = 0; j < lockManager.getSlots().length; j++) { - LockState slot = lockManager.getSlots()[j]; - - total += slot.waitersCount(); - } - - // 2 S_lock on key=0 + 2 S_lock on key=1 - resolve by PK - // 1 S lock + 1 X lock on rowId(0) - read and update row0 - // 1 S lock + 1 X lock on rowId(1) - read and update row1 - return total == 8; - }, 10_000), "Some lockers are missing"); - - tx1.commit(); - } - /** * Test that a lock table behaves correctly in case of lock cache overflow. */ diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java index 7e2621c236b..f3425f0723b 100644 --- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java +++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java @@ -379,7 +379,7 @@ public void result(@Nullable Serializable r) { LOCAL_NODE, new AlwaysSyncedSchemaSyncService(), catalogService, - new TestPlacementDriver(LOCAL_NODE) + new TestPlacementDriver(LOCAL_NODE), mock(ClusterNodeResolver.class) ); From d5e0596dcd857ddbebb35cc910189610a3f98738 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Fri, 15 Dec 2023 16:11:10 +0300 Subject: [PATCH 39/40] IGNITE-17811 Fix attempt --- .../apache/ignite/internal/tx/impl/TxCleanupRequestHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxCleanupRequestHandler.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxCleanupRequestHandler.java index 6671e686032..e79ed8ab4ff 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxCleanupRequestHandler.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxCleanupRequestHandler.java @@ -138,7 +138,7 @@ private void processTxCleanup(TxCleanupMessage txCleanupMessage, String senderId } private void releaseTxLocks(UUID txId) { - lockManager.locks(txId).forEachRemaining(lockManager::release); + lockManager.releaseAll(txId); } private NetworkMessage prepareResponse() { From fa7262f3b86032e471856b67fe39bc88365f00b7 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Fri, 15 Dec 2023 16:41:19 +0300 Subject: [PATCH 40/40] IGNITE-17811 Fix style --- .../internal/table/impl/DummyInternalTableImpl.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java index f3425f0723b..0574317e938 100644 --- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java +++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java @@ -424,9 +424,14 @@ public TxManager txManager() { * Creates a {@link TxManager}. * * @param replicaSvc Replica service to use. + * @param placementDriver Placement driver. * @param txConfiguration Transaction configuration. */ - public static TxManagerImpl txManager(ReplicaService replicaSvc, PlacementDriver placementDriver, TransactionConfiguration txConfiguration) { + public static TxManagerImpl txManager( + ReplicaService replicaSvc, + PlacementDriver placementDriver, + TransactionConfiguration txConfiguration + ) { TopologyService topologyService = mock(TopologyService.class); when(topologyService.localMember()).thenReturn(LOCAL_NODE); @@ -494,8 +499,7 @@ public void addIndexToWaitIfAbsent(int indexId) { } /** - * Dummy messaging service for tests purposes. - * It does not provide any messaging functionality, but allows to trigger events. + * Dummy messaging service for tests purposes. It does not provide any messaging functionality, but allows to trigger events. */ private static class DummyMessagingService extends AbstractMessagingService { private final String localNodeName;