summaryrefslogtreecommitdiffstats
path: root/zkfacade
diff options
context:
space:
mode:
authorHÃ¥kon Hallingstad <hakon@verizonmedia.com>2020-10-01 13:16:11 +0200
committerGitHub <noreply@github.com>2020-10-01 13:16:11 +0200
commit430f72a630a997b6d13bb5870e843d72db6982d4 (patch)
tree1de5d985ada9b4df5361b31c6a0bc5d0f0aabb02 /zkfacade
parentf6f2967780a3a5ff8d39097036a25862694c9ca0 (diff)
parent1726989cbd8bc9234c614ff498643caeb6ed61a7 (diff)
Merge pull request #14647 from vespa-engine/hakonhall/record-locks-taken-for-external-deploys
Record locks taken for external deploys
Diffstat (limited to 'zkfacade')
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/Lock.java7
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttempt.java8
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttemptSamples.java9
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockStats.java78
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/RecordedLockAttempts.java50
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/ThreadLockStats.java94
-rw-r--r--zkfacade/src/test/java/com/yahoo/vespa/curator/stats/LockTest.java22
7 files changed, 202 insertions, 66 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 920bba22804..c9a55e40da1 100644
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/Lock.java
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/Lock.java
@@ -4,6 +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.LockStats;
import com.yahoo.vespa.curator.stats.ThreadLockStats;
import org.apache.curator.framework.recipes.locks.InterProcessLock;
@@ -35,7 +36,7 @@ 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 {
- ThreadLockStats threadLockStats = ThreadLockStats.getCurrentThreadLockStats();
+ ThreadLockStats threadLockStats = LockStats.getForCurrentThread();
threadLockStats.invokingAcquire(lockPath, timeout);
final boolean acquired;
@@ -58,10 +59,10 @@ public class Lock implements Mutex {
public void close() {
try {
mutex.release();
- ThreadLockStats.getCurrentThreadLockStats().lockReleased(lockPath);
+ LockStats.getForCurrentThread().lockReleased(lockPath);
}
catch (Exception e) {
- ThreadLockStats.getCurrentThreadLockStats().lockReleaseFailed(lockPath);
+ LockStats.getForCurrentThread().lockReleaseFailed(lockPath);
throw new RuntimeException("Exception releasing lock '" + lockPath + "'");
}
}
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttempt.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttempt.java
index 3b06377ccf7..d57a4b7d9dc 100644
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttempt.java
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttempt.java
@@ -3,6 +3,8 @@ package com.yahoo.vespa.curator.stats;
import java.time.Duration;
import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
import java.util.Optional;
/**
@@ -19,6 +21,7 @@ public class LockAttempt {
private final String lockPath;
private final Instant callAcquireInstant;
private final Duration timeout;
+ private final List<LockAttempt> nestedLockAttempts = new ArrayList<>();
private volatile Optional<Instant> lockAcquiredInstant = Optional.empty();
private volatile Optional<Instant> terminalStateInstant = Optional.empty();
@@ -56,6 +59,7 @@ public class LockAttempt {
public Optional<Instant> getTimeLockWasAcquired() { return lockAcquiredInstant; }
public Optional<Instant> getTimeTerminalStateWasReached() { return terminalStateInstant; }
public Optional<String> getStackTrace() { return stackTrace; }
+ public List<LockAttempt> getNestedLockAttempts() { return List.copyOf(nestedLockAttempts); }
public Duration getDurationOfAcquire() {
return Duration.between(callAcquireInstant, lockAcquiredInstant.orElseGet(Instant::now));
@@ -82,6 +86,10 @@ public class LockAttempt {
this.stackTrace = Optional.of(threadLockStats.getStackTrace());
}
+ void addNestedLockAttempt(LockAttempt nestedLockAttempt) {
+ nestedLockAttempts.add(nestedLockAttempt);
+ }
+
void acquireFailed() { setTerminalState(LockState.ACQUIRE_FAILED); }
void timedOut() { setTerminalState(LockState.TIMED_OUT); }
void released() { setTerminalState(LockState.RELEASED); }
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttemptSamples.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttemptSamples.java
index 54cb82ebc1e..81435acb5e1 100644
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttemptSamples.java
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttemptSamples.java
@@ -29,8 +29,6 @@ public class LockAttemptSamples {
private final PriorityQueue<LockAttempt> priorityQueue =
new PriorityQueue<>(Comparator.comparing(LockAttempt::getStableTotalDuration));
- LockAttemptSamples() { this(10); }
-
LockAttemptSamples(int maxSamples) {
this.maxSamples = maxSamples;
this.byLockPath = new HashMap<>(maxSamples);
@@ -92,13 +90,6 @@ public class LockAttemptSamples {
}
}
- void clear() {
- synchronized (monitor) {
- byLockPath.clear();
- priorityQueue.clear();
- }
- }
-
private static boolean hasLongerDurationThan(LockAttempt lockAttempt, LockAttempt otherLockAttempt) {
// Use stable total duration to avoid messing up priority queue.
return lockAttempt.getStableTotalDuration().compareTo(otherLockAttempt.getStableTotalDuration()) > 0;
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockStats.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockStats.java
new file mode 100644
index 00000000000..3cb5af80c96
--- /dev/null
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockStats.java
@@ -0,0 +1,78 @@
+// 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 java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class manages statistics related to lock attempts on {@link com.yahoo.vespa.curator.Lock}.
+ *
+ * @author hakon
+ */
+public class LockStats {
+ // No 'volatile' is needed because field is only ever changed for testing which is single-threaded.
+ private static LockStats stats = new LockStats();
+
+ private final ConcurrentHashMap<Thread, ThreadLockStats> statsByThread = new ConcurrentHashMap<>();
+
+ private final LockAttemptSamples completedLockAttemptSamples = new LockAttemptSamples(3);
+
+ // Keep recordings in a priority queue, with the smallest element having the smallest duration.
+ // Recordings can be large, so keep the number of recordings low.
+ private static final int MAX_RECORDINGS = 3;
+ private final Object interestingRecordingsMonitor = new Object();
+ private final PriorityQueue<RecordedLockAttempts> interestingRecordings =
+ new PriorityQueue<>(MAX_RECORDINGS, Comparator.comparing(RecordedLockAttempts::duration));
+
+ private final ConcurrentHashMap<String, LockCounters> countersByLockPath = new ConcurrentHashMap<>();
+
+ /** Returns global stats. */
+ public static LockStats getGlobal() { return stats; }
+
+ /** Returns stats tied to the current thread. */
+ public static ThreadLockStats getForCurrentThread() {
+ return stats.statsByThread.computeIfAbsent(Thread.currentThread(), ThreadLockStats::new);
+ }
+
+ static void clearForTesting() {
+ stats = new LockStats();
+ }
+
+ private LockStats() {}
+
+ public Map<String, LockCounters> getLockCountersByPath() { return Map.copyOf(countersByLockPath); }
+ public List<ThreadLockStats> getThreadLockStats() { return List.copyOf(statsByThread.values()); }
+ public List<LockAttempt> getLockAttemptSamples() { return completedLockAttemptSamples.asList(); }
+
+ public List<RecordedLockAttempts> getHistoricRecordings() {
+ synchronized (interestingRecordingsMonitor) {
+ return List.copyOf(interestingRecordings);
+ }
+ }
+
+ LockCounters getLockCounters(String lockPath) {
+ return countersByLockPath.computeIfAbsent(lockPath, __ -> new LockCounters());
+ }
+
+ void maybeSample(LockAttempt lockAttempt) {
+ completedLockAttemptSamples.maybeSample(lockAttempt);
+ }
+
+ void reportNewStoppedRecording(RecordedLockAttempts recording) {
+ synchronized (interestingRecordings) {
+ if (interestingRecordings.size() < MAX_RECORDINGS) {
+ interestingRecordings.add(recording);
+ } else if (recording.duration().compareTo(interestingRecordings.peek().duration()) > 0) {
+ // peek() retrieves the smallest element according to the PriorityQueue's
+ // comparator.
+
+ interestingRecordings.poll();
+ interestingRecordings.add(recording);
+ }
+ }
+
+ }
+}
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/RecordedLockAttempts.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/RecordedLockAttempts.java
new file mode 100644
index 00000000000..668258028f2
--- /dev/null
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/RecordedLockAttempts.java
@@ -0,0 +1,50 @@
+// 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 java.time.Duration;
+import java.time.Instant;
+import java.util.List;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+/**
+ * Contains information about the lock attempts made by a thread between a start and end.
+ *
+ * <p>Any thread is allowed access the public methods of an instance.</p>
+ *
+ * @author hakon
+ */
+public class RecordedLockAttempts {
+ private static final int MAX_TOP_LEVEL_LOCK_ATTEMPTS = 100;
+
+ private final String recordId;
+ private final Instant startInstant;
+ private volatile Instant endInstant = null;
+ private final ConcurrentLinkedQueue<LockAttempt> lockAttempts = new ConcurrentLinkedQueue<>();
+
+ static RecordedLockAttempts startRecording(String recordId) {
+ return new RecordedLockAttempts(recordId);
+ }
+
+ private RecordedLockAttempts(String recordId) {
+ this.recordId = recordId;
+ startInstant = Instant.now();
+ }
+
+ /** Note: A LockAttempt may have nested lock attempts. */
+ void addTopLevelLockAttempt(LockAttempt lockAttempt) {
+ // guard against recordings that are too long - to cap the memory used
+ if (lockAttempts.size() < MAX_TOP_LEVEL_LOCK_ATTEMPTS) {
+ lockAttempts.add(lockAttempt);
+ }
+ }
+
+ void stopRecording() {
+ endInstant = Instant.now();
+ }
+
+ public String recordId() { return recordId; }
+ public Instant startInstant() { return startInstant; }
+ public Instant endInstant() { return endInstant == null ? Instant.now() : endInstant; }
+ public Duration duration() { return Duration.between(startInstant, endInstant()); }
+ public List<LockAttempt> lockAttempts() { return List.copyOf(lockAttempts); }
+}
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/ThreadLockStats.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/ThreadLockStats.java
index db26523ec37..c994121441a 100644
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/ThreadLockStats.java
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/ThreadLockStats.java
@@ -5,13 +5,12 @@ import com.yahoo.vespa.curator.Lock;
import java.time.Duration;
import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
+import java.util.Optional;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.function.Consumer;
/**
- * This class contains process-wide statistics and information related to acquiring and releasing
+ * This class manages thread-specific statistics and information related to acquiring and releasing
* {@link Lock}. Instances of this class contain information tied to a specific thread and lock path.
*
* <p>Instances of this class are thread-safe as long as foreign threads (!= this.thread) avoid mutable methods.</p>
@@ -20,35 +19,16 @@ import java.util.function.Consumer;
*/
public class ThreadLockStats {
- private static final ConcurrentHashMap<Thread, ThreadLockStats> locks = new ConcurrentHashMap<>();
-
- 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<LockAttempt> lockAttempts = new ConcurrentLinkedDeque<>();
+ /**
+ * The locks are reentrant so there may be more than 1 lock for this thread:
+ * The first LockAttempt in lockAttemptsStack was the first and top-most lock that was acquired.
+ */
+ private final ConcurrentLinkedDeque<LockAttempt> lockAttemptsStack = new ConcurrentLinkedDeque<>();
- public static Map<String, LockCounters> getLockCountersByPath() { return Map.copyOf(countersByLockPath); }
-
- public static List<ThreadLockStats> getThreadLockStats() { return List.copyOf(locks.values()); }
-
- public static List<LockAttempt> getLockAttemptSamples() {
- return COMPLETED_LOCK_ATTEMPT_SAMPLES.asList();
- }
-
- /** Returns the per-thread singleton ThreadLockStats. */
- public static ThreadLockStats getCurrentThreadLockStats() {
- return locks.computeIfAbsent(Thread.currentThread(), ThreadLockStats::new);
- }
-
- static void clearStaticDataForTesting() {
- locks.clear();
- COMPLETED_LOCK_ATTEMPT_SAMPLES.clear();
- countersByLockPath.clear();
- }
+ /** Non-empty if there is an ongoing recording for this thread. */
+ private volatile Optional<RecordedLockAttempts> ongoingRecording = Optional.empty();
ThreadLockStats(Thread currentThread) {
this.thread = currentThread;
@@ -75,26 +55,36 @@ public class ThreadLockStats {
return stackTrace.toString();
}
- public List<LockAttempt> getLockAttempts() { return List.copyOf(lockAttempts); }
+ public List<LockAttempt> getOngoingLockAttempts() { return List.copyOf(lockAttemptsStack); }
+ public Optional<LockAttempt> getTopMostOngoingLockAttempt() { return lockAttemptsStack.stream().findFirst(); }
+ public Optional<RecordedLockAttempts> getOngoingRecording() { return ongoingRecording; }
/** Mutable method (see class doc) */
public void invokingAcquire(String lockPath, Duration timeout) {
- LockCounters lockCounters = getLockCounters(lockPath);
+ LockCounters lockCounters = getGlobalLockCounters(lockPath);
lockCounters.invokeAcquireCount.incrementAndGet();
lockCounters.inCriticalRegionCount.incrementAndGet();
- lockAttempts.addLast(LockAttempt.invokingAcquire(this, lockPath, timeout));
+ LockAttempt lockAttempt = LockAttempt.invokingAcquire(this, lockPath, timeout);
+
+ LockAttempt lastLockAttempt = lockAttemptsStack.peekLast();
+ if (lastLockAttempt == null) {
+ ongoingRecording.ifPresent(recording -> recording.addTopLevelLockAttempt(lockAttempt));
+ } else {
+ lastLockAttempt.addNestedLockAttempt(lockAttempt);
+ }
+ lockAttemptsStack.addLast(lockAttempt);
}
/** Mutable method (see class doc) */
public void acquireFailed(String lockPath) {
- LockCounters lockCounters = getLockCounters(lockPath);
+ LockCounters lockCounters = getGlobalLockCounters(lockPath);
lockCounters.acquireFailedCount.incrementAndGet();
removeLastLockAttempt(lockCounters, LockAttempt::acquireFailed);
}
/** Mutable method (see class doc) */
public void acquireTimedOut(String lockPath) {
- LockCounters lockCounters = getLockCounters(lockPath);
+ LockCounters lockCounters = getGlobalLockCounters(lockPath);
lockCounters.acquireTimedOutCount.incrementAndGet();
removeLastLockAttempt(lockCounters, LockAttempt::timedOut);
@@ -102,8 +92,8 @@ public class ThreadLockStats {
/** Mutable method (see class doc) */
public void lockAcquired(String lockPath) {
- getLockCounters(lockPath).lockAcquiredCount.incrementAndGet();
- LockAttempt lastLockAttempt = lockAttempts.peekLast();
+ getGlobalLockCounters(lockPath).lockAcquiredCount.incrementAndGet();
+ LockAttempt lastLockAttempt = lockAttemptsStack.peekLast();
if (lastLockAttempt == null) {
throw new IllegalStateException("lockAcquired invoked without lockAttempts");
}
@@ -112,32 +102,50 @@ public class ThreadLockStats {
/** Mutable method (see class doc) */
public void lockReleased(String lockPath) {
- LockCounters lockCounters = getLockCounters(lockPath);
+ LockCounters lockCounters = getGlobalLockCounters(lockPath);
lockCounters.locksReleasedCount.incrementAndGet();
removeLastLockAttempt(lockCounters, LockAttempt::released);
}
/** Mutable method (see class doc) */
public void lockReleaseFailed(String lockPath) {
- LockCounters lockCounters = getLockCounters(lockPath);
+ LockCounters lockCounters = getGlobalLockCounters(lockPath);
lockCounters.lockReleaseErrorCount.incrementAndGet();
removeLastLockAttempt(lockCounters, LockAttempt::releasedWithError);
}
- private LockCounters getLockCounters(String lockPath) {
- return countersByLockPath.computeIfAbsent(lockPath, __ -> new LockCounters());
+ /** Mutable method (see class doc) */
+ public void startRecording(String recordId) {
+ ongoingRecording = Optional.of(RecordedLockAttempts.startRecording(recordId));
+ }
+
+ /** Mutable method (see class doc) */
+ public void stopRecording() {
+ if (ongoingRecording.isPresent()) {
+ RecordedLockAttempts recording = ongoingRecording.get();
+ ongoingRecording = Optional.empty();
+
+ // We'll keep the recordings with the longest durations.
+ recording.stopRecording();
+ LockStats.getGlobal().reportNewStoppedRecording(recording);
+ }
+ }
+
+ private LockCounters getGlobalLockCounters(String lockPath) {
+ return LockStats.getGlobal().getLockCounters(lockPath);
}
private void removeLastLockAttempt(LockCounters lockCounters, Consumer<LockAttempt> completeLockAttempt) {
lockCounters.inCriticalRegionCount.decrementAndGet();
- if (lockAttempts.isEmpty()) {
+ if (lockAttemptsStack.isEmpty()) {
lockCounters.noLocksErrorCount.incrementAndGet();
return;
}
- LockAttempt lockAttempt = lockAttempts.pollLast();
+ LockAttempt lockAttempt = lockAttemptsStack.pollLast();
completeLockAttempt.accept(lockAttempt);
- COMPLETED_LOCK_ATTEMPT_SAMPLES.maybeSample(lockAttempt);
+
+ LockStats.getGlobal().maybeSample(lockAttempt);
}
}
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 92911b0dadf..a221975c120 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
@@ -31,7 +31,7 @@ public class LockTest {
@Before
public void setUp() {
- ThreadLockStats.clearStaticDataForTesting();
+ LockStats.clearForTesting();
}
@Test
@@ -49,9 +49,9 @@ public class LockTest {
var expectedCounters = new LockCounters();
expectedCounters.invokeAcquireCount.set(1);
expectedCounters.acquireFailedCount.set(1);
- assertEquals(Map.of(lockPath, expectedCounters), ThreadLockStats.getLockCountersByPath());
+ assertEquals(Map.of(lockPath, expectedCounters), LockStats.getGlobal().getLockCountersByPath());
- List<LockAttempt> slowLockAttempts = ThreadLockStats.getLockAttemptSamples();
+ List<LockAttempt> slowLockAttempts = LockStats.getGlobal().getLockAttemptSamples();
assertEquals(1, slowLockAttempts.size());
LockAttempt slowLockAttempt = slowLockAttempts.get(0);
assertEquals(acquireTimeout, slowLockAttempt.getAcquireTimeout());
@@ -61,10 +61,10 @@ public class LockTest {
assertEquals(LockAttempt.LockState.ACQUIRE_FAILED, slowLockAttempt.getLockState());
assertTrue(slowLockAttempt.getTimeTerminalStateWasReached().isPresent());
- List<ThreadLockStats> threadLockStatsList = ThreadLockStats.getThreadLockStats();
+ List<ThreadLockStats> threadLockStatsList = LockStats.getGlobal().getThreadLockStats();
assertEquals(1, threadLockStatsList.size());
ThreadLockStats threadLockStats = threadLockStatsList.get(0);
- assertEquals(0, threadLockStats.getLockAttempts().size());
+ assertEquals(0, threadLockStats.getOngoingLockAttempts().size());
}
@Test
@@ -81,7 +81,7 @@ public class LockTest {
var expectedCounters = new LockCounters();
expectedCounters.invokeAcquireCount.set(1);
expectedCounters.acquireTimedOutCount.set(1);
- assertEquals(Map.of(lockPath, expectedCounters), ThreadLockStats.getLockCountersByPath());
+ assertEquals(Map.of(lockPath, expectedCounters), LockStats.getGlobal().getLockCountersByPath());
}
@Test
@@ -94,7 +94,7 @@ public class LockTest {
expectedCounters.invokeAcquireCount.set(1);
expectedCounters.lockAcquiredCount.set(1);
expectedCounters.inCriticalRegionCount.set(1);
- assertEquals(Map.of(lockPath, expectedCounters), ThreadLockStats.getLockCountersByPath());
+ assertEquals(Map.of(lockPath, expectedCounters), LockStats.getGlobal().getLockCountersByPath());
// reenter
lock.acquire(acquireTimeout);
@@ -106,13 +106,13 @@ public class LockTest {
lock.close();
expectedCounters.inCriticalRegionCount.set(1);
expectedCounters.locksReleasedCount.set(1);
- assertEquals(Map.of(lockPath, expectedCounters), ThreadLockStats.getLockCountersByPath());
+ assertEquals(Map.of(lockPath, expectedCounters), LockStats.getGlobal().getLockCountersByPath());
// outer-most closes
lock.close();
expectedCounters.inCriticalRegionCount.set(0);
expectedCounters.locksReleasedCount.set(2);
- assertEquals(Map.of(lockPath, expectedCounters), ThreadLockStats.getLockCountersByPath());
+ assertEquals(Map.of(lockPath, expectedCounters), LockStats.getGlobal().getLockCountersByPath());
}
@Test
@@ -125,9 +125,9 @@ public class LockTest {
lock.acquire(acquireTimeout);
lock2.acquire(acquireTimeout);
- List<ThreadLockStats> threadLockStats = ThreadLockStats.getThreadLockStats();
+ List<ThreadLockStats> threadLockStats = LockStats.getGlobal().getThreadLockStats();
assertEquals(1, threadLockStats.size());
- List<LockAttempt> lockAttempts = threadLockStats.get(0).getLockAttempts();
+ List<LockAttempt> lockAttempts = threadLockStats.get(0).getOngoingLockAttempts();
assertEquals(2, lockAttempts.size());
assertEquals(lockPath, lockAttempts.get(0).getLockPath());
assertEquals(LockAttempt.LockState.ACQUIRED, lockAttempts.get(0).getLockState());