You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/10/27 15:17:00 UTC

[GitHub] [ignite-3] sashapolo commented on a change in pull request #414: IGNITE-15536 Use VarHandle to replace UNSAFE in IgniteSpinReadWriteLock

sashapolo commented on a change in pull request #414:
URL: https://github.com/apache/ignite-3/pull/414#discussion_r737430892



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not

Review comment:
       ```suggestion
    * Its blocking methods use the spinwait strategy. When they do so, they are not interruptible (that is, they do not
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt

Review comment:
       ```suggestion
    * acquire requests are received when the write lock is held by someone else, then, on its release, the write lock attempt
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {
             return 0;
         }
     };
 
-    /** */
+    /**
+     * Main state of the lock.
+     * <ul>
+     *     <li>Positive when read lock is acquired by at least one thread; in such a case, this number equals
+     *     to the number of threads holding the read lock. In such a state, read lock may be acquired by any thread,

Review comment:
       ```suggestion
        *     to the number of threads holding the read lock. In such state, the read lock may be acquired by any thread,
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {
             return 0;
         }
     };
 
-    /** */
+    /**
+     * Main state of the lock.
+     * <ul>
+     *     <li>Positive when read lock is acquired by at least one thread; in such a case, this number equals

Review comment:
       ```suggestion
        *     <li>Positive when the read lock has been acquired by at least one thread; in such case, this number equals
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {
             return 0;
         }
     };
 
-    /** */
+    /**
+     * Main state of the lock.
+     * <ul>
+     *     <li>Positive when read lock is acquired by at least one thread; in such a case, this number equals
+     *     to the number of threads holding the read lock. In such a state, read lock may be acquired by any thread,
+     *     while an attempt to acquire a write lock will block or fail.</li>
+     *     <li>Zero when neither read lock, neither write lock is acquired by any thread. This state allows
+     *     a thread to acquire either read or write lock at will.</li>
+     *     <li>-1 when write lock is acquired by exactly one thread. In such a state, any attempt to acquire a read
+     *     or write lock by any thread (but the thread holding the write lock) will block or fail.</li>
+     * </ul>
+     */
     private volatile int state;
 
-    /** */
+    /**
+     * Number of pending write attempts to acquire a write lock. Currently it is only used to prioritize

Review comment:
       ```suggestion
        * Number of pending write attempts to acquire the write lock. Currently it is only used to prioritize
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -172,101 +244,107 @@ public void readUnlock() {
     }
 
     /**
-     * Acquires write lock.
+     * Acquires write lock waiting, if needed. The thread will block until all other threads release both read
+     * and write locks.
      */
     @SuppressWarnings("BusyWait")
     public void writeLock() {
-        long threadId = Thread.currentThread().getId();
-
-        if (threadId == writeLockOwner) {
-            assert state == -1;
-
-            writeLockEntryCnt++;
+        if (writeLockedByCurrentThread()) {
+            incrementWriteLockCount();
 
             return;
         }
 
-        // Increment pending write locks.
-        while (true) {
-            int pendingWLocks0 = pendingWLocks;
-
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 + 1))
-                break;
-        }
-
         boolean interrupted = false;
 
-        while (!compareAndSet(STATE_OFFS, 0, -1)) {
-            try {
-                Thread.sleep(10);
-            }
-            catch (InterruptedException ignored) {
-                interrupted = true;
+        incrementPendingWriteLocks();
+        try {
+            while (!trySwitchStateToWriteLocked()) {
+                try {
+                    Thread.sleep(10);
+                }
+                catch (InterruptedException ignored) {
+                    interrupted = true;
+                }
             }
         }
-
-        // Decrement pending write locks.
-        while (true) {
-            int pendingWLocks0 = pendingWLocks;
-
-            assert pendingWLocks0 > 0;
-
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 - 1))
-                break;
+        finally {
+            decrementPendingWriteLocks();
         }
 
         if (interrupted)
             Thread.currentThread().interrupt();
 
-        assert writeLockOwner == -1;
-
-        writeLockOwner = threadId;
-        writeLockEntryCnt = 1;
+        finishWriteLockAckquire();
     }
 
-    /**
-     * Acquires write lock without sleeping between unsuccessful attempts.
-     */
-    public void writeLock0() {
-        long threadId = Thread.currentThread().getId();
-
-        if (threadId == writeLockOwner) {
-            assert state == -1;
-
-            writeLockEntryCnt++;
+    /***/
+    private void incrementWriteLockCount() {
+        assert state == WRITE_LOCKED;
 
-            return;
-        }
+        writeLockEntryCnt++;
+    }
 
-        // Increment pending write locks.
+    /***/
+    private void incrementPendingWriteLocks() {
         while (true) {
-            int pendingWLocks0 = pendingWLocks;
+            int curPendingWLocks = pendingWLocks;
 
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 + 1))
+            if (compareAndSet(PENDING_WLOCKS_VH, curPendingWLocks, curPendingWLocks + 1))
                 break;
         }
+    }
 
-        for (;;) {
-            if (compareAndSet(STATE_OFFS, 0, -1))
-                break;
-        }
+    /***/
+    private boolean trySwitchStateToWriteLocked() {
+        return compareAndSet(STATE_VH, AVAILABLE, WRITE_LOCKED);
+    }
 
-        // Decrement pending write locks.
+    /***/
+    private void decrementPendingWriteLocks() {
         while (true) {
-            int pendingWLocks0 = pendingWLocks;
+            int curPendingWLocks = pendingWLocks;
 
-            assert pendingWLocks0 > 0;
+            assert curPendingWLocks > 0;
 
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 - 1))
+            if (compareAndSet(PENDING_WLOCKS_VH, curPendingWLocks, curPendingWLocks - 1))
                 break;
         }
+    }
 
