This is an automated email from the ASF dual-hosted git repository.

ibessonov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 0af08752aec IGNITE-28330 Include partition ID into sorted index lock 
key (#7846)
0af08752aec is described below

commit 0af08752aec2bf880f3f5193bf417ad4a8d00d30
Author: Ivan Bessonov <[email protected]>
AuthorDate: Wed Mar 25 10:28:58 2026 +0300

    IGNITE-28330 Include partition ID into sorted index lock key (#7846)
---
 .../table/distributed/SortedIndexLocker.java       | 54 ++++++++++++++++++----
 .../PartitionReplicaListenerIndexLockingTest.java  |  6 ++-
 ...itionReplicaListenerSortedIndexLockingTest.java |  6 ++-
 3 files changed, 54 insertions(+), 12 deletions(-)

diff --git 
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/SortedIndexLocker.java
 
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/SortedIndexLocker.java
index f7177800891..658168388ed 100644
--- 
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/SortedIndexLocker.java
+++ 
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/SortedIndexLocker.java
@@ -35,6 +35,7 @@ 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.util.Cursor;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -46,7 +47,7 @@ public class SortedIndexLocker implements IndexLocker {
     /** Index INF+ value object. */
     private final Object positiveInf;
 
-    private final int indexId;
+    private final PartitionIndexId contextId;
 
     private final LockManager lockManager;
 
@@ -74,7 +75,7 @@ public class SortedIndexLocker implements IndexLocker {
             ColumnsExtractor indexRowResolver,
             boolean unique
     ) {
-        this.indexId = indexId;
+        this.contextId = new PartitionIndexId(partId, indexId);
         this.lockManager = lockManager;
         this.storage = storage;
         this.indexRowResolver = indexRowResolver;
@@ -84,12 +85,12 @@ public class SortedIndexLocker implements IndexLocker {
 
     @Override
     public int id() {
-        return indexId;
+        return contextId.indexId;
     }
 
     @Override
     public CompletableFuture<Void> locksForLookupByKey(UUID txId, BinaryTuple 
key) {
-        return lockManager.acquire(txId, new LockKey(indexId, 
key.byteBuffer()), LockMode.S).thenApply(lock -> null);
+        return lockManager.acquire(txId, new LockKey(contextId, 
key.byteBuffer()), LockMode.S).thenApply(lock -> null);
     }
 
     /** {@inheritDoc} */
@@ -131,7 +132,7 @@ public class SortedIndexLocker implements IndexLocker {
     private CompletableFuture<IndexRow> acquireLockNextKey(UUID txId, 
PeekCursor<IndexRow> peekCursor) {
         IndexRow peekedRow = peekCursor.peek();
 
-        LockKey lockKey = new LockKey(indexId, indexKey(peekedRow));
+        LockKey lockKey = new LockKey(contextId, indexKey(peekedRow));
 
         return lockManager.acquire(txId, lockKey, LockMode.S)
                 .thenCompose(ignore -> {
@@ -179,12 +180,12 @@ public class SortedIndexLocker implements IndexLocker {
             return nullCompletedFuture();
         }
 
-        var nextLockKey = new LockKey(indexId, indexKey(nextRow));
+        var nextLockKey = new LockKey(contextId, indexKey(nextRow));
 
         return lockManager.acquire(txId, nextLockKey, 
LockMode.IX).thenCompose(shortLock -> {
             LockMode modeToLock = currentKeyLockMode(shortLock.lockMode());
 
-            return lockManager.acquire(txId, new LockKey(indexId, 
key.byteBuffer()), modeToLock)
+            return lockManager.acquire(txId, new LockKey(contextId, 
key.byteBuffer()), modeToLock)
                     .thenApply(lock -> new Lock(nextLockKey, LockMode.IX, 
txId));
         });
     }
@@ -206,6 +207,43 @@ public class SortedIndexLocker implements IndexLocker {
     public CompletableFuture<Void> locksForRemove(UUID txId, BinaryRow 
tableRow, RowId rowId) {
         BinaryTuple key = indexRowResolver.extractColumns(tableRow);
 
-        return lockManager.acquire(txId, new LockKey(indexId, 
key.byteBuffer()), LockMode.IX).thenApply(lock -> null);
+        return lockManager.acquire(txId, new LockKey(contextId, 
key.byteBuffer()), LockMode.IX).thenApply(lock -> null);
+    }
+
+    /**
+     * Composite context ID for lock keys, that includes partition ID and 
index ID.
+     */
+    public static class PartitionIndexId {
+        private final int partitionId;
+        private final int indexId;
+        private final int hash;
+
+        /**
+         * Constructor.
+         */
+        public PartitionIndexId(int partitionId, int indexId) {
+            this.partitionId = partitionId;
+            this.indexId = indexId;
+            this.hash = IgniteUtils.hash(65535 * partitionId + indexId);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+
+            PartitionIndexId that = (PartitionIndexId) o;
+            return partitionId == that.partitionId && indexId == that.indexId;
+        }
+
+        @Override
+        public int hashCode() {
+            return hash;
+        }
     }
 }
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 8dc3e18b68f..6573a2773a6 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
@@ -146,6 +146,8 @@ public class PartitionReplicaListenerIndexLockingTest 
extends IgniteAbstractTest
     private static final int PK_INDEX_ID = 1;
     private static final int HASH_INDEX_ID = 2;
     private static final int SORTED_INDEX_ID = 3;
+    private static final SortedIndexLocker.PartitionIndexId 
SORTED_INDEX_CONTEXT_ID
+            = new SortedIndexLocker.PartitionIndexId(PART_ID, SORTED_INDEX_ID);
     private static final int ZONE_ID = 4;
     private static final UUID TRANSACTION_ID = 
TestTransactionIds.newTransactionId();
     private static final HybridClock CLOCK = new HybridClockImpl();
@@ -417,7 +419,7 @@ public class PartitionReplicaListenerIndexLockingTest 
extends IgniteAbstractTest
                         )),
                         hasItem(lockThat(
                                 arg.expectedLockOnSort + " on sorted index",
-                                lock -> Objects.equals(SORTED_INDEX_ID, 
lock.lockKey().contextId())
+                                lock -> 
Objects.equals(SORTED_INDEX_CONTEXT_ID, lock.lockKey().contextId())
                                         && lock.lockMode() == 
arg.expectedLockOnSort
                         ))
                 )
@@ -512,7 +514,7 @@ public class PartitionReplicaListenerIndexLockingTest 
extends IgniteAbstractTest
                             )),
                             hasItem(lockThat(
                                     arg.expectedLockOnSort + " on sorted 
index",
-                                    lock -> Objects.equals(SORTED_INDEX_ID, 
lock.lockKey().contextId())
+                                    lock -> 
Objects.equals(SORTED_INDEX_CONTEXT_ID, lock.lockKey().contextId())
                                             && 
row2SortKeyConverter.extractColumns(row).byteBuffer().equals(lock.lockKey().key())
                                             && lock.lockMode() == 
arg.expectedLockOnSort
                             ))
diff --git 
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerSortedIndexLockingTest.java
 
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerSortedIndexLockingTest.java
index 1da1c1da85e..b835a330d0e 100644
--- 
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerSortedIndexLockingTest.java
+++ 
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerSortedIndexLockingTest.java
@@ -140,6 +140,8 @@ public class PartitionReplicaListenerSortedIndexLockingTest 
extends IgniteAbstra
     private static final int TABLE_ID = 1;
     private static final int PK_INDEX_ID = 1;
     private static final int ZONE_ID = 2;
+    private static final SortedIndexLocker.PartitionIndexId PK_INDEX_CONTEXT_ID
+            = new SortedIndexLocker.PartitionIndexId(PART_ID, PK_INDEX_ID);
     private static final UUID TRANSACTION_ID = 
TestTransactionIds.newTransactionId();
     private static final HybridClock CLOCK = new HybridClockImpl();
     private static final ClockService CLOCK_SERVICE = new 
TestClockService(CLOCK);
@@ -377,7 +379,7 @@ public class PartitionReplicaListenerSortedIndexLockingTest 
extends IgniteAbstra
                 allOf(
                         hasItem(lockThat(
                                 arg.expectedLockOnSortedPk + " on sorted pk 
index",
-                                lock -> Objects.equals(PK_INDEX_ID, 
lock.lockKey().contextId())
+                                lock -> Objects.equals(PK_INDEX_CONTEXT_ID, 
lock.lockKey().contextId())
                                         && 
row2HashKeyConverter.extractColumns(testBinaryRow).byteBuffer().equals(lock.lockKey().key())
                                         && lock.lockMode() == 
arg.expectedLockOnSortedPk
                         ))
@@ -457,7 +459,7 @@ public class PartitionReplicaListenerSortedIndexLockingTest 
extends IgniteAbstra
                     allOf(
                             hasItem(lockThat(
                                     arg.expectedLockOnSortedPk + " on sorted 
pk index",
-                                    lock -> Objects.equals(PK_INDEX_ID, 
lock.lockKey().contextId())
+                                    lock -> 
Objects.equals(PK_INDEX_CONTEXT_ID, lock.lockKey().contextId())
                                             && 
row2HashKeyConverter.extractColumns(row).byteBuffer().equals(lock.lockKey().key())
                                             && lock.lockMode() == 
arg.expectedLockOnSortedPk
                             ))

Reply via email to