HADOOP-18462. InstrumentedWriteLock should consider Reentrant case (#4919). Contributed by ZanderXu.

Reviewed-by: Ashutosh Gupta <ashugpt@amazon.com>
Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
This commit is contained in:
ZanderXu 2022-10-17 12:44:25 +08:00 committed by GitHub
parent 4ff6c9b8de
commit 136291d2d5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 134 additions and 6 deletions

View File

@ -44,12 +44,7 @@ public class InstrumentedReadLock extends InstrumentedLock {
* there can be multiple threads that hold the read lock concurrently.
*/
private final ThreadLocal<Long> readLockHeldTimeStamp =
new ThreadLocal<Long>() {
@Override
protected Long initialValue() {
return Long.MAX_VALUE;
};
};
ThreadLocal.withInitial(() -> Long.MAX_VALUE);
public InstrumentedReadLock(String name, Logger logger,
ReentrantReadWriteLock readWriteLock,

View File

@ -37,6 +37,9 @@
@InterfaceStability.Unstable
public class InstrumentedWriteLock extends InstrumentedLock {
private final ReentrantReadWriteLock readWriteLock;
private volatile long writeLockHeldTimeStamp = 0;
public InstrumentedWriteLock(String name, Logger logger,
ReentrantReadWriteLock readWriteLock,
long minLoggingGapMs, long lockWarningThresholdMs) {
@ -50,5 +53,28 @@ public InstrumentedWriteLock(String name, Logger logger,
long minLoggingGapMs, long lockWarningThresholdMs, Timer clock) {
super(name, logger, readWriteLock.writeLock(), minLoggingGapMs,
lockWarningThresholdMs, clock);
this.readWriteLock = readWriteLock;
}
@Override
public void unlock() {
boolean needReport = readWriteLock.getWriteHoldCount() == 1;
long localWriteReleaseTime = getTimer().monotonicNow();
long localWriteAcquireTime = writeLockHeldTimeStamp;
getLock().unlock();
if (needReport) {
writeLockHeldTimeStamp = 0;
check(localWriteAcquireTime, localWriteReleaseTime, true);
}
}
/**
* Starts timing for the instrumented write lock.
*/
@Override
protected void startLockTiming() {
if (readWriteLock.getWriteHoldCount() == 1) {
writeLockHeldTimeStamp = getTimer().monotonicNow();
}
}
}

View File

@ -233,4 +233,111 @@ protected void logWarning(long lockHeldTime, SuppressedSnapshot stats) {
assertEquals(2, wlogged.get());
assertEquals(1, wsuppresed.get());
}
/**
* Tests the warning when the write lock is held longer than threshold.
*/
@Test(timeout=10000)
public void testWriteLockLongHoldingReportWithReentrant() {
String testname = name.getMethodName();
final AtomicLong time = new AtomicLong(0);
Timer mclock = new Timer() {
@Override
public long monotonicNow() {
return time.get();
}
};
final AtomicLong wlogged = new AtomicLong(0);
final AtomicLong wsuppresed = new AtomicLong(0);
final AtomicLong totalHeldTime = new AtomicLong(0);
ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
InstrumentedWriteLock writeLock = new InstrumentedWriteLock(testname, LOG,
readWriteLock, 2000, 300, mclock) {
@Override
protected void logWarning(long lockHeldTime, SuppressedSnapshot stats) {
totalHeldTime.addAndGet(lockHeldTime);
wlogged.incrementAndGet();
wsuppresed.set(stats.getSuppressedCount());
}
};
InstrumentedReadLock readLock = new InstrumentedReadLock(testname, LOG,
readWriteLock, 2000, 300, mclock) {
@Override
protected void logWarning(long lockHeldTime, SuppressedSnapshot stats) {
totalHeldTime.addAndGet(lockHeldTime);
wlogged.incrementAndGet();
wsuppresed.set(stats.getSuppressedCount());
}
};
writeLock.lock(); // t = 0
time.set(100);
writeLock.lock(); // t = 100
time.set(500);
writeLock.lock(); // t = 500
time.set(2900);
writeLock.unlock(); // t = 2900
readLock.lock(); // t = 2900
time.set(3000);
readLock.unlock(); // t = 3000
writeLock.unlock(); // t = 3000
writeLock.unlock(); // t = 3000
assertEquals(1, wlogged.get());
assertEquals(0, wsuppresed.get());
assertEquals(3000, totalHeldTime.get());
}
/**
* Tests the warning when the read lock is held longer than threshold.
*/
@Test(timeout=10000)
public void testReadLockLongHoldingReportWithReentrant() {
String testname = name.getMethodName();
final AtomicLong time = new AtomicLong(0);
Timer mclock = new Timer() {
@Override
public long monotonicNow() {
return time.get();
}
};
final AtomicLong wlogged = new AtomicLong(0);
final AtomicLong wsuppresed = new AtomicLong(0);
final AtomicLong totalHelpTime = new AtomicLong(0);
ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
InstrumentedReadLock readLock = new InstrumentedReadLock(testname, LOG,
readWriteLock, 2000, 300, mclock) {
@Override
protected void logWarning(long lockHeldTime, SuppressedSnapshot stats) {
totalHelpTime.addAndGet(lockHeldTime);
wlogged.incrementAndGet();
wsuppresed.set(stats.getSuppressedCount());
}
};
readLock.lock(); // t = 0
time.set(100);
readLock.lock(); // t = 100
time.set(500);
readLock.lock(); // t = 500
time.set(3000);
readLock.unlock(); // t = 3000
readLock.unlock(); // t = 3000
readLock.unlock(); // t = 3000
assertEquals(1, wlogged.get());
assertEquals(0, wsuppresed.get());
assertEquals(3000, totalHelpTime.get());
}
}