|
@@ -27,7 +27,6 @@ import com.google.common.util.concurrent.ListenableFuture;
|
|
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
-import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
@@ -161,37 +160,54 @@ public class DatasetVolumeChecker {
|
|
|
* @param dataset - FsDatasetSpi to be checked.
|
|
|
* @return set of failed volumes.
|
|
|
*/
|
|
|
- public Set<StorageLocation> checkAllVolumes(
|
|
|
+ public Set<FsVolumeSpi> checkAllVolumes(
|
|
|
final FsDatasetSpi<? extends FsVolumeSpi> dataset)
|
|
|
throws InterruptedException {
|
|
|
-
|
|
|
- if (timer.monotonicNow() - lastAllVolumesCheck < minDiskCheckGapMs) {
|
|
|
+ final long gap = timer.monotonicNow() - lastAllVolumesCheck;
|
|
|
+ if (gap < minDiskCheckGapMs) {
|
|
|
numSkippedChecks.incrementAndGet();
|
|
|
+ LOG.trace(
|
|
|
+ "Skipped checking all volumes, time since last check {} is less " +
|
|
|
+ "than the minimum gap between checks ({} ms).",
|
|
|
+ gap, minDiskCheckGapMs);
|
|
|
return Collections.emptySet();
|
|
|
}
|
|
|
|
|
|
- lastAllVolumesCheck = timer.monotonicNow();
|
|
|
- final Set<StorageLocation> healthyVolumes = new HashSet<>();
|
|
|
- final Set<StorageLocation> failedVolumes = new HashSet<>();
|
|
|
- final Set<StorageLocation> allVolumes = new HashSet<>();
|
|
|
-
|
|
|
final FsDatasetSpi.FsVolumeReferences references =
|
|
|
dataset.getFsVolumeReferences();
|
|
|
- final CountDownLatch resultsLatch = new CountDownLatch(references.size());
|
|
|
+
|
|
|
+ if (references.size() == 0) {
|
|
|
+ LOG.warn("checkAllVolumesAsync - no volumes can be referenced");
|
|
|
+ return Collections.emptySet();
|
|
|
+ }
|
|
|
+
|
|
|
+ lastAllVolumesCheck = timer.monotonicNow();
|
|
|
+ final Set<FsVolumeSpi> healthyVolumes = new HashSet<>();
|
|
|
+ final Set<FsVolumeSpi> failedVolumes = new HashSet<>();
|
|
|
+ final Set<FsVolumeSpi> allVolumes = new HashSet<>();
|
|
|
+
|
|
|
+ final AtomicLong numVolumes = new AtomicLong(references.size());
|
|
|
+ final CountDownLatch latch = new CountDownLatch(1);
|
|
|
|
|
|
for (int i = 0; i < references.size(); ++i) {
|
|
|
final FsVolumeReference reference = references.getReference(i);
|
|
|
- allVolumes.add(reference.getVolume().getStorageLocation());
|
|
|
+ allVolumes.add(reference.getVolume());
|
|
|
ListenableFuture<VolumeCheckResult> future =
|
|
|
delegateChecker.schedule(reference.getVolume(), IGNORED_CONTEXT);
|
|
|
LOG.info("Scheduled health check for volume {}", reference.getVolume());
|
|
|
Futures.addCallback(future, new ResultHandler(
|
|
|
- reference, healthyVolumes, failedVolumes, resultsLatch, null));
|
|
|
+ reference, healthyVolumes, failedVolumes, numVolumes, new Callback() {
|
|
|
+ @Override
|
|
|
+ public void call(Set<FsVolumeSpi> ignored1,
|
|
|
+ Set<FsVolumeSpi> ignored2) {
|
|
|
+ latch.countDown();
|
|
|
+ }
|
|
|
+ }));
|
|
|
}
|
|
|
|
|
|
// Wait until our timeout elapses, after which we give up on
|
|
|
// the remaining volumes.
|
|
|
- if (!resultsLatch.await(maxAllowedTimeForCheckMs, TimeUnit.MILLISECONDS)) {
|
|
|
+ if (!latch.await(maxAllowedTimeForCheckMs, TimeUnit.MILLISECONDS)) {
|
|
|
LOG.warn("checkAllVolumes timed out after {} ms" +
|
|
|
maxAllowedTimeForCheckMs);
|
|
|
}
|
|
@@ -225,18 +241,28 @@ public class DatasetVolumeChecker {
|
|
|
public boolean checkAllVolumesAsync(
|
|
|
final FsDatasetSpi<? extends FsVolumeSpi> dataset,
|
|
|
Callback callback) {
|
|
|
-
|
|
|
- if (timer.monotonicNow() - lastAllVolumesCheck < minDiskCheckGapMs) {
|
|
|
+ final long gap = timer.monotonicNow() - lastAllVolumesCheck;
|
|
|
+ if (gap < minDiskCheckGapMs) {
|
|
|
numSkippedChecks.incrementAndGet();
|
|
|
+ LOG.trace(
|
|
|
+ "Skipped checking all volumes, time since last check {} is less " +
|
|
|
+ "than the minimum gap between checks ({} ms).",
|
|
|
+ gap, minDiskCheckGapMs);
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- lastAllVolumesCheck = timer.monotonicNow();
|
|
|
- final Set<StorageLocation> healthyVolumes = new HashSet<>();
|
|
|
- final Set<StorageLocation> failedVolumes = new HashSet<>();
|
|
|
final FsDatasetSpi.FsVolumeReferences references =
|
|
|
dataset.getFsVolumeReferences();
|
|
|
- final CountDownLatch latch = new CountDownLatch(references.size());
|
|
|
+
|
|
|
+ if (references.size() == 0) {
|
|
|
+ LOG.warn("checkAllVolumesAsync - no volumes can be referenced");
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ lastAllVolumesCheck = timer.monotonicNow();
|
|
|
+ final Set<FsVolumeSpi> healthyVolumes = new HashSet<>();
|
|
|
+ final Set<FsVolumeSpi> failedVolumes = new HashSet<>();
|
|
|
+ final AtomicLong numVolumes = new AtomicLong(references.size());
|
|
|
|
|
|
LOG.info("Checking {} volumes", references.size());
|
|
|
for (int i = 0; i < references.size(); ++i) {
|
|
@@ -245,7 +271,7 @@ public class DatasetVolumeChecker {
|
|
|
ListenableFuture<VolumeCheckResult> future =
|
|
|
delegateChecker.schedule(reference.getVolume(), IGNORED_CONTEXT);
|
|
|
Futures.addCallback(future, new ResultHandler(
|
|
|
- reference, healthyVolumes, failedVolumes, latch, callback));
|
|
|
+ reference, healthyVolumes, failedVolumes, numVolumes, callback));
|
|
|
}
|
|
|
numAsyncDatasetChecks.incrementAndGet();
|
|
|
return true;
|
|
@@ -260,8 +286,8 @@ public class DatasetVolumeChecker {
|
|
|
* @param healthyVolumes set of volumes that passed disk checks.
|
|
|
* @param failedVolumes set of volumes that failed disk checks.
|
|
|
*/
|
|
|
- void call(Set<StorageLocation> healthyVolumes,
|
|
|
- Set<StorageLocation> failedVolumes);
|
|
|
+ void call(Set<FsVolumeSpi> healthyVolumes,
|
|
|
+ Set<FsVolumeSpi> failedVolumes);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -273,8 +299,10 @@ public class DatasetVolumeChecker {
|
|
|
*
|
|
|
* @param volume the volume that is to be checked.
|
|
|
* @param callback callback to be invoked when the volume check completes.
|
|
|
+ * @return true if the check was scheduled and the callback will be invoked.
|
|
|
+ * false otherwise.
|
|
|
*/
|
|
|
- public void checkVolume(
|
|
|
+ public boolean checkVolume(
|
|
|
final FsVolumeSpi volume,
|
|
|
Callback callback) {
|
|
|
FsVolumeReference volumeReference;
|
|
@@ -283,14 +311,15 @@ public class DatasetVolumeChecker {
|
|
|
} catch (ClosedChannelException e) {
|
|
|
// The volume has already been closed.
|
|
|
callback.call(new HashSet<>(), new HashSet<>());
|
|
|
- return;
|
|
|
+ return false;
|
|
|
}
|
|
|
ListenableFuture<VolumeCheckResult> future =
|
|
|
delegateChecker.schedule(volume, IGNORED_CONTEXT);
|
|
|
numVolumeChecks.incrementAndGet();
|
|
|
Futures.addCallback(future, new ResultHandler(
|
|
|
volumeReference, new HashSet<>(), new HashSet<>(),
|
|
|
- new CountDownLatch(1), callback));
|
|
|
+ new AtomicLong(1), callback));
|
|
|
+ return true;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -299,26 +328,35 @@ public class DatasetVolumeChecker {
|
|
|
private class ResultHandler
|
|
|
implements FutureCallback<VolumeCheckResult> {
|
|
|
private final FsVolumeReference reference;
|
|
|
- private final Set<StorageLocation> failedVolumes;
|
|
|
- private final Set<StorageLocation> healthyVolumes;
|
|
|
- private final CountDownLatch latch;
|
|
|
- private final AtomicLong numVolumes;
|
|
|
+ private final Set<FsVolumeSpi> failedVolumes;
|
|
|
+ private final Set<FsVolumeSpi> healthyVolumes;
|
|
|
+ private final AtomicLong volumeCounter;
|
|
|
|
|
|
@Nullable
|
|
|
private final Callback callback;
|
|
|
|
|
|
+ /**
|
|
|
+ *
|
|
|
+ * @param reference FsVolumeReference to be released when the check is
|
|
|
+ * complete.
|
|
|
+ * @param healthyVolumes set of healthy volumes. If the disk check is
|
|
|
+ * successful, add the volume here.
|
|
|
+ * @param failedVolumes set of failed volumes. If the disk check fails,
|
|
|
+ * add the volume here.
|
|
|
+ * @param semaphore semaphore used to trigger callback invocation.
|
|
|
+ * @param callback invoked when the semaphore can be successfully acquired.
|
|
|
+ */
|
|
|
ResultHandler(FsVolumeReference reference,
|
|
|
- Set<StorageLocation> healthyVolumes,
|
|
|
- Set<StorageLocation> failedVolumes,
|
|
|
- CountDownLatch latch,
|
|
|
+ Set<FsVolumeSpi> healthyVolumes,
|
|
|
+ Set<FsVolumeSpi> failedVolumes,
|
|
|
+ AtomicLong volumeCounter,
|
|
|
@Nullable Callback callback) {
|
|
|
Preconditions.checkState(reference != null);
|
|
|
this.reference = reference;
|
|
|
this.healthyVolumes = healthyVolumes;
|
|
|
this.failedVolumes = failedVolumes;
|
|
|
- this.latch = latch;
|
|
|
+ this.volumeCounter = volumeCounter;
|
|
|
this.callback = callback;
|
|
|
- numVolumes = new AtomicLong(latch.getCount());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -355,13 +393,13 @@ public class DatasetVolumeChecker {
|
|
|
|
|
|
private void markHealthy() {
|
|
|
synchronized (DatasetVolumeChecker.this) {
|
|
|
- healthyVolumes.add(reference.getVolume().getStorageLocation());
|
|
|
+ healthyVolumes.add(reference.getVolume());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private void markFailed() {
|
|
|
synchronized (DatasetVolumeChecker.this) {
|
|
|
- failedVolumes.add(reference.getVolume().getStorageLocation());
|
|
|
+ failedVolumes.add(reference.getVolume());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -372,10 +410,8 @@ public class DatasetVolumeChecker {
|
|
|
|
|
|
private void invokeCallback() {
|
|
|
try {
|
|
|
- latch.countDown();
|
|
|
-
|
|
|
- if (numVolumes.decrementAndGet() == 0 &&
|
|
|
- callback != null) {
|
|
|
+ final long remaining = volumeCounter.decrementAndGet();
|
|
|
+ if (callback != null && remaining == 0) {
|
|
|
callback.call(healthyVolumes, failedVolumes);
|
|
|
}
|
|
|
} catch(Exception e) {
|