|
@@ -18,6 +18,9 @@
|
|
|
|
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
|
|
+import java.util.ArrayList;
|
|
|
|
+import java.util.Collection;
|
|
|
|
+import java.util.Iterator;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
@@ -29,6 +32,7 @@ import java.util.function.Supplier;
|
|
|
|
|
|
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
|
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.INodeMap.INodeMapLock;
|
|
import org.apache.hadoop.ipc.Server;
|
|
import org.apache.hadoop.ipc.Server;
|
|
import org.apache.hadoop.log.LogThrottlingHelper;
|
|
import org.apache.hadoop.log.LogThrottlingHelper;
|
|
import org.apache.hadoop.metrics2.lib.MutableRatesWithAggregation;
|
|
import org.apache.hadoop.metrics2.lib.MutableRatesWithAggregation;
|
|
@@ -129,6 +133,32 @@ class FSNamesystemLock {
|
|
|
|
|
|
private static final String OVERALL_METRIC_NAME = "Overall";
|
|
private static final String OVERALL_METRIC_NAME = "Overall";
|
|
|
|
|
|
|
|
+ private final ThreadLocal<Collection<INodeMapLock>> partitionLocks =
|
|
|
|
+ new ThreadLocal<Collection<INodeMapLock>>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Collection<INodeMapLock> initialValue() {
|
|
|
|
+ return new ArrayList<INodeMapLock>();
|
|
|
|
+ }
|
|
|
|
+ };
|
|
|
|
+
|
|
|
|
+ void addChildLock(INodeMapLock lock) {
|
|
|
|
+ partitionLocks.get().add(lock);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ boolean removeChildLock(INodeMapLock lock) {
|
|
|
|
+ return partitionLocks.get().remove(lock);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ boolean haswWriteChildLock() {
|
|
|
|
+ Iterator<INodeMapLock> iter = partitionLocks.get().iterator();
|
|
|
|
+ // FSNamesystem.LOG.debug("partitionLocks.size = {}", partitionLocks.get().size());
|
|
|
|
+ while(iter.hasNext()) {
|
|
|
|
+ if(iter.next().hasWriteChildLock())
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+
|
|
FSNamesystemLock(Configuration conf,
|
|
FSNamesystemLock(Configuration conf,
|
|
MutableRatesWithAggregation detailedHoldTimeMetrics) {
|
|
MutableRatesWithAggregation detailedHoldTimeMetrics) {
|
|
this(conf, detailedHoldTimeMetrics, new Timer());
|
|
this(conf, detailedHoldTimeMetrics, new Timer());
|
|
@@ -180,11 +210,29 @@ class FSNamesystemLock {
|
|
|
|
|
|
public void readUnlock(String opName,
|
|
public void readUnlock(String opName,
|
|
Supplier<String> lockReportInfoSupplier) {
|
|
Supplier<String> lockReportInfoSupplier) {
|
|
|
|
+ readUnlock(opName, lockReportInfoSupplier, true);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void readUnlock(String opName,
|
|
|
|
+ Supplier<String> lockReportInfoSupplier,
|
|
|
|
+ boolean unlockChildren) {
|
|
final boolean needReport = coarseLock.getReadHoldCount() == 1;
|
|
final boolean needReport = coarseLock.getReadHoldCount() == 1;
|
|
final long readLockIntervalNanos =
|
|
final long readLockIntervalNanos =
|
|
timer.monotonicNowNanos() - readLockHeldTimeStampNanos.get();
|
|
timer.monotonicNowNanos() - readLockHeldTimeStampNanos.get();
|
|
final long currentTimeMs = timer.now();
|
|
final long currentTimeMs = timer.now();
|
|
- coarseLock.readLock().unlock();
|
|
|
|
|
|
+
|
|
|
|
+ if(getReadHoldCount() > 0) { // Current thread holds the lock
|
|
|
|
+ // Unlock the top FSNamesystemLock
|
|
|
|
+ coarseLock.readLock().unlock();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if(unlockChildren) { // Also unlock and remove children locks
|
|
|
|
+ Iterator<INodeMapLock> iter = partitionLocks.get().iterator();
|
|
|
|
+ while(iter.hasNext()) {
|
|
|
|
+ iter.next().readChildUnlock();
|
|
|
|
+ iter.remove();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
if (needReport) {
|
|
if (needReport) {
|
|
addMetric(opName, readLockIntervalNanos, false);
|
|
addMetric(opName, readLockIntervalNanos, false);
|
|
@@ -252,7 +300,7 @@ class FSNamesystemLock {
|
|
* FSNamesystemLock#writeUnlock(String, boolean, Supplier)}
|
|
* FSNamesystemLock#writeUnlock(String, boolean, Supplier)}
|
|
*/
|
|
*/
|
|
public void writeUnlock() {
|
|
public void writeUnlock() {
|
|
- writeUnlock(OP_NAME_OTHER, false, null);
|
|
|
|
|
|
+ writeUnlock(OP_NAME_OTHER, false, null, true);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -262,7 +310,7 @@ class FSNamesystemLock {
|
|
* @param opName Operation name.
|
|
* @param opName Operation name.
|
|
*/
|
|
*/
|
|
public void writeUnlock(String opName) {
|
|
public void writeUnlock(String opName) {
|
|
- writeUnlock(opName, false, null);
|
|
|
|
|
|
+ writeUnlock(opName, false, null, true);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -274,7 +322,7 @@ class FSNamesystemLock {
|
|
*/
|
|
*/
|
|
public void writeUnlock(String opName,
|
|
public void writeUnlock(String opName,
|
|
Supplier<String> lockReportInfoSupplier) {
|
|
Supplier<String> lockReportInfoSupplier) {
|
|
- writeUnlock(opName, false, lockReportInfoSupplier);
|
|
|
|
|
|
+ writeUnlock(opName, false, lockReportInfoSupplier, true);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -286,7 +334,7 @@ class FSNamesystemLock {
|
|
* for long time will be logged in logs and metrics.
|
|
* for long time will be logged in logs and metrics.
|
|
*/
|
|
*/
|
|
public void writeUnlock(String opName, boolean suppressWriteLockReport) {
|
|
public void writeUnlock(String opName, boolean suppressWriteLockReport) {
|
|
- writeUnlock(opName, suppressWriteLockReport, null);
|
|
|
|
|
|
+ writeUnlock(opName, suppressWriteLockReport, null, true);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -297,8 +345,9 @@ class FSNamesystemLock {
|
|
* for long time will be logged in logs and metrics.
|
|
* for long time will be logged in logs and metrics.
|
|
* @param lockReportInfoSupplier The info shown in the lock report
|
|
* @param lockReportInfoSupplier The info shown in the lock report
|
|
*/
|
|
*/
|
|
- private void writeUnlock(String opName, boolean suppressWriteLockReport,
|
|
|
|
- Supplier<String> lockReportInfoSupplier) {
|
|
|
|
|
|
+ public void writeUnlock(String opName, boolean suppressWriteLockReport,
|
|
|
|
+ Supplier<String> lockReportInfoSupplier,
|
|
|
|
+ boolean unlockChildren) {
|
|
final boolean needReport = !suppressWriteLockReport && coarseLock
|
|
final boolean needReport = !suppressWriteLockReport && coarseLock
|
|
.getWriteHoldCount() == 1 && coarseLock.isWriteLockedByCurrentThread();
|
|
.getWriteHoldCount() == 1 && coarseLock.isWriteLockedByCurrentThread();
|
|
final long writeLockIntervalNanos =
|
|
final long writeLockIntervalNanos =
|
|
@@ -329,7 +378,18 @@ class FSNamesystemLock {
|
|
longestWriteLockHeldInfo = new LockHeldInfo();
|
|
longestWriteLockHeldInfo = new LockHeldInfo();
|
|
}
|
|
}
|
|
|
|
|
|
- coarseLock.writeLock().unlock();
|
|
|
|
|
|
+ if(this.isWriteLockedByCurrentThread()) { // Current thread holds the lock
|
|
|
|
+ // Unlock the top FSNamesystemLock
|
|
|
|
+ coarseLock.writeLock().unlock();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if(unlockChildren) { // Unlock and remove children locks
|
|
|
|
+ Iterator<INodeMapLock> iter = partitionLocks.get().iterator();
|
|
|
|
+ while(iter.hasNext()) {
|
|
|
|
+ iter.next().writeChildUnlock();
|
|
|
|
+ iter.remove();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
if (needReport) {
|
|
if (needReport) {
|
|
addMetric(opName, writeLockIntervalNanos, true);
|
|
addMetric(opName, writeLockIntervalNanos, true);
|
|
@@ -355,7 +415,25 @@ class FSNamesystemLock {
|
|
public int getWriteHoldCount() {
|
|
public int getWriteHoldCount() {
|
|
return coarseLock.getWriteHoldCount();
|
|
return coarseLock.getWriteHoldCount();
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Queries if the write lock is held by any thread.
|
|
|
|
+ * @return {@code true} if any thread holds the write lock and
|
|
|
|
+ * {@code false} otherwise
|
|
|
|
+ */
|
|
|
|
+ public boolean isReadLocked() {
|
|
|
|
+ return coarseLock.getReadLockCount() > 0 || isWriteLocked();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Queries if the write lock is held by any thread.
|
|
|
|
+ * @return {@code true} if any thread holds the write lock and
|
|
|
|
+ * {@code false} otherwise
|
|
|
|
+ */
|
|
|
|
+ public boolean isWriteLocked() {
|
|
|
|
+ return coarseLock.isWriteLocked();
|
|
|
|
+ }
|
|
|
|
+
|
|
public boolean isWriteLockedByCurrentThread() {
|
|
public boolean isWriteLockedByCurrentThread() {
|
|
return coarseLock.isWriteLockedByCurrentThread();
|
|
return coarseLock.isWriteLockedByCurrentThread();
|
|
}
|
|
}
|