|
@@ -22,12 +22,15 @@ import java.io.DataOutputStream;
|
|
|
import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
import java.util.Iterator;
|
|
|
+import java.util.LinkedHashSet;
|
|
|
import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
+import com.google.common.cache.Cache;
|
|
|
+import com.google.common.cache.CacheBuilder;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.BlockScanner.Conf;
|
|
@@ -116,6 +119,21 @@ public class VolumeScanner extends Thread {
|
|
|
private final List<BlockIterator> blockIters =
|
|
|
new LinkedList<BlockIterator>();
|
|
|
|
|
|
+ /**
|
|
|
+ * Blocks which are suspect.
|
|
|
+ * The scanner prioritizes scanning these blocks.
|
|
|
+ */
|
|
|
+ private final LinkedHashSet<ExtendedBlock> suspectBlocks =
|
|
|
+ new LinkedHashSet<ExtendedBlock>();
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Blocks which were suspect which we have scanned.
|
|
|
+ * This is used to avoid scanning the same suspect block over and over.
|
|
|
+ */
|
|
|
+ private final Cache<ExtendedBlock, Boolean> recentSuspectBlocks =
|
|
|
+ CacheBuilder.newBuilder().maximumSize(1000)
|
|
|
+ .expireAfterAccess(10, TimeUnit.MINUTES).build();
|
|
|
+
|
|
|
/**
|
|
|
* The current block iterator, or null if there is none.
|
|
|
*/
|
|
@@ -458,10 +476,13 @@ public class VolumeScanner extends Thread {
|
|
|
/**
|
|
|
* Run an iteration of the VolumeScanner loop.
|
|
|
*
|
|
|
+ * @param suspectBlock A suspect block which we should scan, or null to
|
|
|
+ * scan the next regularly scheduled block.
|
|
|
+ *
|
|
|
* @return The number of milliseconds to delay before running the loop
|
|
|
* again, or 0 to re-run the loop immediately.
|
|
|
*/
|
|
|
- private long runLoop() {
|
|
|
+ private long runLoop(ExtendedBlock suspectBlock) {
|
|
|
long bytesScanned = -1;
|
|
|
boolean scanError = false;
|
|
|
ExtendedBlock block = null;
|
|
@@ -477,40 +498,43 @@ public class VolumeScanner extends Thread {
|
|
|
}
|
|
|
|
|
|
// Find a usable block pool to scan.
|
|
|
- if ((curBlockIter == null) || curBlockIter.atEnd()) {
|
|
|
- long timeout = findNextUsableBlockIter();
|
|
|
- if (timeout > 0) {
|
|
|
- LOG.trace("{}: no block pools are ready to scan yet. Waiting " +
|
|
|
- "{} ms.", this, timeout);
|
|
|
+ if (suspectBlock != null) {
|
|
|
+ block = suspectBlock;
|
|
|
+ } else {
|
|
|
+ if ((curBlockIter == null) || curBlockIter.atEnd()) {
|
|
|
+ long timeout = findNextUsableBlockIter();
|
|
|
+ if (timeout > 0) {
|
|
|
+ LOG.trace("{}: no block pools are ready to scan yet. Waiting " +
|
|
|
+ "{} ms.", this, timeout);
|
|
|
+ synchronized (stats) {
|
|
|
+ stats.nextBlockPoolScanStartMs = Time.monotonicNow() + timeout;
|
|
|
+ }
|
|
|
+ return timeout;
|
|
|
+ }
|
|
|
synchronized (stats) {
|
|
|
- stats.nextBlockPoolScanStartMs = Time.monotonicNow() + timeout;
|
|
|
+ stats.scansSinceRestart++;
|
|
|
+ stats.blocksScannedInCurrentPeriod = 0;
|
|
|
+ stats.nextBlockPoolScanStartMs = -1;
|
|
|
}
|
|
|
- return timeout;
|
|
|
+ return 0L;
|
|
|
}
|
|
|
- synchronized (stats) {
|
|
|
- stats.scansSinceRestart++;
|
|
|
- stats.blocksScannedInCurrentPeriod = 0;
|
|
|
- stats.nextBlockPoolScanStartMs = -1;
|
|
|
+ try {
|
|
|
+ block = curBlockIter.nextBlock();
|
|
|
+ } catch (IOException e) {
|
|
|
+ // There was an error listing the next block in the volume. This is a
|
|
|
+ // serious issue.
|
|
|
+ LOG.warn("{}: nextBlock error on {}", this, curBlockIter);
|
|
|
+ // On the next loop iteration, curBlockIter#eof will be set to true, and
|
|
|
+ // we will pick a different block iterator.
|
|
|
+ return 0L;
|
|
|
+ }
|
|
|
+ if (block == null) {
|
|
|
+ // The BlockIterator is at EOF.
|
|
|
+ LOG.info("{}: finished scanning block pool {}",
|
|
|
+ this, curBlockIter.getBlockPoolId());
|
|
|
+ saveBlockIterator(curBlockIter);
|
|
|
+ return 0;
|
|
|
}
|
|
|
- return 0L;
|
|
|
- }
|
|
|
-
|
|
|
- try {
|
|
|
- block = curBlockIter.nextBlock();
|
|
|
- } catch (IOException e) {
|
|
|
- // There was an error listing the next block in the volume. This is a
|
|
|
- // serious issue.
|
|
|
- LOG.warn("{}: nextBlock error on {}", this, curBlockIter);
|
|
|
- // On the next loop iteration, curBlockIter#eof will be set to true, and
|
|
|
- // we will pick a different block iterator.
|
|
|
- return 0L;
|
|
|
- }
|
|
|
- if (block == null) {
|
|
|
- // The BlockIterator is at EOF.
|
|
|
- LOG.info("{}: finished scanning block pool {}",
|
|
|
- this, curBlockIter.getBlockPoolId());
|
|
|
- saveBlockIterator(curBlockIter);
|
|
|
- return 0;
|
|
|
}
|
|
|
long saveDelta = monotonicMs - curBlockIter.getLastSavedMs();
|
|
|
if (saveDelta >= conf.cursorSaveMs) {
|
|
@@ -529,7 +553,7 @@ public class VolumeScanner extends Thread {
|
|
|
} finally {
|
|
|
synchronized (stats) {
|
|
|
stats.bytesScannedInPastHour = scannedBytesSum;
|
|
|
- if (bytesScanned >= 0) {
|
|
|
+ if (bytesScanned > 0) {
|
|
|
stats.blocksScannedInCurrentPeriod++;
|
|
|
stats.blocksScannedSinceRestart++;
|
|
|
}
|
|
@@ -551,6 +575,20 @@ public class VolumeScanner extends Thread {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * If there are elements in the suspectBlocks list, removes
|
|
|
+ * and returns the first one. Otherwise, returns null.
|
|
|
+ */
|
|
|
+ private synchronized ExtendedBlock popNextSuspectBlock() {
|
|
|
+ Iterator<ExtendedBlock> iter = suspectBlocks.iterator();
|
|
|
+ if (!iter.hasNext()) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ ExtendedBlock block = iter.next();
|
|
|
+ iter.remove();
|
|
|
+ return block;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public void run() {
|
|
|
// Record the minute on which the scanner started.
|
|
@@ -563,7 +601,9 @@ public class VolumeScanner extends Thread {
|
|
|
try {
|
|
|
long timeout = 0;
|
|
|
while (true) {
|
|
|
- // Take the lock to check if we should stop.
|
|
|
+ ExtendedBlock suspectBlock = null;
|
|
|
+ // Take the lock to check if we should stop, and access the
|
|
|
+ // suspect block list.
|
|
|
synchronized (this) {
|
|
|
if (stopping) {
|
|
|
break;
|
|
@@ -574,8 +614,9 @@ public class VolumeScanner extends Thread {
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
+ suspectBlock = popNextSuspectBlock();
|
|
|
}
|
|
|
- timeout = runLoop();
|
|
|
+ timeout = runLoop(suspectBlock);
|
|
|
}
|
|
|
} catch (InterruptedException e) {
|
|
|
// We are exiting because of an InterruptedException,
|
|
@@ -612,6 +653,30 @@ public class VolumeScanner extends Thread {
|
|
|
this.interrupt();
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+ public synchronized void markSuspectBlock(ExtendedBlock block) {
|
|
|
+ if (stopping) {
|
|
|
+ LOG.info("{}: Not scheduling suspect block {} for " +
|
|
|
+ "rescanning, because this volume scanner is stopping.", this, block);
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ Boolean recent = recentSuspectBlocks.getIfPresent(block);
|
|
|
+ if (recent != null) {
|
|
|
+ LOG.info("{}: Not scheduling suspect block {} for " +
|
|
|
+ "rescanning, because we rescanned it recently.", this, block);
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ if (suspectBlocks.contains(block)) {
|
|
|
+ LOG.info("{}: suspect block {} is already queued for " +
|
|
|
+ "rescanning.", this, block);
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ suspectBlocks.add(block);
|
|
|
+ recentSuspectBlocks.put(block, true);
|
|
|
+ LOG.info("{}: Scheduling suspect block {} for rescanning.", this, block);
|
|
|
+ notify(); // wake scanner thread.
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Allow the scanner to scan the given block pool.
|
|
|
*
|