You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by il...@apache.org on 2020/12/09 09:20:59 UTC

[ignite] branch master updated: IGNITE-13775 checkpointRWLock wrapper refactoring - Fixes #8516.

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

ilyak pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new be96086  IGNITE-13775 checkpointRWLock wrapper refactoring - Fixes #8516.
be96086 is described below

commit be96086ce0074a4a37b431621e54aace28bf3faf
Author: zstan <st...@gmail.com>
AuthorDate: Wed Dec 9 12:20:34 2020 +0300

    IGNITE-13775 checkpointRWLock wrapper refactoring - Fixes #8516.
    
    Signed-off-by: Ilya Kasnacheev <il...@gmail.com>
---
 .../checkpoint/CheckpointReadWriteLock.java        |  11 +-
 .../apache/ignite/internal/util/IgniteUtils.java   | 179 -----------------
 .../util/ReentrantReadWriteLockWithTracking.java   | 218 +++++++++++++++++++++
 .../persistence/CheckpointReadLockFailureTest.java |  39 +++-
 .../IgnitePdsCacheEntriesExpirationTest.java       |   4 +-
 .../cache/persistence/db/IgnitePdsWithTtlTest.java |   4 +-
 6 files changed, 255 insertions(+), 200 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointReadWriteLock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointReadWriteLock.java
