|
@@ -55,8 +55,10 @@ public class InstrumentedLock implements Lock {
|
|
|
|
|
|
// Tracking counters for lock statistics.
|
|
// Tracking counters for lock statistics.
|
|
private volatile long lockAcquireTimestamp;
|
|
private volatile long lockAcquireTimestamp;
|
|
- private final AtomicLong lastLogTimestamp;
|
|
|
|
- private final AtomicLong warningsSuppressed = new AtomicLong(0);
|
|
|
|
|
|
+ private final AtomicLong lastHoldLogTimestamp;
|
|
|
|
+ private final AtomicLong lastWaitLogTimestamp;
|
|
|
|
+ private final SuppressedStats holdStats = new SuppressedStats();
|
|
|
|
+ private final SuppressedStats waitStats = new SuppressedStats();
|
|
|
|
|
|
/**
|
|
/**
|
|
* Create a instrumented lock instance which logs a warning message
|
|
* Create a instrumented lock instance which logs a warning message
|
|
@@ -91,19 +93,24 @@ public class InstrumentedLock implements Lock {
|
|
this.logger = logger;
|
|
this.logger = logger;
|
|
minLoggingGap = minLoggingGapMs;
|
|
minLoggingGap = minLoggingGapMs;
|
|
lockWarningThreshold = lockWarningThresholdMs;
|
|
lockWarningThreshold = lockWarningThresholdMs;
|
|
- lastLogTimestamp = new AtomicLong(
|
|
|
|
|
|
+ lastHoldLogTimestamp = new AtomicLong(
|
|
clock.monotonicNow() - Math.max(minLoggingGap, lockWarningThreshold));
|
|
clock.monotonicNow() - Math.max(minLoggingGap, lockWarningThreshold));
|
|
|
|
+ lastWaitLogTimestamp = new AtomicLong(lastHoldLogTimestamp.get());
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void lock() {
|
|
public void lock() {
|
|
|
|
+ long waitStart = clock.monotonicNow();
|
|
lock.lock();
|
|
lock.lock();
|
|
|
|
+ check(waitStart, clock.monotonicNow(), false);
|
|
startLockTiming();
|
|
startLockTiming();
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void lockInterruptibly() throws InterruptedException {
|
|
public void lockInterruptibly() throws InterruptedException {
|
|
|
|
+ long waitStart = clock.monotonicNow();
|
|
lock.lockInterruptibly();
|
|
lock.lockInterruptibly();
|
|
|
|
+ check(waitStart, clock.monotonicNow(), false);
|
|
startLockTiming();
|
|
startLockTiming();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -118,11 +125,14 @@ public class InstrumentedLock implements Lock {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public boolean tryLock(long time, TimeUnit unit) throws InterruptedException {
|
|
public boolean tryLock(long time, TimeUnit unit) throws InterruptedException {
|
|
|
|
+ long waitStart = clock.monotonicNow();
|
|
|
|
+ boolean retval = false;
|
|
if (lock.tryLock(time, unit)) {
|
|
if (lock.tryLock(time, unit)) {
|
|
startLockTiming();
|
|
startLockTiming();
|
|
- return true;
|
|
|
|
|
|
+ retval = true;
|
|
}
|
|
}
|
|
- return false;
|
|
|
|
|
|
+ check(waitStart, clock.monotonicNow(), false);
|
|
|
|
+ return retval;
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -130,7 +140,7 @@ public class InstrumentedLock implements Lock {
|
|
long localLockReleaseTime = clock.monotonicNow();
|
|
long localLockReleaseTime = clock.monotonicNow();
|
|
long localLockAcquireTime = lockAcquireTimestamp;
|
|
long localLockAcquireTime = lockAcquireTimestamp;
|
|
lock.unlock();
|
|
lock.unlock();
|
|
- check(localLockAcquireTime, localLockReleaseTime);
|
|
|
|
|
|
+ check(localLockAcquireTime, localLockReleaseTime, true);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -139,12 +149,25 @@ public class InstrumentedLock implements Lock {
|
|
}
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
- void logWarning(long lockHeldTime, long suppressed) {
|
|
|
|
|
|
+ void logWarning(long lockHeldTime, SuppressedSnapshot stats) {
|
|
logger.warn(String.format("Lock held time above threshold: " +
|
|
logger.warn(String.format("Lock held time above threshold: " +
|
|
"lock identifier: %s " +
|
|
"lock identifier: %s " +
|
|
"lockHeldTimeMs=%d ms. Suppressed %d lock warnings. " +
|
|
"lockHeldTimeMs=%d ms. Suppressed %d lock warnings. " +
|
|
|
|
+ "Longest suppressed LockHeldTimeMs=%d. " +
|
|
"The stack trace is: %s" ,
|
|
"The stack trace is: %s" ,
|
|
- name, lockHeldTime, suppressed,
|
|
|
|
|
|
+ name, lockHeldTime, stats.getSuppressedCount(),
|
|
|
|
+ stats.getMaxSuppressedWait(),
|
|
|
|
+ StringUtils.getStackTrace(Thread.currentThread())));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ void logWaitWarning(long lockWaitTime, SuppressedSnapshot stats) {
|
|
|
|
+ logger.warn(String.format("Waited above threshold to acquire lock: " +
|
|
|
|
+ "lock identifier: %s " +
|
|
|
|
+ "waitTimeMs=%d ms. Suppressed %d lock wait warnings. " +
|
|
|
|
+ "Longest suppressed WaitTimeMs=%d. " +
|
|
|
|
+ "The stack trace is: %s", name, lockWaitTime,
|
|
|
|
+ stats.getSuppressedCount(), stats.getMaxSuppressedWait(),
|
|
StringUtils.getStackTrace(Thread.currentThread())));
|
|
StringUtils.getStackTrace(Thread.currentThread())));
|
|
}
|
|
}
|
|
|
|
|
|
@@ -163,27 +186,41 @@ public class InstrumentedLock implements Lock {
|
|
* @param acquireTime - timestamp just after acquiring the lock.
|
|
* @param acquireTime - timestamp just after acquiring the lock.
|
|
* @param releaseTime - timestamp just before releasing the lock.
|
|
* @param releaseTime - timestamp just before releasing the lock.
|
|
*/
|
|
*/
|
|
- protected void check(long acquireTime, long releaseTime) {
|
|
|
|
|
|
+ protected void check(long acquireTime, long releaseTime,
|
|
|
|
+ boolean checkLockHeld) {
|
|
if (!logger.isWarnEnabled()) {
|
|
if (!logger.isWarnEnabled()) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
|
|
final long lockHeldTime = releaseTime - acquireTime;
|
|
final long lockHeldTime = releaseTime - acquireTime;
|
|
if (lockWarningThreshold - lockHeldTime < 0) {
|
|
if (lockWarningThreshold - lockHeldTime < 0) {
|
|
|
|
+ AtomicLong lastLogTime;
|
|
|
|
+ SuppressedStats stats;
|
|
|
|
+ if (checkLockHeld) {
|
|
|
|
+ lastLogTime = lastHoldLogTimestamp;
|
|
|
|
+ stats = holdStats;
|
|
|
|
+ } else {
|
|
|
|
+ lastLogTime = lastWaitLogTimestamp;
|
|
|
|
+ stats = waitStats;
|
|
|
|
+ }
|
|
long now;
|
|
long now;
|
|
long localLastLogTs;
|
|
long localLastLogTs;
|
|
do {
|
|
do {
|
|
now = clock.monotonicNow();
|
|
now = clock.monotonicNow();
|
|
- localLastLogTs = lastLogTimestamp.get();
|
|
|
|
|
|
+ localLastLogTs = lastLogTime.get();
|
|
long deltaSinceLastLog = now - localLastLogTs;
|
|
long deltaSinceLastLog = now - localLastLogTs;
|
|
// check should print log or not
|
|
// check should print log or not
|
|
if (deltaSinceLastLog - minLoggingGap < 0) {
|
|
if (deltaSinceLastLog - minLoggingGap < 0) {
|
|
- warningsSuppressed.incrementAndGet();
|
|
|
|
|
|
+ stats.incrementSuppressed(lockHeldTime);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
- } while (!lastLogTimestamp.compareAndSet(localLastLogTs, now));
|
|
|
|
- long suppressed = warningsSuppressed.getAndSet(0);
|
|
|
|
- logWarning(lockHeldTime, suppressed);
|
|
|
|
|
|
+ } while (!lastLogTime.compareAndSet(localLastLogTs, now));
|
|
|
|
+ SuppressedSnapshot statsSnapshot = stats.snapshot();
|
|
|
|
+ if (checkLockHeld) {
|
|
|
|
+ logWarning(lockHeldTime, statsSnapshot);
|
|
|
|
+ } else {
|
|
|
|
+ logWaitWarning(lockHeldTime, statsSnapshot);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -194,4 +231,60 @@ public class InstrumentedLock implements Lock {
|
|
protected Timer getTimer() {
|
|
protected Timer getTimer() {
|
|
return clock;
|
|
return clock;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Internal class to track statistics about suppressed log messages in an
|
|
|
|
+ * atomic way.
|
|
|
|
+ */
|
|
|
|
+ private static class SuppressedStats {
|
|
|
|
+ private long suppressedCount = 0;
|
|
|
|
+ private long maxSuppressedWait = 0;
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Increments the suppressed counter and increases the max wait time if the
|
|
|
|
+ * passed wait is greater than the current maxSuppressedWait.
|
|
|
|
+ * @param wait The wait time for this suppressed message
|
|
|
|
+ */
|
|
|
|
+ synchronized public void incrementSuppressed(long wait) {
|
|
|
|
+ suppressedCount++;
|
|
|
|
+ if (wait > maxSuppressedWait) {
|
|
|
|
+ maxSuppressedWait = wait;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Captures the current value of the counts into a SuppressedSnapshot object
|
|
|
|
+ * and resets the values to zero.
|
|
|
|
+ *
|
|
|
|
+ * @return SuppressedSnapshot containing the current value of the counters
|
|
|
|
+ */
|
|
|
|
+ synchronized public SuppressedSnapshot snapshot() {
|
|
|
|
+ SuppressedSnapshot snap =
|
|
|
|
+ new SuppressedSnapshot(suppressedCount, maxSuppressedWait);
|
|
|
|
+ suppressedCount = 0;
|
|
|
|
+ maxSuppressedWait = 0;
|
|
|
|
+ return snap;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Immutable class to capture a snapshot of suppressed log message stats.
|
|
|
|
+ */
|
|
|
|
+ protected static class SuppressedSnapshot {
|
|
|
|
+ private long suppressedCount = 0;
|
|
|
|
+ private long maxSuppressedWait = 0;
|
|
|
|
+
|
|
|
|
+ public SuppressedSnapshot(long suppressedCount, long maxWait) {
|
|
|
|
+ this.suppressedCount = suppressedCount;
|
|
|
|
+ this.maxSuppressedWait = maxWait;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public long getMaxSuppressedWait() {
|
|
|
|
+ return maxSuppressedWait;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public long getSuppressedCount() {
|
|
|
|
+ return suppressedCount;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|