|
@@ -1,4 +1,3 @@
|
|
|
-
|
|
|
/**
|
|
|
* Licensed to the Apache Software Foundation (ASF) under one
|
|
|
* or more contributor license agreements. See the NOTICE file
|
|
@@ -43,11 +42,15 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPOR
|
|
|
/**
|
|
|
* Mimics a ReentrantReadWriteLock but does not directly implement the interface
|
|
|
* so more sophisticated locking capabilities and logging/metrics are possible.
|
|
|
- * If {@link org.apache.hadoop.hdfs.DFSConfigKeys#DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY}
|
|
|
- * is set to be true, metrics will be emitted into the FSNamesystem metrics
|
|
|
- * registry for each operation which acquires this lock indicating how long
|
|
|
- * the operation held the lock for. Note that if a thread dies, metrics produced
|
|
|
- * after the most recent snapshot will be lost due to the use of
|
|
|
+ * {@link org.apache.hadoop.hdfs.DFSConfigKeys#DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY}
|
|
|
+ * to be true, metrics will be emitted into the FSNamesystem metrics registry
|
|
|
+ * for each operation which acquires this lock indicating how long the operation
|
|
|
+ * held the lock for. These metrics have names of the form
|
|
|
+ * FSN(Read|Write)LockNanosOperationName, where OperationName denotes the name
|
|
|
+ * of the operation that initiated the lock hold (this will be OTHER for certain
|
|
|
+ * uncategorized operations) and they export the hold time values in
|
|
|
+ * nanoseconds. Note that if a thread dies, metrics produced after the
|
|
|
+ * most recent snapshot will be lost due to the use of
|
|
|
* {@link MutableRatesWithAggregation}. However since threads are re-used
|
|
|
* between operations this should not generally be an issue.
|
|
|
*/
|
|
@@ -63,24 +66,26 @@ class FSNamesystemLock {
|
|
|
* Log statements about long lock hold times will not be produced more
|
|
|
* frequently than this interval.
|
|
|
*/
|
|
|
- private final long lockSuppressWarningInterval;
|
|
|
+ private final long lockSuppressWarningIntervalMs;
|
|
|
|
|
|
/** Threshold (ms) for long holding write lock report. */
|
|
|
- private final long writeLockReportingThreshold;
|
|
|
+ private final long writeLockReportingThresholdMs;
|
|
|
/** Last time stamp for write lock. Keep the longest one for multi-entrance.*/
|
|
|
- private long writeLockHeldTimeStamp;
|
|
|
+ private long writeLockHeldTimeStampNanos;
|
|
|
private int numWriteLockWarningsSuppressed = 0;
|
|
|
- private long timeStampOfLastWriteLockReport = 0;
|
|
|
- private long longestWriteLockHeldInterval = 0;
|
|
|
+ /** Time stamp (ms) of the last time a write lock report was written. */
|
|
|
+ private long timeStampOfLastWriteLockReportMs = 0;
|
|
|
+ /** Longest time (ms) a write lock was held since the last report. */
|
|
|
+ private long longestWriteLockHeldIntervalMs = 0;
|
|
|
|
|
|
/** Threshold (ms) for long holding read lock report. */
|
|
|
- private final long readLockReportingThreshold;
|
|
|
+ private final long readLockReportingThresholdMs;
|
|
|
/**
|
|
|
* Last time stamp for read lock. Keep the longest one for
|
|
|
* multi-entrance. This is ThreadLocal since there could be
|
|
|
* many read locks held simultaneously.
|
|
|
*/
|
|
|
- private final ThreadLocal<Long> readLockHeldTimeStamp =
|
|
|
+ private final ThreadLocal<Long> readLockHeldTimeStampNanos =
|
|
|
new ThreadLocal<Long>() {
|
|
|
@Override
|
|
|
public Long initialValue() {
|
|
@@ -89,13 +94,16 @@ class FSNamesystemLock {
|
|
|
};
|
|
|
private final AtomicInteger numReadLockWarningsSuppressed =
|
|
|
new AtomicInteger(0);
|
|
|
- private final AtomicLong timeStampOfLastReadLockReport = new AtomicLong(0);
|
|
|
- private final AtomicLong longestReadLockHeldInterval = new AtomicLong(0);
|
|
|
+ /** Time stamp (ms) of the last time a read lock report was written. */
|
|
|
+ private final AtomicLong timeStampOfLastReadLockReportMs = new AtomicLong(0);
|
|
|
+ /** Longest time (ms) a read lock was held since the last report. */
|
|
|
+ private final AtomicLong longestReadLockHeldIntervalMs = new AtomicLong(0);
|
|
|
|
|
|
@VisibleForTesting
|
|
|
static final String OP_NAME_OTHER = "OTHER";
|
|
|
private static final String READ_LOCK_METRIC_PREFIX = "FSNReadLock";
|
|
|
private static final String WRITE_LOCK_METRIC_PREFIX = "FSNWriteLock";
|
|
|
+ private static final String LOCK_METRIC_SUFFIX = "Nanos";
|
|
|
|
|
|
FSNamesystemLock(Configuration conf,
|
|
|
MutableRatesWithAggregation detailedHoldTimeMetrics) {
|
|
@@ -110,13 +118,13 @@ class FSNamesystemLock {
|
|
|
this.coarseLock = new ReentrantReadWriteLock(fair);
|
|
|
this.timer = timer;
|
|
|
|
|
|
- this.writeLockReportingThreshold = conf.getLong(
|
|
|
+ this.writeLockReportingThresholdMs = conf.getLong(
|
|
|
DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY,
|
|
|
DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT);
|
|
|
- this.readLockReportingThreshold = conf.getLong(
|
|
|
+ this.readLockReportingThresholdMs = conf.getLong(
|
|
|
DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY,
|
|
|
DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_DEFAULT);
|
|
|
- this.lockSuppressWarningInterval = conf.getTimeDuration(
|
|
|
+ this.lockSuppressWarningIntervalMs = conf.getTimeDuration(
|
|
|
DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
|
|
|
DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS);
|
|
|
this.metricsEnabled = conf.getBoolean(
|
|
@@ -130,7 +138,7 @@ class FSNamesystemLock {
|
|
|
public void readLock() {
|
|
|
coarseLock.readLock().lock();
|
|
|
if (coarseLock.getReadHoldCount() == 1) {
|
|
|
- readLockHeldTimeStamp.set(timer.monotonicNow());
|
|
|
+ readLockHeldTimeStampNanos.set(timer.monotonicNowNanos());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -140,56 +148,59 @@ class FSNamesystemLock {
|
|
|
|
|
|
public void readUnlock(String opName) {
|
|
|
final boolean needReport = coarseLock.getReadHoldCount() == 1;
|
|
|
- final long readLockInterval =
|
|
|
- timer.monotonicNow() - readLockHeldTimeStamp.get();
|
|
|
+ final long readLockIntervalNanos =
|
|
|
+ timer.monotonicNowNanos() - readLockHeldTimeStampNanos.get();
|
|
|
coarseLock.readLock().unlock();
|
|
|
|
|
|
if (needReport) {
|
|
|
- addMetric(opName, readLockInterval, false);
|
|
|
- readLockHeldTimeStamp.remove();
|
|
|
+ addMetric(opName, readLockIntervalNanos, false);
|
|
|
+ readLockHeldTimeStampNanos.remove();
|
|
|
}
|
|
|
- if (needReport && readLockInterval >= this.readLockReportingThreshold) {
|
|
|
+ final long readLockIntervalMs =
|
|
|
+ TimeUnit.NANOSECONDS.toMillis(readLockIntervalNanos);
|
|
|
+ if (needReport && readLockIntervalMs >= this.readLockReportingThresholdMs) {
|
|
|
long localLongestReadLock;
|
|
|
do {
|
|
|
- localLongestReadLock = longestReadLockHeldInterval.get();
|
|
|
- } while (localLongestReadLock - readLockInterval < 0 &&
|
|
|
- !longestReadLockHeldInterval.compareAndSet(localLongestReadLock,
|
|
|
- readLockInterval));
|
|
|
+ localLongestReadLock = longestReadLockHeldIntervalMs.get();
|
|
|
+ } while (localLongestReadLock - readLockIntervalMs < 0 &&
|
|
|
+ !longestReadLockHeldIntervalMs.compareAndSet(localLongestReadLock,
|
|
|
+ readLockIntervalMs));
|
|
|
|
|
|
long localTimeStampOfLastReadLockReport;
|
|
|
- long now;
|
|
|
+ long nowMs;
|
|
|
do {
|
|
|
- now = timer.monotonicNow();
|
|
|
+ nowMs = timer.monotonicNow();
|
|
|
localTimeStampOfLastReadLockReport =
|
|
|
- timeStampOfLastReadLockReport.get();
|
|
|
- if (now - localTimeStampOfLastReadLockReport <
|
|
|
- lockSuppressWarningInterval) {
|
|
|
+ timeStampOfLastReadLockReportMs.get();
|
|
|
+ if (nowMs - localTimeStampOfLastReadLockReport <
|
|
|
+ lockSuppressWarningIntervalMs) {
|
|
|
numReadLockWarningsSuppressed.incrementAndGet();
|
|
|
return;
|
|
|
}
|
|
|
- } while (!timeStampOfLastReadLockReport.compareAndSet(
|
|
|
- localTimeStampOfLastReadLockReport, now));
|
|
|
+ } while (!timeStampOfLastReadLockReportMs.compareAndSet(
|
|
|
+ localTimeStampOfLastReadLockReport, nowMs));
|
|
|
int numSuppressedWarnings = numReadLockWarningsSuppressed.getAndSet(0);
|
|
|
- long longestLockHeldInterval = longestReadLockHeldInterval.getAndSet(0);
|
|
|
+ long longestLockHeldIntervalMs =
|
|
|
+ longestReadLockHeldIntervalMs.getAndSet(0);
|
|
|
FSNamesystem.LOG.info("FSNamesystem read lock held for " +
|
|
|
- readLockInterval + " ms via\n" +
|
|
|
+ readLockIntervalMs + " ms via\n" +
|
|
|
StringUtils.getStackTrace(Thread.currentThread()) +
|
|
|
"\tNumber of suppressed read-lock reports: " + numSuppressedWarnings +
|
|
|
- "\n\tLongest read-lock held interval: " + longestLockHeldInterval);
|
|
|
+ "\n\tLongest read-lock held interval: " + longestLockHeldIntervalMs);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
public void writeLock() {
|
|
|
coarseLock.writeLock().lock();
|
|
|
if (coarseLock.getWriteHoldCount() == 1) {
|
|
|
- writeLockHeldTimeStamp = timer.monotonicNow();
|
|
|
+ writeLockHeldTimeStampNanos = timer.monotonicNowNanos();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
public void writeLockInterruptibly() throws InterruptedException {
|
|
|
coarseLock.writeLock().lockInterruptibly();
|
|
|
if (coarseLock.getWriteHoldCount() == 1) {
|
|
|
- writeLockHeldTimeStamp = timer.monotonicNow();
|
|
|
+ writeLockHeldTimeStampNanos = timer.monotonicNowNanos();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -200,24 +211,29 @@ class FSNamesystemLock {
|
|
|
public void writeUnlock(String opName) {
|
|
|
final boolean needReport = coarseLock.getWriteHoldCount() == 1 &&
|
|
|
coarseLock.isWriteLockedByCurrentThread();
|
|
|
- final long currentTime = timer.monotonicNow();
|
|
|
- final long writeLockInterval = currentTime - writeLockHeldTimeStamp;
|
|
|
+ final long currentTimeNanos = timer.monotonicNowNanos();
|
|
|
+ final long writeLockIntervalNanos =
|
|
|
+ currentTimeNanos - writeLockHeldTimeStampNanos;
|
|
|
+ final long currentTimeMs = TimeUnit.NANOSECONDS.toMillis(currentTimeNanos);
|
|
|
+ final long writeLockIntervalMs =
|
|
|
+ TimeUnit.NANOSECONDS.toMillis(writeLockIntervalNanos);
|
|
|
|
|
|
boolean logReport = false;
|
|
|
int numSuppressedWarnings = 0;
|
|
|
- long longestLockHeldInterval = 0;
|
|
|
- if (needReport && writeLockInterval >= this.writeLockReportingThreshold) {
|
|
|
- if (writeLockInterval > longestWriteLockHeldInterval) {
|
|
|
- longestWriteLockHeldInterval = writeLockInterval;
|
|
|
+ long longestLockHeldIntervalMs = 0;
|
|
|
+ if (needReport &&
|
|
|
+ writeLockIntervalMs >= this.writeLockReportingThresholdMs) {
|
|
|
+ if (writeLockIntervalMs > longestWriteLockHeldIntervalMs) {
|
|
|
+ longestWriteLockHeldIntervalMs = writeLockIntervalMs;
|
|
|
}
|
|
|
- if (currentTime - timeStampOfLastWriteLockReport >
|
|
|
- this.lockSuppressWarningInterval) {
|
|
|
+ if (currentTimeMs - timeStampOfLastWriteLockReportMs >
|
|
|
+ this.lockSuppressWarningIntervalMs) {
|
|
|
logReport = true;
|
|
|
numSuppressedWarnings = numWriteLockWarningsSuppressed;
|
|
|
numWriteLockWarningsSuppressed = 0;
|
|
|
- longestLockHeldInterval = longestWriteLockHeldInterval;
|
|
|
- longestWriteLockHeldInterval = 0;
|
|
|
- timeStampOfLastWriteLockReport = currentTime;
|
|
|
+ longestLockHeldIntervalMs = longestWriteLockHeldIntervalMs;
|
|
|
+ longestWriteLockHeldIntervalMs = 0;
|
|
|
+ timeStampOfLastWriteLockReportMs = currentTimeMs;
|
|
|
} else {
|
|
|
numWriteLockWarningsSuppressed++;
|
|
|
}
|
|
@@ -226,16 +242,16 @@ class FSNamesystemLock {
|
|
|
coarseLock.writeLock().unlock();
|
|
|
|
|
|
if (needReport) {
|
|
|
- addMetric(opName, writeLockInterval, true);
|
|
|
+ addMetric(opName, writeLockIntervalNanos, true);
|
|
|
}
|
|
|
|
|
|
if (logReport) {
|
|
|
FSNamesystem.LOG.info("FSNamesystem write lock held for " +
|
|
|
- writeLockInterval + " ms via\n" +
|
|
|
+ writeLockIntervalMs + " ms via\n" +
|
|
|
StringUtils.getStackTrace(Thread.currentThread()) +
|
|
|
"\tNumber of suppressed write-lock reports: " +
|
|
|
numSuppressedWarnings + "\n\tLongest write-lock held interval: " +
|
|
|
- longestLockHeldInterval);
|
|
|
+ longestLockHeldIntervalMs);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -269,13 +285,14 @@ class FSNamesystemLock {
|
|
|
/**
|
|
|
* Add the lock hold time for a recent operation to the metrics.
|
|
|
* @param operationName Name of the operation for which to record the time
|
|
|
- * @param value Length of time the lock was held
|
|
|
+ * @param value Length of time the lock was held (nanoseconds)
|
|
|
*/
|
|
|
private void addMetric(String operationName, long value, boolean isWrite) {
|
|
|
if (metricsEnabled) {
|
|
|
String metricName =
|
|
|
(isWrite ? WRITE_LOCK_METRIC_PREFIX : READ_LOCK_METRIC_PREFIX) +
|
|
|
- org.apache.commons.lang.StringUtils.capitalize(operationName);
|
|
|
+ org.apache.commons.lang.StringUtils.capitalize(operationName) +
|
|
|
+ LOCK_METRIC_SUFFIX;
|
|
|
detailedHoldTimeMetrics.add(metricName, value);
|
|
|
}
|
|
|
}
|