|
@@ -26,12 +26,12 @@ import java.util.concurrent.locks.Condition;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
-import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
|
import org.apache.hadoop.log.LogThrottlingHelper;
|
|
|
import org.apache.hadoop.metrics2.lib.MutableRatesWithAggregation;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
+import org.apache.hadoop.util.Time;
|
|
|
import org.apache.hadoop.util.Timer;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT;
|
|
@@ -105,14 +105,9 @@ class FSNamesystemLock {
|
|
|
* The info (lock held time and stack trace) when longest time (ms) a read
|
|
|
* lock was held since the last report.
|
|
|
*/
|
|
|
- private final AtomicReference<ReadLockHeldInfo> longestReadLockHeldInfo =
|
|
|
- new AtomicReference<>(new ReadLockHeldInfo(0, null));
|
|
|
-
|
|
|
- /**
|
|
|
- * The stack trace when longest time of the write lock
|
|
|
- * was held since the last report.
|
|
|
- */
|
|
|
- private volatile String longestWriteLockHeldStackTrace;
|
|
|
+ private final AtomicReference<LockHeldInfo> longestReadLockHeldInfo =
|
|
|
+ new AtomicReference<>(new LockHeldInfo(0, 0, null));
|
|
|
+ private LockHeldInfo longestWriteLockHeldInfo = new LockHeldInfo(0, 0, null);
|
|
|
|
|
|
@VisibleForTesting
|
|
|
static final String OP_NAME_OTHER = "OTHER";
|
|
@@ -169,8 +164,9 @@ class FSNamesystemLock {
|
|
|
|
|
|
public void readUnlock(String opName) {
|
|
|
final boolean needReport = coarseLock.getReadHoldCount() == 1;
|
|
|
+ final long currentTimeStampNanos = timer.monotonicNowNanos();
|
|
|
final long readLockIntervalNanos =
|
|
|
- timer.monotonicNowNanos() - readLockHeldTimeStampNanos.get();
|
|
|
+ currentTimeStampNanos - readLockHeldTimeStampNanos.get();
|
|
|
coarseLock.readLock().unlock();
|
|
|
|
|
|
if (needReport) {
|
|
@@ -179,13 +175,15 @@ class FSNamesystemLock {
|
|
|
}
|
|
|
final long readLockIntervalMs =
|
|
|
TimeUnit.NANOSECONDS.toMillis(readLockIntervalNanos);
|
|
|
+ final long currentTimeMs =
|
|
|
+ TimeUnit.NANOSECONDS.toMillis(currentTimeStampNanos);
|
|
|
if (needReport && readLockIntervalMs >= this.readLockReportingThresholdMs) {
|
|
|
- ReadLockHeldInfo localLockHeldInfo;
|
|
|
+ LockHeldInfo localLockHeldInfo;
|
|
|
do {
|
|
|
localLockHeldInfo = longestReadLockHeldInfo.get();
|
|
|
} while (localLockHeldInfo.getIntervalMs() - readLockIntervalMs < 0 &&
|
|
|
!longestReadLockHeldInfo.compareAndSet(localLockHeldInfo,
|
|
|
- new ReadLockHeldInfo(readLockIntervalMs,
|
|
|
+ new LockHeldInfo(currentTimeMs, readLockIntervalMs,
|
|
|
StringUtils.getStackTrace(Thread.currentThread()))));
|
|
|
|
|
|
long localTimeStampOfLastReadLockReport;
|
|
@@ -202,13 +200,13 @@ class FSNamesystemLock {
|
|
|
} while (!timeStampOfLastReadLockReportMs.compareAndSet(
|
|
|
localTimeStampOfLastReadLockReport, nowMs));
|
|
|
int numSuppressedWarnings = numReadLockWarningsSuppressed.getAndSet(0);
|
|
|
- ReadLockHeldInfo lockHeldInfo = longestReadLockHeldInfo
|
|
|
- .getAndSet(new ReadLockHeldInfo(0, null));
|
|
|
+ LockHeldInfo lockHeldInfo =
|
|
|
+ longestReadLockHeldInfo.getAndSet(new LockHeldInfo(0, 0, null));
|
|
|
FSNamesystem.LOG.info(
|
|
|
- "\tNumber of suppressed read-lock reports: {}" +
|
|
|
- "\n\tLongest read-lock held interval: {}ms via {}",
|
|
|
- numSuppressedWarnings, lockHeldInfo.getIntervalMs(),
|
|
|
- lockHeldInfo.getStackTrace());
|
|
|
+ "\tNumber of suppressed read-lock reports: {}"
|
|
|
+ + "\n\tLongest read-lock held at {} for {}ms via {}",
|
|
|
+ numSuppressedWarnings, Time.formatTime(lockHeldInfo.getStartTimeMs()),
|
|
|
+ lockHeldInfo.getIntervalMs(), lockHeldInfo.getStackTrace());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -258,18 +256,21 @@ class FSNamesystemLock {
|
|
|
LogAction logAction = LogThrottlingHelper.DO_NOT_LOG;
|
|
|
if (needReport &&
|
|
|
writeLockIntervalMs >= this.writeLockReportingThresholdMs) {
|
|
|
- SummaryStatistics currentStats =
|
|
|
- writeLockReportLogger.getCurrentStats("write", 0);
|
|
|
- double currentMaxTime = currentStats != null ? currentStats.getMax() : 0;
|
|
|
- if (currentMaxTime < writeLockIntervalMs) {
|
|
|
- longestWriteLockHeldStackTrace =
|
|
|
- StringUtils.getStackTrace(Thread.currentThread());
|
|
|
+ if (longestWriteLockHeldInfo.getIntervalMs() < writeLockIntervalMs) {
|
|
|
+ longestWriteLockHeldInfo =
|
|
|
+ new LockHeldInfo(currentTimeMs, writeLockIntervalMs,
|
|
|
+ StringUtils.getStackTrace(Thread.currentThread()));
|
|
|
}
|
|
|
|
|
|
logAction = writeLockReportLogger
|
|
|
.record("write", currentTimeMs, writeLockIntervalMs);
|
|
|
}
|
|
|
|
|
|
+ LockHeldInfo lockHeldInfo = longestWriteLockHeldInfo;
|
|
|
+ if (logAction.shouldLog()) {
|
|
|
+ longestWriteLockHeldInfo = new LockHeldInfo(0, 0, null);
|
|
|
+ }
|
|
|
+
|
|
|
coarseLock.writeLock().unlock();
|
|
|
|
|
|
if (needReport) {
|
|
@@ -278,12 +279,13 @@ class FSNamesystemLock {
|
|
|
|
|
|
if (logAction.shouldLog()) {
|
|
|
FSNamesystem.LOG.info(
|
|
|
- "\tNumber of suppressed write-lock reports: {}" +
|
|
|
- "\n\tLongest write-lock held interval: {}ms via {} " +
|
|
|
- "\n\tTotal suppressed write-lock held time: {}",
|
|
|
- logAction.getCount() - 1, logAction.getStats(0).getMax(),
|
|
|
- longestWriteLockHeldStackTrace,
|
|
|
- logAction.getStats(0).getSum() - writeLockIntervalMs);
|
|
|
+ "\tNumber of suppressed write-lock reports: {}"
|
|
|
+ + "\n\tLongest write-lock held at {} for {}ms via {}"
|
|
|
+ + "\n\tTotal suppressed write-lock held time: {}",
|
|
|
+ logAction.getCount() - 1,
|
|
|
+ Time.formatTime(lockHeldInfo.getStartTimeMs()),
|
|
|
+ lockHeldInfo.getIntervalMs(), lockHeldInfo.getStackTrace(),
|
|
|
+ logAction.getStats(0).getSum() - lockHeldInfo.getIntervalMs());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -382,17 +384,24 @@ class FSNamesystemLock {
|
|
|
/**
|
|
|
* Read lock Held Info.
|
|
|
*/
|
|
|
- private static class ReadLockHeldInfo {
|
|
|
- /** Read lock held time. */
|
|
|
+ private static class LockHeldInfo {
|
|
|
+ /** Lock held start time. */
|
|
|
+ private Long startTimeMs;
|
|
|
+ /** Lock held time. */
|
|
|
private Long intervalMs;
|
|
|
- /** The stack trace read lock was held. */
|
|
|
+ /** The stack trace lock was held. */
|
|
|
private String stackTrace;
|
|
|
|
|
|
- ReadLockHeldInfo(long intervalMs, String stackTrace) {
|
|
|
+ LockHeldInfo(long startTimeMs, long intervalMs, String stackTrace) {
|
|
|
+ this.startTimeMs = startTimeMs;
|
|
|
this.intervalMs = intervalMs;
|
|
|
this.stackTrace = stackTrace;
|
|
|
}
|
|
|
|
|
|
+ public Long getStartTimeMs() {
|
|
|
+ return this.startTimeMs;
|
|
|
+ }
|
|
|
+
|
|
|
public Long getIntervalMs() {
|
|
|
return this.intervalMs;
|
|
|
}
|
|
@@ -400,16 +409,5 @@ class FSNamesystemLock {
|
|
|
public String getStackTrace() {
|
|
|
return this.stackTrace;
|
|
|
}
|
|
|
-
|
|
|
- @Override
|
|
|
- public int hashCode() {
|
|
|
- return this.intervalMs.hashCode();
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public boolean equals(Object obj) {
|
|
|
- return obj instanceof ReadLockHeldInfo && ((ReadLockHeldInfo) obj)
|
|
|
- .getIntervalMs().compareTo(intervalMs) == 0;
|
|
|
- }
|
|
|
}
|
|
|
}
|