|
@@ -31,6 +31,8 @@ import java.util.Map;
|
|
|
import java.util.TreeMap;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.CopyOnWriteArrayList;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.locks.Condition;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
@@ -41,6 +43,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
+import org.apache.hadoop.util.AutoCloseableLock;
|
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
|
|
@@ -52,7 +55,8 @@ class FsVolumeList {
|
|
|
Collections.synchronizedMap(new TreeMap<String, VolumeFailureInfo>());
|
|
|
private final ConcurrentLinkedQueue<FsVolumeImpl> volumesBeingRemoved =
|
|
|
new ConcurrentLinkedQueue<>();
|
|
|
- private Object checkDirsMutex = new Object();
|
|
|
+ private final AutoCloseableLock checkDirsLock;
|
|
|
+ private final Condition checkDirsLockCondition;
|
|
|
|
|
|
private final VolumeChoosingPolicy<FsVolumeImpl> blockChooser;
|
|
|
private final BlockScanner blockScanner;
|
|
@@ -62,6 +66,8 @@ class FsVolumeList {
|
|
|
VolumeChoosingPolicy<FsVolumeImpl> blockChooser) {
|
|
|
this.blockChooser = blockChooser;
|
|
|
this.blockScanner = blockScanner;
|
|
|
+ this.checkDirsLock = new AutoCloseableLock();
|
|
|
+ this.checkDirsLockCondition = checkDirsLock.newCondition();
|
|
|
for (VolumeFailureInfo volumeFailureInfo: initialVolumeFailureInfos) {
|
|
|
volumeFailureInfos.put(volumeFailureInfo.getFailedStorageLocation(),
|
|
|
volumeFailureInfo);
|
|
@@ -224,12 +230,12 @@ class FsVolumeList {
|
|
|
/**
|
|
|
* Calls {@link FsVolumeImpl#checkDirs()} on each volume.
|
|
|
*
|
|
|
- * Use checkDirsMutext to allow only one instance of checkDirs() call
|
|
|
+ * Use {@link checkDirsLock} to allow only one instance of checkDirs() call.
|
|
|
*
|
|
|
* @return list of all the failed volumes.
|
|
|
*/
|
|
|
Set<File> checkDirs() {
|
|
|
- synchronized(checkDirsMutex) {
|
|
|
+ try (AutoCloseableLock lock = checkDirsLock.acquire()) {
|
|
|
Set<File> failedVols = null;
|
|
|
|
|
|
// Make a copy of volumes for performing modification
|
|
@@ -260,7 +266,7 @@ class FsVolumeList {
|
|
|
+ " failure volumes.");
|
|
|
}
|
|
|
|
|
|
- waitVolumeRemoved(5000, checkDirsMutex);
|
|
|
+ waitVolumeRemoved(5000, checkDirsLockCondition);
|
|
|
return failedVols;
|
|
|
}
|
|
|
}
|
|
@@ -271,13 +277,13 @@ class FsVolumeList {
|
|
|
*
|
|
|
* @param sleepMillis interval to recheck.
|
|
|
*/
|
|
|
- void waitVolumeRemoved(int sleepMillis, Object monitor) {
|
|
|
+ void waitVolumeRemoved(int sleepMillis, Condition condition) {
|
|
|
while (!checkVolumesRemoved()) {
|
|
|
if (FsDatasetImpl.LOG.isDebugEnabled()) {
|
|
|
FsDatasetImpl.LOG.debug("Waiting for volume reference to be released.");
|
|
|
}
|
|
|
try {
|
|
|
- monitor.wait(sleepMillis);
|
|
|
+ condition.await(sleepMillis, TimeUnit.MILLISECONDS);
|
|
|
} catch (InterruptedException e) {
|
|
|
FsDatasetImpl.LOG.info("Thread interrupted when waiting for "
|
|
|
+ "volume reference to be released.");
|