-        assert writeLockOwner == -1;
+    /***/
+    private void finishWriteLockAckquire() {

Review comment:
       ```suggestion
       private void finishWriteLockAcquire() {
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -172,101 +244,107 @@ public void readUnlock() {
     }
 
     /**
-     * Acquires write lock.
+     * Acquires write lock waiting, if needed. The thread will block until all other threads release both read
+     * and write locks.
      */
     @SuppressWarnings("BusyWait")
     public void writeLock() {
-        long threadId = Thread.currentThread().getId();
-
-        if (threadId == writeLockOwner) {
-            assert state == -1;
-
-            writeLockEntryCnt++;
+        if (writeLockedByCurrentThread()) {
+            incrementWriteLockCount();
 
             return;
         }
 
-        // Increment pending write locks.
-        while (true) {
-            int pendingWLocks0 = pendingWLocks;
-
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 + 1))
-                break;
-        }
-
         boolean interrupted = false;
 
-        while (!compareAndSet(STATE_OFFS, 0, -1)) {
-            try {
-                Thread.sleep(10);
-            }
-            catch (InterruptedException ignored) {
-                interrupted = true;
+        incrementPendingWriteLocks();
+        try {
+            while (!trySwitchStateToWriteLocked()) {
+                try {
+                    Thread.sleep(10);
+                }
+                catch (InterruptedException ignored) {
+                    interrupted = true;
+                }
             }
         }
-
-        // Decrement pending write locks.
-        while (true) {
-            int pendingWLocks0 = pendingWLocks;
-
-            assert pendingWLocks0 > 0;
-
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 - 1))
-                break;
+        finally {
+            decrementPendingWriteLocks();
         }
 
         if (interrupted)
             Thread.currentThread().interrupt();
 
-        assert writeLockOwner == -1;
-
-        writeLockOwner = threadId;
-        writeLockEntryCnt = 1;
+        finishWriteLockAckquire();
     }
 
-    /**
-     * Acquires write lock without sleeping between unsuccessful attempts.
-     */
-    public void writeLock0() {
-        long threadId = Thread.currentThread().getId();
-
-        if (threadId == writeLockOwner) {
-            assert state == -1;
-
-            writeLockEntryCnt++;
+    /***/
+    private void incrementWriteLockCount() {
+        assert state == WRITE_LOCKED;
 
-            return;
-        }
+        writeLockEntryCnt++;
+    }
 
-        // Increment pending write locks.
+    /***/
+    private void incrementPendingWriteLocks() {
         while (true) {
-            int pendingWLocks0 = pendingWLocks;
+            int curPendingWLocks = pendingWLocks;
 
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 + 1))
+            if (compareAndSet(PENDING_WLOCKS_VH, curPendingWLocks, curPendingWLocks + 1))
                 break;
         }
+    }
 
-        for (;;) {
-            if (compareAndSet(STATE_OFFS, 0, -1))
-                break;
-        }
+    /***/
+    private boolean trySwitchStateToWriteLocked() {
+        return compareAndSet(STATE_VH, AVAILABLE, WRITE_LOCKED);
+    }
 
-        // Decrement pending write locks.
+    /***/
+    private void decrementPendingWriteLocks() {
         while (true) {
-            int pendingWLocks0 = pendingWLocks;
+            int curPendingWLocks = pendingWLocks;
 
-            assert pendingWLocks0 > 0;
+            assert curPendingWLocks > 0;
 
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 - 1))
+            if (compareAndSet(PENDING_WLOCKS_VH, curPendingWLocks, curPendingWLocks - 1))
                 break;
         }
+    }
 
-        assert writeLockOwner == -1;
+    /***/
+    private void finishWriteLockAckquire() {
+        assert writeLockOwner == NO_OWNER;
 
-        writeLockOwner = threadId;
+        writeLockOwner = Thread.currentThread().getId();
         writeLockEntryCnt = 1;
     }
 
+    /**
+     * Acquires write lock without sleeping between unsuccessful attempts. Instead, the spinwait eats cycles of
+     * the core it gets at full speed. It is non-interruptible as its {@link #writeLock()} cousin.
+     */
+    public void writeLock0() {
+        if (writeLockedByCurrentThread()) {
+            incrementWriteLockCount();
+
+            return;
+        }
+
+        incrementPendingWriteLocks();
+        try {
+            while (true) {

Review comment:
       ```suggestion
               while (!trySwitchStateToWriteLocked()) {
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -378,25 +430,34 @@ public void writeUnlock() {
         }
 
         writeLockEntryCnt = 0;
-        writeLockOwner = -1;
+        writeLockOwner = NO_OWNER;
 
         // Current thread holds write and read locks and is releasing
         // write lock now.
-        int update = readLockEntryCnt.get() > 0 ? 1 : 0;
+        int update = readLockEntryCnt.get() > 0 ? 1 : AVAILABLE;
 
-        boolean b = compareAndSet(STATE_OFFS, -1, update);
+        boolean b = compareAndSet(STATE_VH, WRITE_LOCKED, update);
 
         assert b;
     }
 
     /**
-     * @param offs Offset.
+     * @param varHandle VarHandle.
      * @param expect Expected.
      * @param update Update.
      * @return {@code True} on success.
      */
-    private boolean compareAndSet(long offs, int expect, int update) {
-        return GridUnsafe.compareAndSwapInt(this, offs, expect, update);
+    private boolean compareAndSet(VarHandle varHandle, int expect, int update) {
+        return varHandle.compareAndSet(this, expect, update);
+    }
+
+    /**
+     * Returns count of pending write lock requests count. Only used by tests, should not be used in production code.

Review comment:
       ```suggestion
        * Returns the count of pending write lock requests count. Only used by tests, should not be used in production code.
   ```

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLockTest.java
##########
@@ -0,0 +1,530 @@
+/*
+ * 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.util;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests for {@link IgniteSpinReadWriteLock}.
+ */
+@Timeout(20)
+class IgniteSpinReadWriteLockTest {
+    /** The lock under test. */
+    private final IgniteSpinReadWriteLock lock = new IgniteSpinReadWriteLock();
+
+    /** Poller used to poll for 10 seconds. */
+    private final Poller poller10Sec = new Poller(Duration.ofSeconds(10), Duration.ofMillis(10));
+
+    /** Executor service used to run tasks in threads different from the main test thread. */
+    private final ExecutorService executor = Executors.newCachedThreadPool();
+
+    /**
+     * Cleans up after a test.
+     *
+     * @throws Exception if something goes wrong
+     */
+    @AfterEach
+    void cleanup() throws Exception {
+        releaseReadLockHeldByCurrentThread();
+        releaseWriteLockHeldByCurrentThread();
+
+        executor.shutdownNow();
+        executor.awaitTermination(3, TimeUnit.SECONDS);
+    }
+
+    /***/
+    private void releaseReadLockHeldByCurrentThread() {
+        while (true) {
+            try {
+                lock.readUnlock();
+            }
+            catch (IllegalMonitorStateException e) {
+                // released our read lock completely
+                break;
+            }
+        }
+    }
+
+    /***/
+    private void releaseWriteLockHeldByCurrentThread() {
+        while (lock.writeLockedByCurrentThread())
+            lock.writeUnlock();
+    }
+
+    /**
+     *
+     */
+    @Test
+    void readLockDoesNotAllowWriteLockToBeAcquired() {
+        lock.readLock();
+
+        assertThatWriteLockAcquireAttemptBlocksForever();
+
+        lock.readUnlock();
+    }
+
+    /***/
+    private void assertThatWriteLockAcquireAttemptBlocksForever() {
+        Future<?> future = executor.submit(lock::writeLock);
+
+        assertThrows(TimeoutException.class, () -> future.get(500, TimeUnit.MILLISECONDS));
+    }
+
+    /**
+     *
+     */
+    @Test
+    void readLockDoesNotAllowWriteLockWithoutSleepsToBeAcquired() {
+        lock.readLock();
+
+        assertThatWriteLockAcquireAttemptWithoutSleepsBlocksForever();
+
+        lock.readUnlock();
+    }
+
+    /**
+     *
+     */
+    @Test
+    void readLockDoesNotAllowWriteLockToBeAcquiredWithTimeout() throws Exception {
+        lock.readLock();
+
+        Boolean acquired = callWithTimeout(() -> lock.tryWriteLock(1, TimeUnit.MILLISECONDS));
+        assertThat(acquired, is(false));
+
+        lock.readUnlock();
+    }
+
+    /**
+     *
+     */
+    @Test
+    void readLockAllowsReadLockToBeAcquired() throws Exception {
+        lock.readLock();
+
+        assertThatReadLockCanBeAcquired();
+    }
+
+    /***/
+    private void assertThatReadLockCanBeAcquired() throws InterruptedException, ExecutionException, TimeoutException {
+        Future<?> readLockAttemptFuture = executor.submit(lock::readLock);
+        getWithTimeout(readLockAttemptFuture);
+    }
+
+    /***/
+    private <T> T callWithTimeout(Callable<T> call) throws ExecutionException, InterruptedException, TimeoutException {
+        Future<T> future = executor.submit(call);
+        return getWithTimeout(future);
+    }
+
+    /***/
+    private void runWithTimeout(Runnable runnable) throws ExecutionException, InterruptedException, TimeoutException {
+        Future<?> future = executor.submit(runnable);
+        getWithTimeout(future);
+    }
+
+    /***/
+    private <T> T getWithTimeout(Future<? extends T> future) throws ExecutionException,

Review comment:
       I think that this method can be inlined, for example `executor.submit(runnable).get(10, TimeUnit.SECONDS)`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -378,25 +430,34 @@ public void writeUnlock() {
         }
 
         writeLockEntryCnt = 0;
-        writeLockOwner = -1;
+        writeLockOwner = NO_OWNER;
 
         // Current thread holds write and read locks and is releasing
         // write lock now.
-        int update = readLockEntryCnt.get() > 0 ? 1 : 0;
+        int update = readLockEntryCnt.get() > 0 ? 1 : AVAILABLE;
 
-        boolean b = compareAndSet(STATE_OFFS, -1, update);
+        boolean b = compareAndSet(STATE_VH, WRITE_LOCKED, update);
 
         assert b;
     }
 
     /**
-     * @param offs Offset.
+     * @param varHandle VarHandle.
      * @param expect Expected.
      * @param update Update.
      * @return {@code True} on success.
      */
-    private boolean compareAndSet(long offs, int expect, int update) {
-        return GridUnsafe.compareAndSwapInt(this, offs, expect, update);
+    private boolean compareAndSet(VarHandle varHandle, int expect, int update) {
+        return varHandle.compareAndSet(this, expect, update);
+    }
+
+    /**
+     * Returns count of pending write lock requests count. Only used by tests, should not be used in production code.
+     *
+     * @return count of pending requests to get write lock

Review comment:
       ```suggestion
        * @return count of pending requests to get the write lock
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);

Review comment:
       I would suggest using the `ExceptionInInitializerError` here

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>

Review comment:
       There's no need to use closing `<\p>` tags in javadocs, see https://www.oracle.com/technical-resources/articles/java/javadoc-tool.html#examples 

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */

Review comment:
       ```suggestion
       /** {@link VarHandle} used to access the {@code state} field. */
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */

Review comment:
       ```suggestion
       /** {@link VarHandle} used to access the {@code pendingWLocks} field. */
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {

Review comment:
       can be replaced with `ThreadLocal.withInitial(() -> 0);`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */

Review comment:
       ```suggestion
       /** Signals that nobody currently owns the read lock. */
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {
             return 0;
         }
     };
 
-    /** */
+    /**
+     * Main state of the lock.
+     * <ul>
+     *     <li>Positive when read lock is acquired by at least one thread; in such a case, this number equals
+     *     to the number of threads holding the read lock. In such a state, read lock may be acquired by any thread,
+     *     while an attempt to acquire a write lock will block or fail.</li>
+     *     <li>Zero when neither read lock, neither write lock is acquired by any thread. This state allows
+     *     a thread to acquire either read or write lock at will.</li>
+     *     <li>-1 when write lock is acquired by exactly one thread. In such a state, any attempt to acquire a read

Review comment:
       ```suggestion
        *     <li>-1 when the write lock has been acquired by exactly one thread. In such state, any attempt to acquire the read
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {
             return 0;
         }
     };
 
-    /** */
+    /**
+     * Main state of the lock.
+     * <ul>
+     *     <li>Positive when read lock is acquired by at least one thread; in such a case, this number equals
+     *     to the number of threads holding the read lock. In such a state, read lock may be acquired by any thread,
+     *     while an attempt to acquire a write lock will block or fail.</li>
+     *     <li>Zero when neither read lock, neither write lock is acquired by any thread. This state allows
+     *     a thread to acquire either read or write lock at will.</li>
+     *     <li>-1 when write lock is acquired by exactly one thread. In such a state, any attempt to acquire a read
+     *     or write lock by any thread (but the thread holding the write lock) will block or fail.</li>
+     * </ul>
+     */
     private volatile int state;
 
-    /** */
+    /**
+     * Number of pending write attempts to acquire a write lock. Currently it is only used to prioritize
+     * write lock attempts over read lock attempts when a write lock is released (so, if both an attempt to acquire
+     * write lock and an attempt to acquire read lock are waiting for write lock to be released, write lock attempt
+     * will be served first when the release happens).
+     */
     private volatile int pendingWLocks;
 
-    /** */
-    private long writeLockOwner = -1;
+    /** ID of the thread holding write lock (or -1 if the write lock is not held). */

Review comment:
       ```suggestion
       /** ID of the thread holding write lock (or {@link #NO_OWNER} if the write lock is not held). */
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {
             return 0;
         }
     };
 
-    /** */
+    /**
+     * Main state of the lock.
+     * <ul>
+     *     <li>Positive when read lock is acquired by at least one thread; in such a case, this number equals
+     *     to the number of threads holding the read lock. In such a state, read lock may be acquired by any thread,
+     *     while an attempt to acquire a write lock will block or fail.</li>
+     *     <li>Zero when neither read lock, neither write lock is acquired by any thread. This state allows
+     *     a thread to acquire either read or write lock at will.</li>
+     *     <li>-1 when write lock is acquired by exactly one thread. In such a state, any attempt to acquire a read
+     *     or write lock by any thread (but the thread holding the write lock) will block or fail.</li>

Review comment:
       ```suggestion
        *     or the write lock by any thread (but the thread holding the write lock) will block or fail.</li>
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {
             return 0;
         }
     };
 
-    /** */
+    /**
+     * Main state of the lock.
+     * <ul>
+     *     <li>Positive when read lock is acquired by at least one thread; in such a case, this number equals
+     *     to the number of threads holding the read lock. In such a state, read lock may be acquired by any thread,
+     *     while an attempt to acquire a write lock will block or fail.</li>
+     *     <li>Zero when neither read lock, neither write lock is acquired by any thread. This state allows

Review comment:
       ```suggestion
        *     <li>Zero when neither the read lock, nor the write lock has been acquired by any thread. This state allows
   ```

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/Poller.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.util.function.BooleanSupplier;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import static java.time.Instant.now;
+
+/**
+ * Allows to poll till some condition is met, with timeout.
+ */
+public class Poller {

Review comment:
       I'm sorry, but I think that this class is a massive overkill: it is only used in one place to check the `pendingWriteLocksCount` value. I think that you can simply use `IgniteTestUtils.waitForCondition` instead

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {
             return 0;
         }
     };
 
-    /** */
+    /**
+     * Main state of the lock.
+     * <ul>
+     *     <li>Positive when read lock is acquired by at least one thread; in such a case, this number equals
+     *     to the number of threads holding the read lock. In such a state, read lock may be acquired by any thread,
+     *     while an attempt to acquire a write lock will block or fail.</li>
+     *     <li>Zero when neither read lock, neither write lock is acquired by any thread. This state allows
+     *     a thread to acquire either read or write lock at will.</li>
+     *     <li>-1 when write lock is acquired by exactly one thread. In such a state, any attempt to acquire a read
+     *     or write lock by any thread (but the thread holding the write lock) will block or fail.</li>
+     * </ul>
+     */
     private volatile int state;
 
-    /** */
+    /**
+     * Number of pending write attempts to acquire a write lock. Currently it is only used to prioritize
+     * write lock attempts over read lock attempts when a write lock is released (so, if both an attempt to acquire

Review comment:
       ```suggestion
        * write lock attempts over read lock attempts when the write lock has been released (so, if both an attempt to acquire
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -110,29 +155,55 @@ public void readLock() {
     }
 
     /**
-     * Tries to acquire read lock.
+     * Whether current thread already holds any lock.

Review comment:
       ```suggestion
        * Whether the current thread already holds any lock.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {
             return 0;
         }
     };
 
-    /** */
+    /**
+     * Main state of the lock.
+     * <ul>
+     *     <li>Positive when read lock is acquired by at least one thread; in such a case, this number equals
+     *     to the number of threads holding the read lock. In such a state, read lock may be acquired by any thread,
+     *     while an attempt to acquire a write lock will block or fail.</li>

Review comment:
       ```suggestion
        *     while an attempt to acquire the write lock will block or fail.</li>
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {
             return 0;
         }
     };
 
-    /** */
+    /**
+     * Main state of the lock.
+     * <ul>
+     *     <li>Positive when read lock is acquired by at least one thread; in such a case, this number equals
+     *     to the number of threads holding the read lock. In such a state, read lock may be acquired by any thread,
+     *     while an attempt to acquire a write lock will block or fail.</li>
+     *     <li>Zero when neither read lock, neither write lock is acquired by any thread. This state allows
+     *     a thread to acquire either read or write lock at will.</li>

Review comment:
       ```suggestion
        *     a thread to acquire either the read or the write lock at will.</li>
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -172,101 +244,107 @@ public void readUnlock() {
     }
 
     /**
-     * Acquires write lock.
+     * Acquires write lock waiting, if needed. The thread will block until all other threads release both read

Review comment:
       ```suggestion
        * Acquires the write lock waiting, if needed. The thread will block until all other threads release both read
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {
             return 0;
         }
     };
 
-    /** */
+    /**
+     * Main state of the lock.
+     * <ul>
+     *     <li>Positive when read lock is acquired by at least one thread; in such a case, this number equals
+     *     to the number of threads holding the read lock. In such a state, read lock may be acquired by any thread,
+     *     while an attempt to acquire a write lock will block or fail.</li>
+     *     <li>Zero when neither read lock, neither write lock is acquired by any thread. This state allows
+     *     a thread to acquire either read or write lock at will.</li>
+     *     <li>-1 when write lock is acquired by exactly one thread. In such a state, any attempt to acquire a read
+     *     or write lock by any thread (but the thread holding the write lock) will block or fail.</li>
+     * </ul>
+     */
     private volatile int state;
 
-    /** */
+    /**
+     * Number of pending write attempts to acquire a write lock. Currently it is only used to prioritize
+     * write lock attempts over read lock attempts when a write lock is released (so, if both an attempt to acquire
+     * write lock and an attempt to acquire read lock are waiting for write lock to be released, write lock attempt
+     * will be served first when the release happens).
+     */
     private volatile int pendingWLocks;
 
-    /** */
-    private long writeLockOwner = -1;
+    /** ID of the thread holding write lock (or -1 if the write lock is not held). */
+    private long writeLockOwner = NO_OWNER;
 
-    /** */
+    /** Number of times the write lock holder locked the write lock (used to track reentrance). */
     private int writeLockEntryCnt;
 
     /**
-     * Acquires read lock.
+     * Acquires read lock. If write lock is held by another thread, this blocks until the write lock is released

Review comment:
       ```suggestion
        * Acquires the read lock. If the write lock is held by another thread, this blocks until the write lock is released
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -110,29 +155,55 @@ public void readLock() {
     }
 
     /**
-     * Tries to acquire read lock.
+     * Whether current thread already holds any lock.
      *
-     * @return {@code true} if acquired.
+     * @param currentThreadReadLockAcquiredCount how many times current thread acquired (without releasing yet)
+     *                                           the read lock
+     * @return true if current thread already holds any lock
+     */
+    private boolean alreadyHoldingAnyLock(int currentThreadReadLockAcquiredCount) {
+        return currentThreadReadLockAcquiredCount > 0 || writeLockedByCurrentThread();
+    }
+
+    /***/
+    private void incrementCurrentThreadReadLockCount(int cnt) {
+        assert state > 0 || state == WRITE_LOCKED;
+
+        readLockEntryCnt.set(cnt + 1);
+    }
+
+    /***/
+    private boolean writeLockedOrGoingToBe(int curState) {
+        return curState == WRITE_LOCKED || pendingWLocks > 0;
+    }
+
+    /***/
+    private boolean tryAdvanceStateToReadLocked(int curState) {
+        return compareAndSet(STATE_VH, curState, curState + 1);
+    }
+
+    /**
+     * Tries to acquire read lock. No spinwait is used if the lock cannot be acquired immediately.

Review comment:
       ```suggestion
        * Tries to acquire the read lock. No spinwait is used if the lock cannot be acquired immediately.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -17,77 +17,122 @@
 
 package org.apache.ignite.internal.util;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.ignite.internal.tostring.S;
 
 /**
- * Spin read-write lock.
+ * <p>Spin read-write lock.
+ * Its blocking methods use spinwait strategy. When they do so, they are not interruptible (that is, they do not
+ * break their loop on interruption signal).
+ * </p>
+ * <p>The locks are reentrant (that is, the same thread can acquire the same lock a few times in a row and then
+ * release them same number of times.</p>
+ * <p>
+ * Write lock acquire requests are prioritized over read lock acquire requests. That is, if both read and write lock
+ * acquire requests are received when the write lock is held by someone else, then, on its release, write lock attempt
+ * will be served first.
+ * </p>
  */
 public class IgniteSpinReadWriteLock {
-    /** */
-    private static final long PENDING_WLOCKS_OFFS;
+    /** Signals that nobody currently owns read lock. */
+    private static final long NO_OWNER = -1;
 
-    /** */
-    private static final long STATE_OFFS;
+    /**
+     * State -1 means that the write lock is acquired.
+     *
+     * @see #state
+     */
+    private static final int WRITE_LOCKED = -1;
 
     /**
-     * TODO: replace UNSAFE usage using VarHandle https://issues.apache.org/jira/browse/IGNITE-15536
+     * State 0 means that both read and write locks are available for acquiring.
+     *
+     * @see #state
      */
+    private static final int AVAILABLE = 0;
+
+    /** {@link VarHandle} used to access pendingWLocks field. */
+    private static final VarHandle PENDING_WLOCKS_VH;
+
+    /** {@link VarHandle} used to access state field. */
+    private static final VarHandle STATE_VH;
+
     static {
         try {
-            STATE_OFFS = GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "state", int.class);
 
-            PENDING_WLOCKS_OFFS =
-                GridUnsafe.objectFieldOffset(IgniteSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+            PENDING_WLOCKS_VH = MethodHandles.lookup()
+                    .findVarHandle(IgniteSpinReadWriteLock.class, "pendingWLocks", int.class);
         }
-        catch (NoSuchFieldException e) {
+        catch (ReflectiveOperationException e) {
             throw new Error(e);
         }
     }
 
-    /** */
-    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<Integer>() {
-        @Override protected Integer initialValue() {
+    /** Number of times read lock was acquired, per thread (used to track reentrance). */
+    private final ThreadLocal<Integer> readLockEntryCnt = new ThreadLocal<>() {
+        @Override
+        protected Integer initialValue() {
             return 0;
         }
     };
 
-    /** */
+    /**
+     * Main state of the lock.
+     * <ul>
+     *     <li>Positive when read lock is acquired by at least one thread; in such a case, this number equals
+     *     to the number of threads holding the read lock. In such a state, read lock may be acquired by any thread,
+     *     while an attempt to acquire a write lock will block or fail.</li>
+     *     <li>Zero when neither read lock, neither write lock is acquired by any thread. This state allows
+     *     a thread to acquire either read or write lock at will.</li>
+     *     <li>-1 when write lock is acquired by exactly one thread. In such a state, any attempt to acquire a read
+     *     or write lock by any thread (but the thread holding the write lock) will block or fail.</li>
+     * </ul>
+     */
     private volatile int state;
 
-    /** */
+    /**
+     * Number of pending write attempts to acquire a write lock. Currently it is only used to prioritize
+     * write lock attempts over read lock attempts when a write lock is released (so, if both an attempt to acquire
+     * write lock and an attempt to acquire read lock are waiting for write lock to be released, write lock attempt

Review comment:
       ```suggestion
        * the write lock and an attempt to acquire the read lock are waiting for write lock to be released, a write lock attempt
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -172,101 +244,107 @@ public void readUnlock() {
     }
 
     /**
-     * Acquires write lock.
+     * Acquires write lock waiting, if needed. The thread will block until all other threads release both read
+     * and write locks.
      */
     @SuppressWarnings("BusyWait")
     public void writeLock() {
-        long threadId = Thread.currentThread().getId();
-
-        if (threadId == writeLockOwner) {
-            assert state == -1;
-
-            writeLockEntryCnt++;
+        if (writeLockedByCurrentThread()) {
+            incrementWriteLockCount();
 
             return;
         }
 
-        // Increment pending write locks.
-        while (true) {
-            int pendingWLocks0 = pendingWLocks;
-
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 + 1))
-                break;
-        }
-
         boolean interrupted = false;
 
-        while (!compareAndSet(STATE_OFFS, 0, -1)) {
-            try {
-                Thread.sleep(10);
-            }
-            catch (InterruptedException ignored) {
-                interrupted = true;
+        incrementPendingWriteLocks();
+        try {
+            while (!trySwitchStateToWriteLocked()) {
+                try {
+                    Thread.sleep(10);

Review comment:
       I think that the sleep value can also be extracted into a constant

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -172,101 +244,107 @@ public void readUnlock() {
     }
 
     /**
-     * Acquires write lock.
+     * Acquires write lock waiting, if needed. The thread will block until all other threads release both read
+     * and write locks.
      */
     @SuppressWarnings("BusyWait")
     public void writeLock() {
-        long threadId = Thread.currentThread().getId();
-
-        if (threadId == writeLockOwner) {
-            assert state == -1;
-
-            writeLockEntryCnt++;
+        if (writeLockedByCurrentThread()) {
+            incrementWriteLockCount();
 
             return;
         }
 
-        // Increment pending write locks.
-        while (true) {
-            int pendingWLocks0 = pendingWLocks;
-
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 + 1))
-                break;
-        }
-
         boolean interrupted = false;
 
-        while (!compareAndSet(STATE_OFFS, 0, -1)) {
-            try {
-                Thread.sleep(10);
-            }
-            catch (InterruptedException ignored) {
-                interrupted = true;
+        incrementPendingWriteLocks();
+        try {
+            while (!trySwitchStateToWriteLocked()) {
+                try {
+                    Thread.sleep(10);
+                }
+                catch (InterruptedException ignored) {
+                    interrupted = true;
+                }
             }
         }
-
-        // Decrement pending write locks.
-        while (true) {
-            int pendingWLocks0 = pendingWLocks;
-
-            assert pendingWLocks0 > 0;
-
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 - 1))
-                break;
+        finally {
+            decrementPendingWriteLocks();
         }
 
         if (interrupted)
             Thread.currentThread().interrupt();
 
-        assert writeLockOwner == -1;
-
-        writeLockOwner = threadId;
-        writeLockEntryCnt = 1;
+        finishWriteLockAckquire();
     }
 
-    /**
-     * Acquires write lock without sleeping between unsuccessful attempts.
-     */
-    public void writeLock0() {
-        long threadId = Thread.currentThread().getId();
-
-        if (threadId == writeLockOwner) {
-            assert state == -1;
-
-            writeLockEntryCnt++;
+    /***/
+    private void incrementWriteLockCount() {
+        assert state == WRITE_LOCKED;
 
-            return;
-        }
+        writeLockEntryCnt++;
+    }
 
-        // Increment pending write locks.
+    /***/
+    private void incrementPendingWriteLocks() {
         while (true) {
-            int pendingWLocks0 = pendingWLocks;
+            int curPendingWLocks = pendingWLocks;
 
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 + 1))
+            if (compareAndSet(PENDING_WLOCKS_VH, curPendingWLocks, curPendingWLocks + 1))
                 break;
         }
+    }
 
-        for (;;) {
-            if (compareAndSet(STATE_OFFS, 0, -1))
-                break;
-        }
+    /***/
+    private boolean trySwitchStateToWriteLocked() {
+        return compareAndSet(STATE_VH, AVAILABLE, WRITE_LOCKED);
+    }
 
-        // Decrement pending write locks.
+    /***/
+    private void decrementPendingWriteLocks() {
         while (true) {
-            int pendingWLocks0 = pendingWLocks;
+            int curPendingWLocks = pendingWLocks;
 
-            assert pendingWLocks0 > 0;
+            assert curPendingWLocks > 0;
 
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 - 1))
+            if (compareAndSet(PENDING_WLOCKS_VH, curPendingWLocks, curPendingWLocks - 1))
                 break;
         }
+    }
 
-        assert writeLockOwner == -1;
+    /***/
+    private void finishWriteLockAckquire() {
+        assert writeLockOwner == NO_OWNER;
 
-        writeLockOwner = threadId;
+        writeLockOwner = Thread.currentThread().getId();
         writeLockEntryCnt = 1;
     }
 
+    /**
+     * Acquires write lock without sleeping between unsuccessful attempts. Instead, the spinwait eats cycles of

Review comment:
       ```suggestion
        * Acquires the write lock without sleeping between unsuccessful attempts. Instead, the spinwait eats cycles of
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -275,26 +353,20 @@ public boolean writeLockedByCurrentThread() {
     }
 
     /**
-     * Tries to acquire write lock.
+     * Tries to acquire write lock. Never blocks: if any lock is already acquired by someone else,

Review comment:
       ```suggestion
        * Tries to acquire the write lock. Never blocks: if any lock has already been acquired by someone else,
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -303,42 +375,32 @@ public boolean tryWriteLock() {
     }
 
     /**
+     * Tries to acquire the write lock with timeout. If it gets the write lock before the timeout expires, then
+     * returns {@code true}. If the timeout expires before the lock becomes available, returns {@code false}.
+     *
      * @param timeout Timeout.
      * @param unit Unit.
-     * @return {@code True} if write lock has been acquired.
+     * @return {@code true} if write lock has been acquired in time; {@code false} otherwise

Review comment:
       ```suggestion
        * @return {@code true} if the write lock has been acquired in time; {@code false} otherwise
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -275,26 +353,20 @@ public boolean writeLockedByCurrentThread() {
     }
 
     /**
-     * Tries to acquire write lock.
+     * Tries to acquire write lock. Never blocks: if any lock is already acquired by someone else,
+     * returns {@code false} immediately.
      *
-     * @return {@code True} if write lock has been acquired.
+     * @return {@code true} if write lock has been acquired, {@code false} otherwise

Review comment:
       ```suggestion
        * @return {@code true} if the write lock has been acquired, {@code false} otherwise
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -303,42 +375,32 @@ public boolean tryWriteLock() {
     }
 
     /**
+     * Tries to acquire the write lock with timeout. If it gets the write lock before the timeout expires, then
+     * returns {@code true}. If the timeout expires before the lock becomes available, returns {@code false}.
+     *
      * @param timeout Timeout.
      * @param unit Unit.
-     * @return {@code True} if write lock has been acquired.
+     * @return {@code true} if write lock has been acquired in time; {@code false} otherwise
      * @throws InterruptedException If interrupted.
      */
     @SuppressWarnings("BusyWait")
     public boolean tryWriteLock(long timeout, TimeUnit unit) throws InterruptedException {
-        long threadId = Thread.currentThread().getId();
-
-        if (threadId == writeLockOwner) {
-            assert state == -1;
-
-            writeLockEntryCnt++;
+        if (writeLockedByCurrentThread()) {
+            incrementWriteLockCount();
 
             return true;
         }
 
+        incrementPendingWriteLocks();
         try {
-            // Increment pending write locks.
-            while (true) {
-                int pendingWLocks0 = pendingWLocks;
-
-                if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 + 1))
-                    break;
-            }
 

Review comment:
       ```suggestion
   ```

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLockTest.java
##########
@@ -0,0 +1,530 @@
+/*
+ * 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.util;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests for {@link IgniteSpinReadWriteLock}.
+ */
+@Timeout(20)
+class IgniteSpinReadWriteLockTest {
+    /** The lock under test. */
+    private final IgniteSpinReadWriteLock lock = new IgniteSpinReadWriteLock();
+
+    /** Poller used to poll for 10 seconds. */
+    private final Poller poller10Sec = new Poller(Duration.ofSeconds(10), Duration.ofMillis(10));
+
+    /** Executor service used to run tasks in threads different from the main test thread. */
+    private final ExecutorService executor = Executors.newCachedThreadPool();
+
+    /**
+     * Cleans up after a test.
+     *
+     * @throws Exception if something goes wrong
+     */
+    @AfterEach
+    void cleanup() throws Exception {
+        releaseReadLockHeldByCurrentThread();
+        releaseWriteLockHeldByCurrentThread();
+
+        executor.shutdownNow();

Review comment:
       We have a utility method for closing executors, see `IgniteUtils#shutdownAndAwaitTermination`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -350,25 +412,15 @@ public boolean tryWriteLock(long timeout, TimeUnit unit) throws InterruptedExcep
             }
         }
         finally {
-            // Decrement pending write locks.
-            while (true) {
-                int pendingWLocks0 = pendingWLocks;
-
-                assert pendingWLocks0 > 0;
-
-                if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 - 1))
-                    break;
-            }
+            decrementPendingWriteLocks();
         }
     }
 
     /**
      * Releases write lock.

Review comment:
       ```suggestion
        * Releases the write lock.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -172,101 +244,107 @@ public void readUnlock() {
     }
 
     /**
-     * Acquires write lock.
+     * Acquires write lock waiting, if needed. The thread will block until all other threads release both read
+     * and write locks.
      */
     @SuppressWarnings("BusyWait")
     public void writeLock() {
-        long threadId = Thread.currentThread().getId();
-
-        if (threadId == writeLockOwner) {
-            assert state == -1;
-
-            writeLockEntryCnt++;
+        if (writeLockedByCurrentThread()) {
+            incrementWriteLockCount();
 
             return;
         }
 
-        // Increment pending write locks.
-        while (true) {
-            int pendingWLocks0 = pendingWLocks;
-
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 + 1))
-                break;
-        }
-
         boolean interrupted = false;
 
-        while (!compareAndSet(STATE_OFFS, 0, -1)) {
-            try {
-                Thread.sleep(10);
-            }
-            catch (InterruptedException ignored) {
-                interrupted = true;
+        incrementPendingWriteLocks();
+        try {
+            while (!trySwitchStateToWriteLocked()) {
+                try {
+                    Thread.sleep(10);
+                }
+                catch (InterruptedException ignored) {
+                    interrupted = true;
+                }
             }
         }
-
-        // Decrement pending write locks.
-        while (true) {
-            int pendingWLocks0 = pendingWLocks;
-
-            assert pendingWLocks0 > 0;
-
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 - 1))
-                break;
+        finally {
+            decrementPendingWriteLocks();
         }
 
         if (interrupted)
             Thread.currentThread().interrupt();
 
-        assert writeLockOwner == -1;
-
-        writeLockOwner = threadId;
-        writeLockEntryCnt = 1;
+        finishWriteLockAckquire();
     }
 
-    /**
-     * Acquires write lock without sleeping between unsuccessful attempts.
-     */
-    public void writeLock0() {
-        long threadId = Thread.currentThread().getId();
-
-        if (threadId == writeLockOwner) {
-            assert state == -1;
-
-            writeLockEntryCnt++;
+    /***/
+    private void incrementWriteLockCount() {
+        assert state == WRITE_LOCKED;
 
-            return;
-        }
+        writeLockEntryCnt++;
+    }
 
-        // Increment pending write locks.
+    /***/
+    private void incrementPendingWriteLocks() {
         while (true) {
-            int pendingWLocks0 = pendingWLocks;
+            int curPendingWLocks = pendingWLocks;
 
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 + 1))
+            if (compareAndSet(PENDING_WLOCKS_VH, curPendingWLocks, curPendingWLocks + 1))
                 break;
         }
+    }
 
-        for (;;) {
-            if (compareAndSet(STATE_OFFS, 0, -1))
-                break;
-        }
+    /***/
+    private boolean trySwitchStateToWriteLocked() {
+        return compareAndSet(STATE_VH, AVAILABLE, WRITE_LOCKED);
+    }
 
-        // Decrement pending write locks.
+    /***/
+    private void decrementPendingWriteLocks() {
         while (true) {
-            int pendingWLocks0 = pendingWLocks;
+            int curPendingWLocks = pendingWLocks;
 
-            assert pendingWLocks0 > 0;
+            assert curPendingWLocks > 0;
 
-            if (compareAndSet(PENDING_WLOCKS_OFFS, pendingWLocks0, pendingWLocks0 - 1))
+            if (compareAndSet(PENDING_WLOCKS_VH, curPendingWLocks, curPendingWLocks - 1))
                 break;
         }
+    }
 
-        assert writeLockOwner == -1;
+    /***/
+    private void finishWriteLockAckquire() {
+        assert writeLockOwner == NO_OWNER;
 
-        writeLockOwner = threadId;
+        writeLockOwner = Thread.currentThread().getId();
         writeLockEntryCnt = 1;
     }
 
+    /**
+     * Acquires write lock without sleeping between unsuccessful attempts. Instead, the spinwait eats cycles of
+     * the core it gets at full speed. It is non-interruptible as its {@link #writeLock()} cousin.
+     */
+    public void writeLock0() {

Review comment:
       I would suggest to rename this method

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLock.java
##########
@@ -141,7 +212,8 @@ public boolean tryReadLock() {
     }
 
     /**
-     * Read unlock.
+     * Releases the read lock.
+     * If the current thread does not hold the read lock, then {@link IllegalMonitorStateException} is thrown.

Review comment:
       I would suggest to move this sentence to the `@throws` section

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLockTest.java
##########
@@ -0,0 +1,530 @@
+/*
+ * 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.util;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests for {@link IgniteSpinReadWriteLock}.
+ */
+@Timeout(20)
+class IgniteSpinReadWriteLockTest {
+    /** The lock under test. */
+    private final IgniteSpinReadWriteLock lock = new IgniteSpinReadWriteLock();
+
+    /** Poller used to poll for 10 seconds. */
+    private final Poller poller10Sec = new Poller(Duration.ofSeconds(10), Duration.ofMillis(10));
+
+    /** Executor service used to run tasks in threads different from the main test thread. */
+    private final ExecutorService executor = Executors.newCachedThreadPool();
+
+    /**
+     * Cleans up after a test.
+     *
+     * @throws Exception if something goes wrong
+     */
+    @AfterEach
+    void cleanup() throws Exception {
+        releaseReadLockHeldByCurrentThread();
+        releaseWriteLockHeldByCurrentThread();
+
+        executor.shutdownNow();
+        executor.awaitTermination(3, TimeUnit.SECONDS);
+    }
+
+    /***/
+    private void releaseReadLockHeldByCurrentThread() {
+        while (true) {
+            try {
+                lock.readUnlock();
+            }
+            catch (IllegalMonitorStateException e) {
+                // released our read lock completely
+                break;
+            }
+        }
+    }
+
+    /***/
+    private void releaseWriteLockHeldByCurrentThread() {
+        while (lock.writeLockedByCurrentThread())
+            lock.writeUnlock();
+    }
+
+    /**
+     *
+     */
+    @Test
+    void readLockDoesNotAllowWriteLockToBeAcquired() {
+        lock.readLock();
+
+        assertThatWriteLockAcquireAttemptBlocksForever();
+
+        lock.readUnlock();
+    }
+
+    /***/
+    private void assertThatWriteLockAcquireAttemptBlocksForever() {
+        Future<?> future = executor.submit(lock::writeLock);
+
+        assertThrows(TimeoutException.class, () -> future.get(500, TimeUnit.MILLISECONDS));
+    }
+
+    /**
+     *
+     */
+    @Test
+    void readLockDoesNotAllowWriteLockWithoutSleepsToBeAcquired() {
+        lock.readLock();
+
+        assertThatWriteLockAcquireAttemptWithoutSleepsBlocksForever();
+
+        lock.readUnlock();
+    }
+
+    /**
+     *
+     */
+    @Test
+    void readLockDoesNotAllowWriteLockToBeAcquiredWithTimeout() throws Exception {
+        lock.readLock();
+
+        Boolean acquired = callWithTimeout(() -> lock.tryWriteLock(1, TimeUnit.MILLISECONDS));
+        assertThat(acquired, is(false));
+
+        lock.readUnlock();
+    }
+
+    /**
+     *
+     */
+    @Test
+    void readLockAllowsReadLockToBeAcquired() throws Exception {
+        lock.readLock();
+
+        assertThatReadLockCanBeAcquired();
+    }
+
+    /***/
+    private void assertThatReadLockCanBeAcquired() throws InterruptedException, ExecutionException, TimeoutException {

Review comment:
       can be replaced with `runWithTimeout(lock::readLock)`

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/Probe.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.util;
+
+/**
+ * A probe used on each iteration of polling at {@link Poller} to sample next element and decide whether polling
+ * needs to be stopped successfully wtth a value.
+ *
+ * @param <T> result type
+ */
+public interface Probe<T> {

Review comment:
       Please see my previous comment

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/IgniteSpinReadWriteLockTest.java
##########
@@ -0,0 +1,530 @@
+/*
+ * 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.util;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests for {@link IgniteSpinReadWriteLock}.
+ */
+@Timeout(20)
+class IgniteSpinReadWriteLockTest {
+    /** The lock under test. */
+    private final IgniteSpinReadWriteLock lock = new IgniteSpinReadWriteLock();
+
+    /** Poller used to poll for 10 seconds. */
+    private final Poller poller10Sec = new Poller(Duration.ofSeconds(10), Duration.ofMillis(10));
+
+    /** Executor service used to run tasks in threads different from the main test thread. */
+    private final ExecutorService executor = Executors.newCachedThreadPool();
+
+    /**
+     * Cleans up after a test.
+     *
+     * @throws Exception if something goes wrong
+     */
+    @AfterEach
+    void cleanup() throws Exception {
+        releaseReadLockHeldByCurrentThread();
+        releaseWriteLockHeldByCurrentThread();
+
+        executor.shutdownNow();
+        executor.awaitTermination(3, TimeUnit.SECONDS);
+    }
+
+    /***/
+    private void releaseReadLockHeldByCurrentThread() {
+        while (true) {
+            try {
+                lock.readUnlock();
+            }
+            catch (IllegalMonitorStateException e) {
+                // released our read lock completely
+                break;
+            }
+        }
+    }
+
+    /***/
+    private void releaseWriteLockHeldByCurrentThread() {
+        while (lock.writeLockedByCurrentThread())
+            lock.writeUnlock();
+    }
+
+    /**
+     *
+     */
+    @Test
+    void readLockDoesNotAllowWriteLockToBeAcquired() {
+        lock.readLock();
+
+        assertThatWriteLockAcquireAttemptBlocksForever();
+
+        lock.readUnlock();
+    }
+
+    /***/
+    private void assertThatWriteLockAcquireAttemptBlocksForever() {
+        Future<?> future = executor.submit(lock::writeLock);
+
+        assertThrows(TimeoutException.class, () -> future.get(500, TimeUnit.MILLISECONDS));
+    }
+
+    /**
+     *
+     */
+    @Test
+    void readLockDoesNotAllowWriteLockWithoutSleepsToBeAcquired() {
+        lock.readLock();
+
+        assertThatWriteLockAcquireAttemptWithoutSleepsBlocksForever();
+
+        lock.readUnlock();
+    }
+
+    /**
+     *
+     */
+    @Test
+    void readLockDoesNotAllowWriteLockToBeAcquiredWithTimeout() throws Exception {
+        lock.readLock();
+
+        Boolean acquired = callWithTimeout(() -> lock.tryWriteLock(1, TimeUnit.MILLISECONDS));
+        assertThat(acquired, is(false));
+
+        lock.readUnlock();
+    }
+
+    /**
+     *
+     */
+    @Test
+    void readLockAllowsReadLockToBeAcquired() throws Exception {
+        lock.readLock();
+
+        assertThatReadLockCanBeAcquired();
+    }
+
+    /***/
+    private void assertThatReadLockCanBeAcquired() throws InterruptedException, ExecutionException, TimeoutException {
+        Future<?> readLockAttemptFuture = executor.submit(lock::readLock);
+        getWithTimeout(readLockAttemptFuture);
+    }
+
+    /***/
+    private <T> T callWithTimeout(Callable<T> call) throws ExecutionException, InterruptedException, TimeoutException {
+        Future<T> future = executor.submit(call);
+        return getWithTimeout(future);
+    }
+
+    /***/
+    private void runWithTimeout(Runnable runnable) throws ExecutionException, InterruptedException, TimeoutException {
+        Future<?> future = executor.submit(runnable);
+        getWithTimeout(future);
+    }
+
+    /***/
+    private <T> T getWithTimeout(Future<? extends T> future) throws ExecutionException,

Review comment:
       ```suggestion
       private static <T> T getWithTimeout(Future<? extends T> future) throws ExecutionException,
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org