summaryrefslogtreecommitdiffstats
path: root/zkfacade
diff options
context:
space:
mode:
authorHåkon Hallingstad <hakon@verizonmedia.com>2020-09-28 13:49:49 +0200
committerHåkon Hallingstad <hakon@verizonmedia.com>2020-09-28 13:49:49 +0200
commit4fd2db62039bf5fed95dcb71bd3ff59267812c7c (patch)
treee87e51758151beaf7b8ea386de928f59dd8f9791 /zkfacade
parent6e284ef06a2e6dafbaeae8e486b6f68fa53d5d48 (diff)
LockInfo -> LockAttempt, ThreadLockInfo -> ThreadLockStats, and more
Diffstat (limited to 'zkfacade')
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/Lock.java14
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttempt.java (renamed from zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockInfo.java)16
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttemptSamples.java (renamed from zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockInfoSamples.java)50
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/ThreadLockStats.java (renamed from zkfacade/src/main/java/com/yahoo/vespa/curator/stats/ThreadLockInfo.java)54
-rw-r--r--zkfacade/src/test/java/com/yahoo/vespa/curator/stats/LockAttemptSamplesTest.java (renamed from zkfacade/src/test/java/com/yahoo/vespa/curator/stats/LockInfoSamplesTest.java)24
-rw-r--r--zkfacade/src/test/java/com/yahoo/vespa/curator/stats/LockTest.java56
6 files changed, 111 insertions, 103 deletions
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/Lock.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/Lock.java
index b630995d6b4..6e3b79ec5ce 100644
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/Lock.java
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/Lock.java
@@ -4,7 +4,7 @@ package com.yahoo.vespa.curator;
import com.google.common.util.concurrent.UncheckedTimeoutException;
import com.yahoo.path.Path;
import com.yahoo.transaction.Mutex;
-import com.yahoo.vespa.curator.stats.ThreadLockInfo;
+import com.yahoo.vespa.curator.stats.ThreadLockStats;
import org.apache.curator.framework.recipes.locks.InterProcessLock;
import java.time.Duration;
@@ -35,28 +35,28 @@ public class Lock implements Mutex {
/** Take the lock with the given timeout. This may be called multiple times from the same thread - each matched by a close */
public void acquire(Duration timeout) throws UncheckedTimeoutException {
- ThreadLockInfo threadLockInfo = ThreadLockInfo.getCurrentThreadLockInfo();
- threadLockInfo.invokingAcquire(lockPath, timeout);
+ ThreadLockStats threadLockStats = ThreadLockStats.getCurrentThreadLockInfo();
+ threadLockStats.invokingAcquire(lockPath, timeout);
final boolean acquired;
try {
acquired = mutex.acquire(timeout.toMillis(), TimeUnit.MILLISECONDS);
} catch (Exception e) {
- threadLockInfo.acquireFailed(lockPath);
+ threadLockStats.acquireFailed(lockPath);
throw new RuntimeException("Exception acquiring lock '" + lockPath + "'", e);
}
if (!acquired) {
- threadLockInfo.acquireTimedOut(lockPath);
+ threadLockStats.acquireTimedOut(lockPath);
throw new UncheckedTimeoutException("Timed out after waiting " + timeout +
" to acquire lock '" + lockPath + "'");
}
- threadLockInfo.lockAcquired(lockPath);
+ threadLockStats.lockAcquired(lockPath);
}
@Override
public void close() {
- ThreadLockInfo.getCurrentThreadLockInfo().lockReleased(lockPath);
+ ThreadLockStats.getCurrentThreadLockInfo().lockReleased(lockPath);
try {
mutex.release();
}
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockInfo.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttempt.java
index f40c6f0498f..c092fb6c289 100644
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockInfo.java
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttempt.java
@@ -13,9 +13,9 @@ import java.util.Optional;
*
* @author hakon
*/
-public class LockInfo {
+public class LockAttempt {
- private final ThreadLockInfo threadLockInfo;
+ private final ThreadLockStats threadLockStats;
private final String lockPath;
private final Instant callAcquireInstant;
private final Duration timeout;
@@ -24,8 +24,8 @@ public class LockInfo {
private volatile Optional<Instant> terminalStateInstant = Optional.empty();
private volatile Optional<String> stackTrace = Optional.empty();
- public static LockInfo invokingAcquire(ThreadLockInfo threadLockInfo, String lockPath, Duration timeout) {
- return new LockInfo(threadLockInfo, lockPath, timeout, Instant.now());
+ public static LockAttempt invokingAcquire(ThreadLockStats threadLockStats, String lockPath, Duration timeout) {
+ return new LockAttempt(threadLockStats, lockPath, timeout, Instant.now());
}
public enum LockState {
@@ -40,14 +40,14 @@ public class LockInfo {
private volatile LockState lockState = LockState.ACQUIRING;
- private LockInfo(ThreadLockInfo threadLockInfo, String lockPath, Duration timeout, Instant callAcquireInstant) {
- this.threadLockInfo = threadLockInfo;
+ private LockAttempt(ThreadLockStats threadLockStats, String lockPath, Duration timeout, Instant callAcquireInstant) {
+ this.threadLockStats = threadLockStats;
this.lockPath = lockPath;
this.callAcquireInstant = callAcquireInstant;
this.timeout = timeout;
}
- public String getThreadName() { return threadLockInfo.getThreadName(); }
+ public String getThreadName() { return threadLockStats.getThreadName(); }
public String getLockPath() { return lockPath; }
public Instant getTimeAcquiredWasInvoked() { return callAcquireInstant; }
public Duration getAcquireTimeout() { return timeout; }
@@ -78,7 +78,7 @@ public class LockInfo {
// This method is public. If invoked concurrently, the this.stackTrace may be updated twice,
// which is fine.
- this.stackTrace = Optional.of(threadLockInfo.getStackTrace());
+ this.stackTrace = Optional.of(threadLockStats.getStackTrace());
}
void acquireFailed() { setTerminalState(LockState.ACQUIRE_FAILED); }
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockInfoSamples.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttemptSamples.java
index ae84c5c984a..54cb82ebc1e 100644
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockInfoSamples.java
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttemptSamples.java
@@ -8,42 +8,42 @@ import java.util.Map;
import java.util.PriorityQueue;
/**
- * Collection containing "interesting" {@code LockInfo}s.
+ * Collection containing "interesting" {@code LockAttempt}s.
*
* @author hakon
*/
// @ThreadSafe
-public class LockInfoSamples {
+public class LockAttemptSamples {
private final int maxSamples;
/** Ensure atomic operations on this collection. */
private final Object monitor = new Object();
/** Keep at most one sample for each lock path. */
- private final Map<String, LockInfo> byLockPath;
+ private final Map<String, LockAttempt> byLockPath;
/**
* Priority queue containing all samples. The head of this queue (peek()/poll())
- * returns the LockInfo with the smallest duration.
+ * returns the LockAttempt with the smallest duration.
*/
- private final PriorityQueue<LockInfo> priorityQueue =
- new PriorityQueue<>(Comparator.comparing(LockInfo::getStableTotalDuration));
+ private final PriorityQueue<LockAttempt> priorityQueue =
+ new PriorityQueue<>(Comparator.comparing(LockAttempt::getStableTotalDuration));
- LockInfoSamples() { this(10); }
+ LockAttemptSamples() { this(10); }
- LockInfoSamples(int maxSamples) {
+ LockAttemptSamples(int maxSamples) {
this.maxSamples = maxSamples;
this.byLockPath = new HashMap<>(maxSamples);
}
int size() { return byLockPath.size(); }
- boolean maybeSample(LockInfo lockInfo) {
+ boolean maybeSample(LockAttempt lockAttempt) {
final boolean added;
synchronized (monitor) {
- if (shouldAdd(lockInfo)) {
- byLockPath.put(lockInfo.getLockPath(), lockInfo);
- priorityQueue.add(lockInfo);
+ if (shouldAdd(lockAttempt)) {
+ byLockPath.put(lockAttempt.getLockPath(), lockAttempt);
+ priorityQueue.add(lockAttempt);
added = true;
} else {
added = false;
@@ -53,18 +53,18 @@ public class LockInfoSamples {
if (added) {
// Unnecessary to invoke under synchronized, although it means that some samples
// may be without stack trace (just retry if that happens).
- lockInfo.fillStackTrace();
+ lockAttempt.fillStackTrace();
}
return added;
}
- private boolean shouldAdd(LockInfo lockInfo) {
- LockInfo existingLockInfo = byLockPath.get(lockInfo.getLockPath());
- if (existingLockInfo != null) {
- if (hasLongerDurationThan(lockInfo, existingLockInfo)) {
- byLockPath.remove(existingLockInfo.getLockPath());
- priorityQueue.remove(existingLockInfo);
+ private boolean shouldAdd(LockAttempt lockAttempt) {
+ LockAttempt existingLockAttempt = byLockPath.get(lockAttempt.getLockPath());
+ if (existingLockAttempt != null) {
+ if (hasLongerDurationThan(lockAttempt, existingLockAttempt)) {
+ byLockPath.remove(existingLockAttempt.getLockPath());
+ priorityQueue.remove(existingLockAttempt);
return true;
}
@@ -76,17 +76,17 @@ public class LockInfoSamples {
}
// peek() and poll() retrieves the smallest element.
- existingLockInfo = priorityQueue.peek(); // cannot be null
- if (hasLongerDurationThan(lockInfo, existingLockInfo)) {
+ existingLockAttempt = priorityQueue.peek(); // cannot be null
+ if (hasLongerDurationThan(lockAttempt, existingLockAttempt)) {
priorityQueue.poll();
- byLockPath.remove(existingLockInfo.getLockPath());
+ byLockPath.remove(existingLockAttempt.getLockPath());
return true;
}
return false;
}
- List<LockInfo> asList() {
+ List<LockAttempt> asList() {
synchronized (monitor) {
return List.copyOf(byLockPath.values());
}
@@ -99,8 +99,8 @@ public class LockInfoSamples {
}
}
- private static boolean hasLongerDurationThan(LockInfo lockInfo, LockInfo otherLockInfo) {
+ private static boolean hasLongerDurationThan(LockAttempt lockAttempt, LockAttempt otherLockAttempt) {
// Use stable total duration to avoid messing up priority queue.
- return lockInfo.getStableTotalDuration().compareTo(otherLockInfo.getStableTotalDuration()) > 0;
+ return lockAttempt.getStableTotalDuration().compareTo(otherLockAttempt.getStableTotalDuration()) > 0;
}
}
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/ThreadLockInfo.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/ThreadLockStats.java
index 92e6eb453b6..117844e17ee 100644
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/ThreadLockInfo.java
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/ThreadLockStats.java
@@ -18,39 +18,39 @@ import java.util.function.Consumer;
*
* @author hakon
*/
-public class ThreadLockInfo {
+public class ThreadLockStats {
- private static final ConcurrentHashMap<Thread, ThreadLockInfo> locks = new ConcurrentHashMap<>();
+ private static final ConcurrentHashMap<Thread, ThreadLockStats> locks = new ConcurrentHashMap<>();
- private static final LockInfoSamples completedLockInfoSamples = new LockInfoSamples();
+ private static final LockAttemptSamples COMPLETED_LOCK_ATTEMPT_SAMPLES = new LockAttemptSamples();
private static final ConcurrentHashMap<String, LockCounters> countersByLockPath = new ConcurrentHashMap<>();
private final Thread thread;
/** The locks are reentrant so there may be more than 1 lock for this thread. */
- private final ConcurrentLinkedDeque<LockInfo> lockInfos = new ConcurrentLinkedDeque<>();
+ private final ConcurrentLinkedDeque<LockAttempt> lockAttempts = new ConcurrentLinkedDeque<>();
public static Map<String, LockCounters> getLockCountersByPath() { return Map.copyOf(countersByLockPath); }
- public static List<ThreadLockInfo> getThreadLockInfos() { return List.copyOf(locks.values()); }
+ public static List<ThreadLockStats> getThreadLockInfos() { return List.copyOf(locks.values()); }
- public static List<LockInfo> getLockInfoSamples() {
- return completedLockInfoSamples.asList();
+ public static List<LockAttempt> getLockInfoSamples() {
+ return COMPLETED_LOCK_ATTEMPT_SAMPLES.asList();
}
- /** Returns the per-thread singleton ThreadLockInfo. */
- public static ThreadLockInfo getCurrentThreadLockInfo() {
- return locks.computeIfAbsent(Thread.currentThread(), ThreadLockInfo::new);
+ /** Returns the per-thread singleton ThreadLockStats. */
+ public static ThreadLockStats getCurrentThreadLockInfo() {
+ return locks.computeIfAbsent(Thread.currentThread(), ThreadLockStats::new);
}
static void clearStaticDataForTesting() {
locks.clear();
- completedLockInfoSamples.clear();
+ COMPLETED_LOCK_ATTEMPT_SAMPLES.clear();
countersByLockPath.clear();
}
- ThreadLockInfo(Thread currentThread) {
+ ThreadLockStats(Thread currentThread) {
this.thread = currentThread;
}
@@ -75,65 +75,65 @@ public class ThreadLockInfo {
return stackTrace.toString();
}
- public List<LockInfo> getLockInfos() { return List.copyOf(lockInfos); }
+ public List<LockAttempt> getLockAttempts() { return List.copyOf(lockAttempts); }
/** Mutable method (see class doc) */
public void invokingAcquire(String lockPath, Duration timeout) {
LockCounters lockCounters = getLockCounters(lockPath);
lockCounters.invokeAcquireCount.incrementAndGet();
lockCounters.inCriticalRegionCount.incrementAndGet();
- lockInfos.addLast(LockInfo.invokingAcquire(this, lockPath, timeout));
+ lockAttempts.addLast(LockAttempt.invokingAcquire(this, lockPath, timeout));
}
/** Mutable method (see class doc) */
public void acquireFailed(String lockPath) {
LockCounters lockCounters = getLockCounters(lockPath);
lockCounters.acquireFailedCount.incrementAndGet();
- removeLastLockInfo(lockCounters, LockInfo::acquireFailed);
+ removeLastLockInfo(lockCounters, LockAttempt::acquireFailed);
}
/** Mutable method (see class doc) */
public void acquireTimedOut(String lockPath) {
LockCounters lockCounters = getLockCounters(lockPath);
- if (lockInfos.size() > 1) {
+ if (lockAttempts.size() > 1) {
lockCounters.timeoutOnReentrancyErrorCount.incrementAndGet();
}
lockCounters.acquireTimedOutCount.incrementAndGet();
- removeLastLockInfo(lockCounters, LockInfo::timedOut);
+ removeLastLockInfo(lockCounters, LockAttempt::timedOut);
}
/** Mutable method (see class doc) */
public void lockAcquired(String lockPath) {
getLockCounters(lockPath).lockAcquiredCount.incrementAndGet();
- LockInfo lastLockInfo = lockInfos.peekLast();
- if (lastLockInfo == null) {
- throw new IllegalStateException("lockAcquired invoked without lockInfos");
+ LockAttempt lastLockAttempt = lockAttempts.peekLast();
+ if (lastLockAttempt == null) {
+ throw new IllegalStateException("lockAcquired invoked without lockAttempts");
}
- lastLockInfo.lockAcquired();
+ lastLockAttempt.lockAcquired();
}
/** Mutable method (see class doc) */
public void lockReleased(String lockPath) {
LockCounters lockCounters = getLockCounters(lockPath);
lockCounters.locksReleasedCount.incrementAndGet();
- removeLastLockInfo(lockCounters, LockInfo::released);
+ removeLastLockInfo(lockCounters, LockAttempt::released);
}
private LockCounters getLockCounters(String lockPath) {
return countersByLockPath.computeIfAbsent(lockPath, __ -> new LockCounters());
}
- private void removeLastLockInfo(LockCounters lockCounters, Consumer<LockInfo> completeLockInfo) {
+ private void removeLastLockInfo(LockCounters lockCounters, Consumer<LockAttempt> completeLockInfo) {
lockCounters.inCriticalRegionCount.decrementAndGet();
- if (lockInfos.isEmpty()) {
+ if (lockAttempts.isEmpty()) {
lockCounters.noLocksErrorCount.incrementAndGet();
return;
}
- LockInfo lockInfo = lockInfos.pollLast();
- completeLockInfo.accept(lockInfo);
- completedLockInfoSamples.maybeSample(lockInfo);
+ LockAttempt lockAttempt = lockAttempts.pollLast();
+ completeLockInfo.accept(lockAttempt);
+ COMPLETED_LOCK_ATTEMPT_SAMPLES.maybeSample(lockAttempt);
}
}
diff --git a/zkfacade/src/test/java/com/yahoo/vespa/curator/stats/LockInfoSamplesTest.java b/zkfacade/src/test/java/com/yahoo/vespa/curator/stats/LockAttemptSamplesTest.java
index 4a14b0cc1b2..6f877631b7b 100644
--- a/zkfacade/src/test/java/com/yahoo/vespa/curator/stats/LockInfoSamplesTest.java
+++ b/zkfacade/src/test/java/com/yahoo/vespa/curator/stats/LockAttemptSamplesTest.java
@@ -1,4 +1,5 @@
-package com.yahoo.vespa.curator.stats;// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.curator.stats;
import org.junit.Test;
@@ -11,13 +12,16 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
-public class LockInfoSamplesTest {
- private final LockInfoSamples samples = new LockInfoSamples(2);
- private ThreadLockInfo threadLockInfo;
+/**
+ * @author hakon
+ */
+public class LockAttemptSamplesTest {
+ private final LockAttemptSamples samples = new LockAttemptSamples(2);
+ private ThreadLockStats threadLockStats;
@Test
public void test() {
- threadLockInfo = new ThreadLockInfo(Thread.currentThread());
+ threadLockStats = new ThreadLockStats(Thread.currentThread());
assertTrue(maybeSample("1", 10));
@@ -36,7 +40,7 @@ public class LockInfoSamplesTest {
// new path, expels "2"
assertTrue(maybeSample("4", 6));
- Map<String, LockInfo> lockInfos = samples.asList().stream().collect(Collectors.toMap(
+ Map<String, LockAttempt> lockInfos = samples.asList().stream().collect(Collectors.toMap(
lockInfo -> lockInfo.getLockPath(),
lockInfo -> lockInfo));
assertEquals(2, lockInfos.size());
@@ -49,10 +53,10 @@ public class LockInfoSamplesTest {
}
private boolean maybeSample(String lockPath, int secondsDuration) {
- LockInfo lockInfo = LockInfo.invokingAcquire(threadLockInfo, lockPath, Duration.ofSeconds(1));
- Instant instant = lockInfo.getTimeAcquiredWasInvoked().plus(Duration.ofSeconds(secondsDuration));
- lockInfo.setTerminalState(LockInfo.LockState.RELEASED, instant);
- return samples.maybeSample(lockInfo);
+ LockAttempt lockAttempt = LockAttempt.invokingAcquire(threadLockStats, lockPath, Duration.ofSeconds(1));
+ Instant instant = lockAttempt.getTimeAcquiredWasInvoked().plus(Duration.ofSeconds(secondsDuration));
+ lockAttempt.setTerminalState(LockAttempt.LockState.RELEASED, instant);
+ return samples.maybeSample(lockAttempt);
}
} \ No newline at end of file
diff --git a/zkfacade/src/test/java/com/yahoo/vespa/curator/stats/LockTest.java b/zkfacade/src/test/java/com/yahoo/vespa/curator/stats/LockTest.java
index 984a04c6d4e..477fe650bb5 100644
--- a/zkfacade/src/test/java/com/yahoo/vespa/curator/stats/LockTest.java
+++ b/zkfacade/src/test/java/com/yahoo/vespa/curator/stats/LockTest.java
@@ -1,4 +1,5 @@
-package com.yahoo.vespa.curator.stats;// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.curator.stats;
import com.yahoo.vespa.curator.Lock;
import org.apache.curator.framework.recipes.locks.InterProcessLock;
@@ -19,6 +20,9 @@ import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
+/**
+ * @author hakon
+ */
public class LockTest {
private final InterProcessLock mutex = mock(InterProcessLock.class);
private final String lockPath = "/lock/path";
@@ -27,7 +31,7 @@ public class LockTest {
@Before
public void setUp() {
- ThreadLockInfo.clearStaticDataForTesting();
+ ThreadLockStats.clearStaticDataForTesting();
}
@Test
@@ -45,22 +49,22 @@ public class LockTest {
var expectedCounters = new LockCounters();
expectedCounters.invokeAcquireCount.set(1);
expectedCounters.acquireFailedCount.set(1);
- assertEquals(Map.of(lockPath, expectedCounters), ThreadLockInfo.getLockCountersByPath());
+ assertEquals(Map.of(lockPath, expectedCounters), ThreadLockStats.getLockCountersByPath());
- List<LockInfo> slowLockInfos = ThreadLockInfo.getLockInfoSamples();
- assertEquals(1, slowLockInfos.size());
- LockInfo slowLockInfo = slowLockInfos.get(0);
- assertEquals(acquireTimeout, slowLockInfo.getAcquireTimeout());
- Optional<String> stackTrace = slowLockInfo.getStackTrace();
+ List<LockAttempt> slowLockAttempts = ThreadLockStats.getLockInfoSamples();
+ assertEquals(1, slowLockAttempts.size());
+ LockAttempt slowLockAttempt = slowLockAttempts.get(0);
+ assertEquals(acquireTimeout, slowLockAttempt.getAcquireTimeout());
+ Optional<String> stackTrace = slowLockAttempt.getStackTrace();
assertTrue(stackTrace.isPresent());
assertTrue("bad stacktrace: " + stackTrace.get(), stackTrace.get().contains(".Lock.acquire(Lock.java"));
- assertEquals(LockInfo.LockState.ACQUIRE_FAILED, slowLockInfo.getLockState());
- assertTrue(slowLockInfo.getTimeTerminalStateWasReached().isPresent());
+ assertEquals(LockAttempt.LockState.ACQUIRE_FAILED, slowLockAttempt.getLockState());
+ assertTrue(slowLockAttempt.getTimeTerminalStateWasReached().isPresent());
- List<ThreadLockInfo> threadLockInfos = ThreadLockInfo.getThreadLockInfos();
- assertEquals(1, threadLockInfos.size());
- ThreadLockInfo threadLockInfo = threadLockInfos.get(0);
- assertEquals(0, threadLockInfo.getLockInfos().size());
+ List<ThreadLockStats> threadLockStatsList = ThreadLockStats.getThreadLockInfos();
+ assertEquals(1, threadLockStatsList.size());
+ ThreadLockStats threadLockStats = threadLockStatsList.get(0);
+ assertEquals(0, threadLockStats.getLockAttempts().size());
}
@Test
@@ -77,7 +81,7 @@ public class LockTest {
var expectedCounters = new LockCounters();
expectedCounters.invokeAcquireCount.set(1);
expectedCounters.acquireTimedOutCount.set(1);
- assertEquals(Map.of(lockPath, expectedCounters), ThreadLockInfo.getLockCountersByPath());
+ assertEquals(Map.of(lockPath, expectedCounters), ThreadLockStats.getLockCountersByPath());
}
@Test
@@ -90,7 +94,7 @@ public class LockTest {
expectedCounters.invokeAcquireCount.set(1);
expectedCounters.lockAcquiredCount.set(1);
expectedCounters.inCriticalRegionCount.set(1);
- assertEquals(Map.of(lockPath, expectedCounters), ThreadLockInfo.getLockCountersByPath());
+ assertEquals(Map.of(lockPath, expectedCounters), ThreadLockStats.getLockCountersByPath());
// reenter
lock.acquire(acquireTimeout);
@@ -102,13 +106,13 @@ public class LockTest {
lock.close();
expectedCounters.inCriticalRegionCount.set(1);
expectedCounters.locksReleasedCount.set(1);
- assertEquals(Map.of(lockPath, expectedCounters), ThreadLockInfo.getLockCountersByPath());
+ assertEquals(Map.of(lockPath, expectedCounters), ThreadLockStats.getLockCountersByPath());
// outer-most closes
lock.close();
expectedCounters.inCriticalRegionCount.set(0);
expectedCounters.locksReleasedCount.set(2);
- assertEquals(Map.of(lockPath, expectedCounters), ThreadLockInfo.getLockCountersByPath());
+ assertEquals(Map.of(lockPath, expectedCounters), ThreadLockStats.getLockCountersByPath());
}
@Test
@@ -121,14 +125,14 @@ public class LockTest {
lock.acquire(acquireTimeout);
lock2.acquire(acquireTimeout);
- List<ThreadLockInfo> threadLockInfos = ThreadLockInfo.getThreadLockInfos();
- assertEquals(1, threadLockInfos.size());
- List<LockInfo> lockInfos = threadLockInfos.get(0).getLockInfos();
- assertEquals(2, lockInfos.size());
- assertEquals(lockPath, lockInfos.get(0).getLockPath());
- assertEquals(LockInfo.LockState.ACQUIRED, lockInfos.get(0).getLockState());
- assertEquals(lockPath2, lockInfos.get(1).getLockPath());
- assertEquals(LockInfo.LockState.ACQUIRED, lockInfos.get(1).getLockState());
+ List<ThreadLockStats> threadLockStats = ThreadLockStats.getThreadLockInfos();
+ assertEquals(1, threadLockStats.size());
+ List<LockAttempt> lockAttempts = threadLockStats.get(0).getLockAttempts();
+ assertEquals(2, lockAttempts.size());
+ assertEquals(lockPath, lockAttempts.get(0).getLockPath());
+ assertEquals(LockAttempt.LockState.ACQUIRED, lockAttempts.get(0).getLockState());
+ assertEquals(lockPath2, lockAttempts.get(1).getLockPath());
+ assertEquals(LockAttempt.LockState.ACQUIRED, lockAttempts.get(1).getLockState());
lock.close();
lock.close();