index 2066201..ad4d457 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointReadWriteLock.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointReadWriteLock.java
@@ -18,12 +18,11 @@
 package org.apache.ignite.internal.processors.cache.persistence.checkpoint;
 
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Function;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
-import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.ReentrantReadWriteLockWithTracking;
 
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.IGNITE_PDS_LOG_CP_READ_LOCK_HOLDERS;
@@ -45,18 +44,16 @@ public class CheckpointReadWriteLock {
     static final String CHECKPOINT_RUNNER_THREAD_PREFIX = "checkpoint-runner";
 
     /** Checkpont lock. */
-    private final ReentrantReadWriteLock checkpointLock;
+    private final ReentrantReadWriteLockWithTracking checkpointLock;
 
     /**
      * @param logger Logger.
      */
     CheckpointReadWriteLock(Function<Class<?>, IgniteLogger> logger) {
-        ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-
         if (getBoolean(IGNITE_PDS_LOG_CP_READ_LOCK_HOLDERS))
-            checkpointLock = new U.ReentrantReadWriteLockTracer(lock, logger.apply(getClass()), 5_000);
+            checkpointLock = new ReentrantReadWriteLockWithTracking(logger.apply(getClass()), 5_000);
         else
-            checkpointLock = lock;
+            checkpointLock = new ReentrantReadWriteLockWithTracking();
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index bda05be..2dc9910 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -139,7 +139,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Consumer;
 import java.util.jar.JarFile;
 import java.util.logging.ConsoleHandler;
@@ -242,7 +241,6 @@ import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.P1;
-import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.SB;
@@ -363,9 +361,6 @@ public abstract class IgniteUtils {
     /** Default user version. */
     public static final String DFLT_USER_VERSION = "0";
 
-    /** Lock hold message. */
-    public static final String LOCK_HOLD_MESSAGE = "ReadLock held the lock more than ";
-
     /** Cache for {@link GridPeerDeployAware} fields to speed up reflection. */
     private static final ConcurrentMap<String, IgniteBiTuple<Class<?>, Collection<Field>>> p2pFields =
         new ConcurrentHashMap<>();
@@ -11542,180 +11537,6 @@ public abstract class IgniteUtils {
         };
     }
 
-    /** */
-    public static class ReentrantReadWriteLockTracer extends ReentrantReadWriteLock {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Read lock. */
-        private final ReadLockTracer readLock;
-
-        /** Write lock. */
-        private final WriteLockTracer writeLock;
-
-        /** Lock print threshold. */
-        private long readLockThreshold;
-
-        /** */
-        private IgniteLogger log;
-
-        /**
-         * @param delegate RWLock delegate.
-         * @param log Ignite logger.
-         * @param readLockThreshold ReadLock threshold timeout.
-         *
-         */
-        public ReentrantReadWriteLockTracer(ReentrantReadWriteLock delegate, IgniteLogger log, long readLockThreshold) {
-            this.log = log;
-
-            readLock = new ReadLockTracer(delegate, log, readLockThreshold);
-
-            writeLock = new WriteLockTracer(delegate);
-
-            this.readLockThreshold = readLockThreshold;
-        }
-
-        /** {@inheritDoc} */
-        @Override public ReadLock readLock() {
-            return readLock;
-        }
-
-        /** {@inheritDoc} */
-        @Override public WriteLock writeLock() {
-            return writeLock;
-        }
-
-        /** */
-        public long lockWaitThreshold() {
-            return readLockThreshold;
-        }
-    }
-
-    /** */
-    private static class ReadLockTracer extends ReentrantReadWriteLock.ReadLock {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Delegate. */
-        private final ReentrantReadWriteLock.ReadLock delegate;
-
-        /** */
-        private static final ThreadLocal<T2<Integer, Long>> READ_LOCK_HOLDER_TS =
-            ThreadLocal.withInitial(() -> new T2<>(0, 0L));
-
-        /** */
-        private IgniteLogger log;
-
-        /** */
-        private long readLockThreshold;
-
-        /** */
-        public ReadLockTracer(ReentrantReadWriteLock lock, IgniteLogger log, long readLockThreshold) {
-            super(lock);
-
-            delegate = lock.readLock();
-
-            this.log = log;
-
-            this.readLockThreshold = readLockThreshold;
-        }
-
-        /** */
-        private void inc() {
-            T2<Integer, Long> val = READ_LOCK_HOLDER_TS.get();
-
-            int cntr = val.get1();
-
-            if (cntr == 0)
-                val.set2(U.currentTimeMillis());
-
-            val.set1(++cntr);
-
-            READ_LOCK_HOLDER_TS.set(val);
-        }
-
-        /** */
-        private void dec() {
-            T2<Integer, Long> val = READ_LOCK_HOLDER_TS.get();
-
-            int cntr = val.get1();
-
-            if (--cntr == 0) {
-                long timeout = U.currentTimeMillis() - val.get2();
-
-                if (timeout > readLockThreshold) {
-                    GridStringBuilder sb = new GridStringBuilder();
-
-                    sb.a(LOCK_HOLD_MESSAGE + timeout + " ms." + nl());
-
-                    U.printStackTrace(Thread.currentThread().getId(), sb);
-
-                    U.warn(log, sb.toString());
-                }
-            }
-
-            val.set1(cntr);
-
-            READ_LOCK_HOLDER_TS.set(val);
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("LockAcquiredButNotSafelyReleased")
-        @Override public void lock() {
-            delegate.lock();
-
-            inc();
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("LockAcquiredButNotSafelyReleased")
-        @Override public void lockInterruptibly() throws InterruptedException {
-            delegate.lockInterruptibly();
-
-            inc();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean tryLock() {
-            if (delegate.tryLock()) {
-                inc();
-
-                return true;
-            }
-            else
-                return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean tryLock(long time, @NotNull TimeUnit unit) throws InterruptedException {
-            if (delegate.tryLock(time, unit)) {
-                inc();
-
-                return true;
-            }
-            else
-                return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void unlock() {
-            delegate.unlock();
-
-            dec();
-        }
-    }
-
-    /** */
-    private static class WriteLockTracer extends ReentrantReadWriteLock.WriteLock {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        public WriteLockTracer(ReentrantReadWriteLock lock) {
-            super(lock);
-        }
-    }
-
     /**
      * @param key Cipher Key.
      * @param encMode Enc mode see {@link Cipher#ENCRYPT_MODE}, {@link Cipher#DECRYPT_MODE}, etc.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ReentrantReadWriteLockWithTracking.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ReentrantReadWriteLockWithTracking.java
new file mode 100644
index 0000000..227be47
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ReentrantReadWriteLockWithTracking.java
@@ -0,0 +1,218 @@
+/*
+ * 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.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.util.IgniteUtils.nl;
+
+/** ReentrantReadWriteLock adapter with readLock tracking. */
+public class ReentrantReadWriteLockWithTracking implements ReadWriteLock {
+    /** Lock hold message. */
+    public static final String LOCK_HOLD_MESSAGE = "ReadLock held the lock more than ";
+
+    /** Lock print threshold. */
+    private long readLockThreshold;
+
+    /** Delegate instance. */
+    private final ReentrantReadWriteLock delegate = new ReentrantReadWriteLock();
+
+    /** Read lock holder. */
+    private ReentrantReadWriteLock.ReadLock readLock;
+
+    /** Write lock holder. */
+    private ReentrantReadWriteLock.WriteLock writeLock = new ReentrantReadWriteLock.WriteLock(delegate) {};
+
+    /**
+     * ReentrantRWLock wrapper, provides additional trace info on {@link ReadLockWithTracking#unlock()} method, if someone
+     * holds the lock more than {@code readLockThreshold}.
+     *
+     * @param log Ignite logger.
+     * @param readLockThreshold ReadLock threshold timeout.
+     */
+    public ReentrantReadWriteLockWithTracking(IgniteLogger log, long readLockThreshold) {
+        readLock = new ReadLockWithTracking(delegate, log, readLockThreshold);
+
+        this.readLockThreshold = readLockThreshold;
+    }
+
+    /** Delegator implementation. */
+    public ReentrantReadWriteLockWithTracking() {
+        readLock = new ReentrantReadWriteLock.ReadLock(delegate) {};
+    }
+
+    /** {@inheritDoc} */
+    @Override public ReentrantReadWriteLock.ReadLock readLock() {
+        return readLock;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ReentrantReadWriteLock.WriteLock writeLock() {
+        return writeLock;
+    }
+
+    /** */
+    public long lockWaitThreshold() {
+        return readLockThreshold;
+    }
+
+    /**
+     * Queries if the write lock is held by the current thread.
+     *
+     * @return {@code true} if the current thread holds the write lock and
+     *         {@code false} otherwise
+     */
+    public boolean isWriteLockedByCurrentThread() {
+        return delegate.isWriteLockedByCurrentThread();
+    }
+
+    /**
+     * Queries the number of reentrant read holds on this lock by the
+     * current thread.  A reader thread has a hold on a lock for
+     * each lock action that is not matched by an unlock action.
+     *
+     * @return the number of holds on the read lock by the current thread,
+     *         or zero if the read lock is not held by the current thread
+     */
+    public int getReadHoldCount() {
+        return delegate.getReadHoldCount();
+    }
+
+    /**
+     * Queries the number of read locks held for this lock. This
+     * method is designed for use in monitoring system state, not for
+     * synchronization control.
+     * @return the number of read locks held
+     */
+    public int getReadLockCount() {
+        return delegate.getReadLockCount();
+    }
+
+    /** Tracks long rlock holders. */
+    public static class ReadLockWithTracking extends ReentrantReadWriteLock.ReadLock {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private static final ThreadLocal<T2<Integer, Long>> READ_LOCK_HOLDER_TS =
+            ThreadLocal.withInitial(() -> new T2<>(0, 0L));
+
+        /** */
+        private IgniteLogger log;
+
+        /** */
+        private long readLockThreshold;
+
+        /** */
+        protected ReadLockWithTracking(ReentrantReadWriteLock lock, @Nullable IgniteLogger log, long readLockThreshold) {
+            super(lock);
+
+            this.log = log;
+
+            this.readLockThreshold = readLockThreshold;
+        }
+
+        /** */
+        private void inc() {
+            T2<Integer, Long> val = READ_LOCK_HOLDER_TS.get();
+
+            int cntr = val.get1();
+
+            if (cntr == 0)
+                val.set2(U.currentTimeMillis());
+
+            val.set1(++cntr);
+
+            READ_LOCK_HOLDER_TS.set(val);
+        }
+
+        /** */
+        private void dec() {
+            T2<Integer, Long> val = READ_LOCK_HOLDER_TS.get();
+
+            int cntr = val.get1();
+
+            if (--cntr == 0) {
+                long timeout = U.currentTimeMillis() - val.get2();
+
+                if (timeout > readLockThreshold) {
+                    GridStringBuilder sb = new GridStringBuilder();
+
+                    sb.a(LOCK_HOLD_MESSAGE + timeout + " ms." + nl());
+
+                    U.printStackTrace(Thread.currentThread().getId(), sb);
+
+                    U.warn(log, sb.toString());
+                }
+            }
+
+            val.set1(cntr);
+
+            READ_LOCK_HOLDER_TS.set(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void lock() {
+            super.lock();
+
+            inc();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void lockInterruptibly() throws InterruptedException {
+            super.lockInterruptibly();
+
+            inc();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean tryLock() {
+            if (super.tryLock()) {
+                inc();
+
+                return true;
+            }
+            else
+                return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean tryLock(long timeout, TimeUnit unit) throws InterruptedException {
+            if (super.tryLock(timeout, unit)) {
+                inc();
+
+                return true;
+            }
+            else
+                return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void unlock() {
+            super.unlock();
+
+            dec();
+        }
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/CheckpointReadLockFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/CheckpointReadLockFailureTest.java
index f761387..a1ebfb4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/CheckpointReadLockFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/CheckpointReadLockFailureTest.java
@@ -21,8 +21,8 @@ import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointReadWriteLock;
+import org.apache.ignite.internal.util.ReentrantReadWriteLockWithTracking;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.ListeningTestLogger;
@@ -45,7 +46,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.IGNITE_PDS_LOG_CP_READ_LOCK_HOLDERS;
-import static org.apache.ignite.internal.util.IgniteUtils.LOCK_HOLD_MESSAGE;
+import static org.apache.ignite.internal.util.ReentrantReadWriteLockWithTracking.LOCK_HOLD_MESSAGE;
 
 /**
  * Tests critical failure handling on checkpoint read lock acquisition errors.
@@ -125,7 +126,7 @@ public class CheckpointReadLockFailureTest extends GridCommonAbstractTest {
 
         IgniteEx ig = startGrid(0);
 
-        ig.cluster().active(true);
+        ig.cluster().state(ClusterState.ACTIVE);
 
         GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ig.context().cache().context().database();
 
@@ -168,7 +169,7 @@ public class CheckpointReadLockFailureTest extends GridCommonAbstractTest {
     public void testPrintCpRLockHolder() throws Exception {
         CountDownLatch canRelease = new CountDownLatch(1);
 
-        testLog = new ListeningTestLogger(false, log);
+        testLog = new ListeningTestLogger(log);
 
         LogListener lsnr = LogListener.matches(LOCK_HOLD_MESSAGE).build();
 
@@ -176,14 +177,14 @@ public class CheckpointReadLockFailureTest extends GridCommonAbstractTest {
 
         IgniteEx ig = startGrid(0);
 
-        ig.cluster().active(true);
+        ig.cluster().state(ClusterState.ACTIVE);
 
         GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ig.context().cache().context().database();
 
         CheckpointReadWriteLock checkpointReadWriteLock = U.field(
             db.checkpointManager.checkpointTimeoutLock(), "checkpointReadWriteLock"
         );
-        U.ReentrantReadWriteLockTracer tracker = U.field(checkpointReadWriteLock, "checkpointLock");
+        ReentrantReadWriteLockWithTracking tracker = U.field(checkpointReadWriteLock, "checkpointLock");
 
         GridTestUtils.runAsync(() -> {
             checkpointReadWriteLock.readLock();
@@ -212,7 +213,7 @@ public class CheckpointReadLockFailureTest extends GridCommonAbstractTest {
     public void testReentrance() throws Exception {
         IgniteEx ig = startGrid(0);
 
-        ig.cluster().active(true);
+        ig.cluster().state(ClusterState.ACTIVE);
 
         GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ig.context().cache().context().database();
 
@@ -220,7 +221,7 @@ public class CheckpointReadLockFailureTest extends GridCommonAbstractTest {
             db.checkpointManager.checkpointTimeoutLock(), "checkpointReadWriteLock"
         );
 
-        ReentrantReadWriteLock rwLock = U.field(checkpointReadWriteLock, "checkpointLock");
+        ReentrantReadWriteLockWithTracking rwLock = U.field(checkpointReadWriteLock, "checkpointLock");
 
         CountDownLatch waitFirstRLock = new CountDownLatch(1);
 
@@ -243,7 +244,7 @@ public class CheckpointReadLockFailureTest extends GridCommonAbstractTest {
                 waitSecondRLock.await();
             }
             catch (InterruptedException e) {
-                e.printStackTrace();
+                fail(e.toString());
             }
 
             rwLock.readLock().unlock();
@@ -254,7 +255,7 @@ public class CheckpointReadLockFailureTest extends GridCommonAbstractTest {
                 waitFirstRLock.await();
             }
             catch (InterruptedException e) {
-                e.printStackTrace();
+                fail(e.toString());
             }
 
             try {
@@ -288,4 +289,22 @@ public class CheckpointReadLockFailureTest extends GridCommonAbstractTest {
 
         stopGrid(0);
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    @WithSystemProperty(key = IGNITE_PDS_LOG_CP_READ_LOCK_HOLDERS, value = "true")
+    public void testWriteLockedByCurrentThread() {
+        ReentrantReadWriteLockWithTracking wrapped = new ReentrantReadWriteLockWithTracking(log, 1_000);
+
+        wrapped.writeLock().lock();
+
+        try {
+            assertTrue(wrapped.isWriteLockedByCurrentThread());
+        }
+        finally {
+            wrapped.writeLock().unlock();
+        }
+    }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheEntriesExpirationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheEntriesExpirationTest.java
index 74ac211..986c680 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheEntriesExpirationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheEntriesExpirationTest.java
@@ -21,7 +21,6 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 import javax.cache.expiry.CreatedExpiryPolicy;
 import javax.cache.expiry.Duration;
 import org.apache.ignite.IgniteCache;
@@ -44,6 +43,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.Grid
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl;
 import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointReadWriteLock;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.ReentrantReadWriteLockWithTracking;
 import org.apache.ignite.internal.util.lang.IgniteInClosure2X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -243,7 +243,7 @@ public class IgnitePdsCacheEntriesExpirationTest extends GridCommonAbstractTest
             db.checkpointManager.checkpointTimeoutLock(), "checkpointReadWriteLock"
         );
 
-        ReentrantReadWriteLock rwLock = U.field(checkpointReadWriteLock, "checkpointLock");
+        ReentrantReadWriteLockWithTracking rwLock = U.field(checkpointReadWriteLock, "checkpointLock");
 
         int key = 0;
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java
index 47351df..f502e58 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java
@@ -23,7 +23,6 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 import javax.cache.expiry.AccessedExpiryPolicy;
 import javax.cache.expiry.CreatedExpiryPolicy;
 import javax.cache.expiry.Duration;
@@ -55,6 +54,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.util.ReentrantReadWriteLockWithTracking;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.lang.GridCursor;
 import org.apache.ignite.internal.util.typedef.F;
@@ -286,7 +286,7 @@ public class IgnitePdsWithTtlTest extends GridCommonAbstractTest {
                 checkpointManager.checkpointTimeoutLock(), "checkpointReadWriteLock"
             );
 
-            ReentrantReadWriteLock lock = U.field(checkpointReadWriteLock, "checkpointLock");
+            ReentrantReadWriteLockWithTracking lock = U.field(checkpointReadWriteLock, "checkpointLock");
 
             while (!timeoutReached.get()) {
                 try {