aboutsummaryrefslogtreecommitdiffstats
path: root/zkfacade/src/main/java/com/yahoo/vespa/curator/stats
diff options
context:
space:
mode:
authorHåkon Hallingstad <hakon@verizonmedia.com>2020-10-05 15:58:54 +0200
committerHåkon Hallingstad <hakon@verizonmedia.com>2020-10-05 15:58:54 +0200
commit967bff7bcfb043dbb5a926b7f61e68c8971759df (patch)
tree17fe62c79d51ece23e9c9daa7af82edfdecf10c7 /zkfacade/src/main/java/com/yahoo/vespa/curator/stats
parente925ef8b0a33ed0e67e09ca9320b386339e08cea (diff)
Make richer latency stats
Makes a LatencyStats which provides some useful metrics, best explained there and in LatencyMetrics. This includes latency metrics, the "QPS" (e.g. the number of acquire() per second), and load metrics. Unfortunately I had to move from atomics to synchronized to accomplish this, but I see no other way.
Diffstat (limited to 'zkfacade/src/main/java/com/yahoo/vespa/curator/stats')
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/AtomicDurationSum.java86
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/DurationSum.java41
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyMetrics.java77
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyStats.java169
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyStore.java56
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockAttempt.java16
-rw-r--r--zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockMetrics.java60
7 files changed, 260 insertions, 245 deletions
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/AtomicDurationSum.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/AtomicDurationSum.java
deleted file mode 100644
index 7addd661b79..00000000000
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/AtomicDurationSum.java
+++ /dev/null
@@ -1,86 +0,0 @@
-// 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.util.concurrent.atomic.AtomicLong;
-
-/**
- * An instance of {@link AtomicDurationSum} provides atomic operations on a {@link Duration}
- * and integer counter doublet: You can add a duration and increment the counter atomically,
- * and get the doublet and reset the duration and counter to zero atomically.
- *
- * <p>The duration and counter must be small: You can add the equivalent of 16M durations of 1 minute each:
- * The cumulative duration must be between -17 and 17 years, and the maximum count is 16M.
- * The duration will have millisecond resolution. Overflow of count affects duration.</p>
- *
- * <p>Motivation: Metrics are snapshot and reset to zero every minute. Durations of typically 1 minute
- * are then summed to produce a cumulative {@link Duration} and an associated count, both of which are
- * therefore small numbers and can be represented in a compact and atomic form. The alternative is to
- * use synchronization (which is slow) or allow inconsistencies between the duration and count
- * (e.g. a sum of 2 latencies but a count of 1 makes the metrics noisy).</p>
- *
- * @author hakon
- */
-public class AtomicDurationSum {
-
- // Why 40? The duration-part requires 16 bits to represent 1 minute. If we require 1 minute
- // durations can be added until both the duration-part and count-part are full, the remaining
- // 48 bits must be divided equally, hence 16 + 24 = 40. Seems to give a nice balance.
- static final long DURATION_BITS = 40;
- static final long COUNT_BITS = Long.SIZE - DURATION_BITS;
- static final long DURATION_MASK = -1L << COUNT_BITS;
- static final long COUNT_MASK = -1L >>> DURATION_BITS;
- // The most significant bit of duration is a sign bit, which complicates the initializer.
- static final long MIN_DURATION = -1L << (DURATION_BITS - 1);
- static final long MAX_DURATION = (DURATION_MASK << 1) >>> (COUNT_BITS + 1);
- static final long MAX_COUNT = COUNT_MASK;
- static final long MIN_COUNT = 0L;
-
- private static final long ZERO_DURATION_AND_COUNT = 0L;
-
- // Representation:
- // - A signed long of 40 bits storing the duration in milliseconds
- // - An unsigned int of 24 bits storing the count
- private final AtomicLong encodedAtomic = new AtomicLong(ZERO_DURATION_AND_COUNT);
-
- /** Initializes to zero duration and count. */
- public AtomicDurationSum() {}
-
- /** Add duration and increment count. */
- void add(Duration duration) {
- encodedAtomic.addAndGet(encodeDuration(duration) | 1L);
- }
-
- public DurationSum get() {
- long snapshot = encodedAtomic.get();
- return new DurationSum(decodeDuration(snapshot), decodeCount(snapshot));
- }
-
- /** Get the current {@link DurationSum} and reset the duration and count doublet to zero. */
- public DurationSum getAndReset() {
- long snapshot = encodedAtomic.getAndSet(ZERO_DURATION_AND_COUNT);
- return new DurationSum(decodeDuration(snapshot), decodeCount(snapshot));
- }
-
- static long encodeDuration(Duration duration) {
- long millis = duration.toMillis();
- if (millis < MIN_DURATION || millis > MAX_DURATION) {
- throw new IllegalArgumentException("Duration outside legal range: " + duration);
- }
-
- return millis << COUNT_BITS;
- }
-
- static Duration decodeDuration(long encoded) {
- return Duration.ofMillis(encoded >> COUNT_BITS);
- }
-
- static int decodeCount(long encoded) {
- return (int) (encoded & COUNT_MASK);
- }
-
- @Override
- public String toString() {
- return get().toString();
- }
-}
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/DurationSum.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/DurationSum.java
deleted file mode 100644
index 3c1f0ec7f43..00000000000
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/DurationSum.java
+++ /dev/null
@@ -1,41 +0,0 @@
-// 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.util.Optional;
-
-/**
- * Represents a sum and count of Duration.
- *
- * @author hakon
- */
-public class DurationSum {
-
- private final Duration duration;
- private final int count;
-
- DurationSum(Duration duration, int count) {
- this.duration = duration;
- this.count = count;
- }
-
- public Duration duration() { return duration; }
- public int count() { return count; }
-
- public Optional<Duration> averageDuration() {
- if (count <= 0) {
- return Optional.empty();
- }
-
- long averageMillis = Math.round(duration.toMillis() / (double) count);
- return Optional.of(Duration.ofMillis(averageMillis));
- }
-
- @Override
- public String toString() {
- return "DurationSum{" +
- "duration=" + duration +
- ", count=" + count +
- '}';
- }
-}
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyMetrics.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyMetrics.java
index 61807b5db19..22af158faa9 100644
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyMetrics.java
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyMetrics.java
@@ -1,39 +1,82 @@
// 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 static java.lang.Math.round;
+
/**
- * Metrics on the latency of execution of some piece of code, e.g. the acquiring of a lock.
+ * Metrics on the <em>time interval</em> associated with e.g. the acquiring of a lock.
+ *
+ * <p>In the language of {@link LatencyStats}, these metrics relate to time intervals associated
+ * with e.g. the acquiring of a lock, collected over time period.</p>
*
+ * @see LatencyStats
* @author hakon
*/
+// @Immutable
public class LatencyMetrics {
- private final DurationSum cumulativeLatency;
- private final float load;
- public LatencyMetrics(DurationSum cumulativeLatency, float load) {
- this.cumulativeLatency = cumulativeLatency;
+ private final Duration latency;
+ private final Duration maxLatency;
+ private final Duration maxActiveLatency;
+ private final double startHz;
+ private final double endHz;
+ private final double load;
+ private final int maxLoad;
+ private final int currentLoad;
+
+ public LatencyMetrics(Duration latency, Duration maxLatency, Duration maxActiveLatency,
+ double startHz, double endHz,
+ double load, int maxLoad, int currentLoad) {
+ this.latency = latency;
+ this.maxLatency = maxLatency;
+ this.maxActiveLatency = maxActiveLatency;
+ this.startHz = startHz;
+ this.endHz = endHz;
this.load = load;
+ this.maxLoad = maxLoad;
+ this.currentLoad = currentLoad;
}
- /**
- * The total time spent by all threads accumulating latency in an implicit time period,
- * e.g. a metric snapshot window, divided by the duration of the time period.
- */
- public float load() { return load; }
+ /** Returns the average latency of all intervals that ended in the period. */
+ public double latencySeconds() { return secondsWithMillis(latency); }
- /** Returns the average latency in seconds with milliseconds resolution, or 0.0 by default. */
- public float averageInSeconds() {
- return cumulativeLatency.averageDuration().map(average -> average.toMillis() / 1000f).orElse(0f);
- }
+ /** Returns the maximum latency of any interval that ended in the period. */
+ public double maxLatencySeconds() { return secondsWithMillis(maxLatency); }
+
+ /** Return the maximum latency of any interval that ended in the period, or is still active. */
+ public double maxActiveLatencySeconds() { return secondsWithMillis(maxActiveLatency); }
- /** The number of latency-producing events. */
- public int count() { return cumulativeLatency.count(); }
+ /** Returns the average number of intervals that started in the period per second. */
+ public double startHz() { return roundTo3DecimalPlaces(startHz); }
+
+ /** Returns the average number of intervals that ended in the period per second. */
+ public double endHz() { return roundTo3DecimalPlaces(endHz); }
+
+ /** The average load of the implied time period, with 3 decimal places precision. */
+ public double load() { return roundTo3DecimalPlaces(load); }
+
+ /** Returns the maximum number of concurrently active intervals in the period. */
+ public int maxLoad() { return maxLoad; }
+
+ /** Returns the number of active intervals right now. */
+ public int currentLoad() { return currentLoad; }
@Override
public String toString() {
return "LatencyMetrics{" +
- "cumulativeLatency=" + cumulativeLatency +
+ "averageLatency=" + latency +
+ ", maxLatency=" + maxLatency +
+ ", maxActiveLatency=" + maxActiveLatency +
+ ", numIntervalsStarted=" + startHz +
+ ", numIntervalsEnded=" + endHz +
", load=" + load +
+ ", maxLoad=" + maxLoad +
+ ", currentLoad=" + currentLoad +
'}';
}
+
+ private double secondsWithMillis(Duration duration) { return round(duration.toMillis()) / 1000.0; }
+ private double roundTo3DecimalPlaces(double value) { return round(value * 1000) / 1000.0; }
}
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyStats.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyStats.java
new file mode 100644
index 00000000000..0dc6e5ffb70
--- /dev/null
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyStats.java
@@ -0,0 +1,169 @@
+// 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.util.Comparator;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.function.LongSupplier;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import static java.lang.Math.max;
+import static java.lang.Math.round;
+
+/**
+ * An instance of {@code LatencyStats} keeps track of statistics related to <em>time intervals</em> that
+ * start at a particular moment in time and ends at a later time. A typical example is the processing of
+ * requests: Each newly received request starts a new interval, and ends when the response is sent.
+ *
+ * <p>The statistics only applies to the current <em>time period</em>, and can be retrieved as a
+ * {@link LatencyMetrics} instance from e.g. {@link #getLatencyMetrics()}. This fits well with how Yamas
+ * works: it collects metrics since last collection every minute or so.</p>
+ *
+ * @see LatencyMetrics
+ * @author hakon
+ */
+// @Thread-Safe
+public class LatencyStats {
+
+ private static Logger logger = Logger.getLogger(LatencyStats.class.getName());
+
+ private final LongSupplier nanoTimeSupplier;
+
+ // NB: Keep these in sync with resetForNewPeriod()
+ private final Object monitor = new Object();
+ private long startOfPeriodNanos;
+ private long endOfPeriodNanos;
+ private double cumulativeLoadNanos;
+ private Duration cumulativeLatency;
+ private Duration maxLatency;
+ private int numIntervalsStarted;
+ private int numIntervalsEnded;
+ private final HashSet<ActiveIntervalInfo> activeIntervals = new HashSet<>();
+ private int maxLoad;
+
+ /** Creates an empty LatencyStats starting the first time period now. */
+ public LatencyStats() { this(System::nanoTime); }
+
+ LatencyStats(LongSupplier nanoTimeSupplier) {
+ this.nanoTimeSupplier = nanoTimeSupplier;
+ this.endOfPeriodNanos = nanoTimeSupplier.getAsLong();
+ resetForNewPeriod();
+ }
+
+ /** @see #startNewInterval() */
+ public interface ActiveInterval extends AutoCloseable {
+ @Override void close();
+ }
+
+ /**
+ * Starts a new (active) interval. The caller MUST call {@link ActiveInterval#close()} on the
+ * returned instance exactly once, which will end the interval.
+ */
+ public ActiveInterval startNewInterval() {
+ synchronized (monitor) {
+ pushEndOfPeriodToNow();
+ ActiveIntervalInfo activeIntervalInfo = new ActiveIntervalInfo(endOfPeriodNanos);
+ activeIntervals.add(activeIntervalInfo);
+ maxLoad = max(maxLoad, activeIntervals.size()) ;
+ ++numIntervalsStarted;
+ return () -> endInterval(activeIntervalInfo);
+ }
+ }
+
+ /** Returns the metrics for the current time period up to now. */
+ public LatencyMetrics getLatencyMetrics() {
+ synchronized (monitor) {
+ pushEndOfPeriodToNow();
+ return makeLatencyMetrics();
+ }
+ }
+
+ /** Returns the metrics for the current time period up to now, and starts a new period. */
+ public LatencyMetrics getLatencyMetricsAndStartNewPeriod() {
+ synchronized (monitor) {
+ pushEndOfPeriodToNow();
+ LatencyMetrics metrics = makeLatencyMetrics();
+ resetForNewPeriod();
+ return metrics;
+ }
+ }
+
+ private static class ActiveIntervalInfo {
+ private final long startNanos;
+ public ActiveIntervalInfo(long startOfIntervalNanos) { this.startNanos = startOfIntervalNanos; }
+ public long startOfIntervalNanos() { return startNanos; }
+ }
+
+ private void resetForNewPeriod() {
+ startOfPeriodNanos = endOfPeriodNanos;
+ cumulativeLoadNanos = 0.0;
+ cumulativeLatency = Duration.ZERO;
+ maxLatency = Duration.ZERO;
+ numIntervalsStarted = 0;
+ numIntervalsEnded = 0;
+ maxLoad = activeIntervals.size();
+ }
+
+ private void pushEndOfPeriodToNow() {
+ long currentNanos = nanoTimeSupplier.getAsLong();
+ cumulativeLoadNanos += activeIntervals.size() * (currentNanos - endOfPeriodNanos);
+ endOfPeriodNanos = currentNanos;
+ }
+
+ private void endInterval(ActiveIntervalInfo activeInterval) {
+ boolean wasRemoved;
+ synchronized (monitor) {
+ pushEndOfPeriodToNow();
+ wasRemoved = activeIntervals.remove(activeInterval);
+ Duration latency = Duration.ofNanos(endOfPeriodNanos - activeInterval.startOfIntervalNanos());
+ cumulativeLatency = cumulativeLatency.plus(latency);
+ if (latency.compareTo(maxLatency) > 0) {
+ maxLatency = latency;
+ }
+ ++numIntervalsEnded;
+ }
+
+ if (!wasRemoved) {
+ // Exception made to dump stack trace.
+ logger.log(Level.WARNING, "Interval of latency stats was closed twice", new IllegalStateException());
+ }
+ }
+
+ /** Returns the metrics for the startOfPeriodNanos and endOfPeriodNanos period. */
+ private LatencyMetrics makeLatencyMetrics() {
+ Duration latency = numIntervalsEnded <= 0 ?
+ Duration.ZERO :
+ Duration.ofNanos(round(cumulativeLatency.toNanos() / (double) numIntervalsEnded));
+
+ Optional<Duration> maxLatencyFromActiveIntervals = activeIntervals.stream()
+ .map(ActiveIntervalInfo::startOfIntervalNanos)
+ .min(Comparator.comparing(value -> value))
+ .map(startOfIntervalNanos -> Duration.ofNanos(endOfPeriodNanos - startOfIntervalNanos));
+ Duration maxActiveLatency = maxLatencyFromActiveIntervals
+ .filter(latencyCandidate -> latencyCandidate.compareTo(maxLatency) > 0)
+ .orElse(maxLatency);
+
+ final double startHz, endHz, load;
+ long periodNanos = endOfPeriodNanos - startOfPeriodNanos;
+ if (periodNanos > 0) {
+ double periodSeconds = periodNanos / 1_000_000_000.0;
+ startHz = numIntervalsStarted / periodSeconds;
+ endHz = numIntervalsEnded / periodSeconds;
+ load = cumulativeLoadNanos / periodNanos;
+ } else {
+ startHz = endHz = 0.0;
+ load = activeIntervals.size();
+ }
+
+ return new LatencyMetrics(latency,
+ maxLatency,
+ maxActiveLatency,
+ startHz,
+ endHz,
+ load,
+ maxLoad,
+ activeIntervals.size());
+ }
+}
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyStore.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyStore.java
deleted file mode 100644
index 0b1056ddeba..00000000000
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LatencyStore.java
+++ /dev/null
@@ -1,56 +0,0 @@
-// 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.Clock;
-import java.time.Duration;
-import java.time.Instant;
-
-/**
- * Stores the latency of e.g. acquiring the lock.
- *
- * @author hakon
- */
-public class LatencyStore {
-
- private final AtomicDurationSum latencySum = new AtomicDurationSum();
- private final Clock clock;
- private volatile Instant startOfPeriod;
-
- LatencyStore() { this(Clock.systemUTC()); }
-
- LatencyStore(Clock clock) {
- this.clock = clock;
- startOfPeriod = clock.instant();
- }
-
- void reportLatency(Duration latency) {
- latencySum.add(latency);
- }
-
- public LatencyMetrics getLatencyMetrics() {
- return makeMetricsForPeriod(latencySum.get(), startOfPeriod, clock.instant());
- }
-
- public LatencyMetrics getAndResetLatencyMetrics() {
- Instant newStartOfPeriod = clock.instant();
- DurationSum latencySumOfPeriod = latencySum.getAndReset();
- LatencyMetrics latencyMetrics = makeMetricsForPeriod(latencySumOfPeriod, startOfPeriod, newStartOfPeriod);
- startOfPeriod = newStartOfPeriod;
- return latencyMetrics;
- }
-
- private static LatencyMetrics makeMetricsForPeriod(DurationSum latencySum, Instant start, Instant end) {
- long millisPeriod = Duration.between(start, end).toMillis();
- long normalizedMillisPeriod = Math.max(1L, millisPeriod);
- double load = Math.round(latencySum.duration().toMillis() * 1000.0 / normalizedMillisPeriod) / 1000.0;
- return new LatencyMetrics(latencySum, (float) load);
- }
-
- @Override
- public String toString() {
- return "LatencyStore{" +
- "latencySum=" + latencySum +
- ", startOfPeriod=" + startOfPeriod +
- '}';
- }
-}
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 e25470ac227..3a62c7bf988 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
@@ -23,6 +23,9 @@ public class LockAttempt {
private final Duration timeout;
private final LockMetrics lockMetrics;
private final List<LockAttempt> nestedLockAttempts = new ArrayList<>();
+ private final LatencyStats.ActiveInterval activeAcquireInterval;
+ // Only accessed by mutating thread:
+ private LatencyStats.ActiveInterval activeLockedInterval = null;
private volatile Optional<Instant> lockAcquiredInstant = Optional.empty();
private volatile Optional<Instant> terminalStateInstant = Optional.empty();
@@ -53,8 +56,7 @@ public class LockAttempt {
this.callAcquireInstant = callAcquireInstant;
this.timeout = timeout;
this.lockMetrics = lockMetrics;
-
- lockMetrics.acquireInvoked();
+ this.activeAcquireInterval = lockMetrics.acquireInvoked();
}
public String getThreadName() { return threadLockStats.getThreadName(); }
@@ -99,28 +101,28 @@ public class LockAttempt {
void acquireFailed() {
setTerminalState(LockState.ACQUIRE_FAILED);
- lockMetrics.acquireFailed(getDurationOfAcquire());
+ lockMetrics.acquireFailed(activeAcquireInterval);
}
void timedOut() {
setTerminalState(LockState.TIMED_OUT);
- lockMetrics.acquireTimedOut(getDurationOfAcquire());
+ lockMetrics.acquireTimedOut(activeAcquireInterval);
}
void lockAcquired() {
lockState = LockState.ACQUIRED;
lockAcquiredInstant = Optional.of(Instant.now());
- lockMetrics.lockAcquired(getDurationOfAcquire());
+ activeLockedInterval = lockMetrics.lockAcquired(activeAcquireInterval);
}
void released() {
setTerminalState(LockState.RELEASED);
- lockMetrics.release(getDurationWithLock(), getDuration());
+ lockMetrics.release(activeLockedInterval);
}
void releasedWithError() {
setTerminalState(LockState.RELEASED_WITH_ERROR);
- lockMetrics.releaseFailed(getDurationWithLock(), getDuration());
+ lockMetrics.releaseFailed(activeLockedInterval);
}
void setTerminalState(LockState terminalState) { setTerminalState(terminalState, Instant.now()); }
diff --git a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockMetrics.java b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockMetrics.java
index a5bf0124a54..51dcf3dd8de 100644
--- a/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockMetrics.java
+++ b/zkfacade/src/main/java/com/yahoo/vespa/curator/stats/LockMetrics.java
@@ -1,7 +1,6 @@
// 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.util.concurrent.atomic.AtomicInteger;
/**
@@ -24,49 +23,43 @@ public class LockMetrics {
private final AtomicInteger cumulativeReleaseCount = new AtomicInteger(0);
private final AtomicInteger cumulativeReleaseFailedCount = new AtomicInteger(0);
- private final AtomicInteger acquiringNow = new AtomicInteger(0);
- private final AtomicInteger lockedNow = new AtomicInteger(0);
+ private final LatencyStats acquireStats = new LatencyStats();
+ private final LatencyStats lockedStats = new LatencyStats();
- private final LatencyStore acquireLatencyStore = new LatencyStore();
- private final LatencyStore lockedLatencyStore = new LatencyStore();
-
- void acquireInvoked() {
+ /** Returns a Runnable that must be invoked when the acquire() finishes. */
+ LatencyStats.ActiveInterval acquireInvoked() {
acquireCount.incrementAndGet();
cumulativeAcquireCount.incrementAndGet();
- acquiringNow.incrementAndGet();
+ return acquireStats.startNewInterval();
}
- void acquireFailed(Duration acquireLatency) {
- acquiringNow.decrementAndGet();
+ void acquireFailed(LatencyStats.ActiveInterval acquireInterval) {
+ acquireInterval.close();
acquireFailedCount.incrementAndGet();
cumulativeAcquireFailedCount.incrementAndGet();
- acquireLatencyStore.reportLatency(acquireLatency);
}
- void acquireTimedOut(Duration acquireLatency) {
- acquiringNow.decrementAndGet();
+ void acquireTimedOut(LatencyStats.ActiveInterval acquireInterval) {
+ acquireInterval.close();
acquireTimedOutCount.incrementAndGet();
cumulativeAcquireTimedOutCount.incrementAndGet();
- acquireLatencyStore.reportLatency(acquireLatency);
}
- void lockAcquired(Duration acquireLatency) {
- acquiringNow.decrementAndGet();
+ LatencyStats.ActiveInterval lockAcquired(LatencyStats.ActiveInterval acquireInterval) {
+ acquireInterval.close();
acquireSucceededCount.incrementAndGet();
cumulativeAcquireSucceededCount.incrementAndGet();
- acquireLatencyStore.reportLatency(acquireLatency);
- lockedNow.incrementAndGet();
+ return lockedStats.startNewInterval();
}
- void release(Duration lockedLatency, Duration totalLatency) {
- lockedNow.decrementAndGet();
+ void release(LatencyStats.ActiveInterval lockedInterval) {
+ lockedInterval.close();
releaseCount.incrementAndGet();
cumulativeReleaseCount.incrementAndGet();
- lockedLatencyStore.reportLatency(lockedLatency);
}
- void releaseFailed(Duration lockedLatency, Duration totalLatency) {
- release(lockedLatency, totalLatency);
+ void releaseFailed(LatencyStats.ActiveInterval lockedInterval) {
+ release(lockedInterval);
releaseFailedCount.incrementAndGet();
cumulativeReleaseFailedCount.incrementAndGet();
}
@@ -85,14 +78,11 @@ public class LockMetrics {
public int getCumulativeReleaseCount() { return cumulativeReleaseCount.get(); }
public int getCumulativeReleaseFailedCount() { return cumulativeReleaseFailedCount.get(); }
- public int getAcquiringNow() { return acquiringNow.get(); }
- public int getLockedNow() { return lockedNow.get(); }
-
- public LatencyMetrics getAcquireLatencyMetrics() { return acquireLatencyStore.getLatencyMetrics(); }
- public LatencyMetrics getLockedLatencyMetrics() { return lockedLatencyStore.getLatencyMetrics(); }
+ public LatencyMetrics getAcquireLatencyMetrics() { return acquireStats.getLatencyMetrics(); }
+ public LatencyMetrics getLockedLatencyMetrics() { return lockedStats.getLatencyMetrics(); }
- public LatencyMetrics getAndResetAcquireLatencyMetrics() { return acquireLatencyStore.getAndResetLatencyMetrics(); }
- public LatencyMetrics getAndResetLockedLatencyMetrics() { return lockedLatencyStore.getAndResetLatencyMetrics(); }
+ public LatencyMetrics getAndResetAcquireLatencyMetrics() { return acquireStats.getLatencyMetricsAndStartNewPeriod(); }
+ public LatencyMetrics getAndResetLockedLatencyMetrics() { return lockedStats.getLatencyMetricsAndStartNewPeriod(); }
// For tests
void setAcquireCount(int count) { acquireCount.set(count); }
@@ -110,10 +100,6 @@ public class LockMetrics {
void setCumulativeReleaseCount(int count) { cumulativeReleaseCount.set(count); }
void setCumulativeReleaseFailedCount(int count) { cumulativeReleaseFailedCount.set(count); }
- // For tests
- void setAcquiringNow(int count) { acquiringNow.set(count); }
- void setLockedNow(int count) { lockedNow.set(count); }
-
@Override
public String toString() {
return "LockMetrics{" +
@@ -129,10 +115,8 @@ public class LockMetrics {
", cumulativeAcquireSucceededCount=" + cumulativeAcquireSucceededCount +
", cumulativeReleaseCount=" + cumulativeReleaseCount +
", cumulativeReleaseFailedCount=" + cumulativeReleaseFailedCount +
- ", acquiringNow=" + acquiringNow +
- ", lockedNow=" + lockedNow +
- ", acquireLatencyStore=" + acquireLatencyStore +
- ", lockedLatencyStore=" + lockedLatencyStore +
+ ", acquireStats=" + acquireStats +
+ ", lockedStats=" + lockedStats +
'}';
}
}