|
@@ -22,7 +22,6 @@ import java.io.File;
|
|
|
import java.io.FilenameFilter;
|
|
|
import java.io.IOException;
|
|
|
import java.util.Arrays;
|
|
|
-import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
@@ -37,9 +36,6 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|
|
import java.util.concurrent.ThreadLocalRandom;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
-import java.util.regex.Matcher;
|
|
|
-import java.util.regex.Pattern;
|
|
|
-
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
@@ -47,10 +43,9 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.util.AutoCloseableLock;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
|
-import org.apache.hadoop.io.IOUtils;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
|
|
|
import org.apache.hadoop.util.Daemon;
|
|
|
import org.apache.hadoop.util.StopWatch;
|
|
|
import org.apache.hadoop.util.Time;
|
|
@@ -209,200 +204,6 @@ public class DirectoryScanner implements Runnable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Tracks the files and other information related to a block on the disk
|
|
|
- * Missing file is indicated by setting the corresponding member
|
|
|
- * to null.
|
|
|
- *
|
|
|
- * Because millions of these structures may be created, we try to save
|
|
|
- * memory here. So instead of storing full paths, we store path suffixes.
|
|
|
- * The block file, if it exists, will have a path like this:
|
|
|
- * <volume_base_path>/<block_path>
|
|
|
- * So we don't need to store the volume path, since we already know what the
|
|
|
- * volume is.
|
|
|
- *
|
|
|
- * The metadata file, if it exists, will have a path like this:
|
|
|
- * <volume_base_path>/<block_path>_<genstamp>.meta
|
|
|
- * So if we have a block file, there isn't any need to store the block path
|
|
|
- * again.
|
|
|
- *
|
|
|
- * The accessor functions take care of these manipulations.
|
|
|
- */
|
|
|
- static class ScanInfo implements Comparable<ScanInfo> {
|
|
|
- private final long blockId;
|
|
|
-
|
|
|
- /**
|
|
|
- * The block file path, relative to the volume's base directory.
|
|
|
- * If there was no block file found, this may be null. If 'vol'
|
|
|
- * is null, then this is the full path of the block file.
|
|
|
- */
|
|
|
- private final String blockSuffix;
|
|
|
-
|
|
|
- /**
|
|
|
- * The suffix of the meta file path relative to the block file.
|
|
|
- * If blockSuffix is null, then this will be the entire path relative
|
|
|
- * to the volume base directory, or an absolute path if vol is also
|
|
|
- * null.
|
|
|
- */
|
|
|
- private final String metaSuffix;
|
|
|
-
|
|
|
- private final FsVolumeSpi volume;
|
|
|
-
|
|
|
- /**
|
|
|
- * Get the file's length in async block scan
|
|
|
- */
|
|
|
- private final long blockFileLength;
|
|
|
-
|
|
|
- private final static Pattern CONDENSED_PATH_REGEX =
|
|
|
- Pattern.compile("(?<!^)(\\\\|/){2,}");
|
|
|
-
|
|
|
- private final static String QUOTED_FILE_SEPARATOR =
|
|
|
- Matcher.quoteReplacement(File.separator);
|
|
|
-
|
|
|
- /**
|
|
|
- * Get the most condensed version of the path.
|
|
|
- *
|
|
|
- * For example, the condensed version of /foo//bar is /foo/bar
|
|
|
- * Unlike {@link File#getCanonicalPath()}, this will never perform I/O
|
|
|
- * on the filesystem.
|
|
|
- *
|
|
|
- * @param path the path to condense
|
|
|
- * @return the condensed path
|
|
|
- */
|
|
|
- private static String getCondensedPath(String path) {
|
|
|
- return CONDENSED_PATH_REGEX.matcher(path).
|
|
|
- replaceAll(QUOTED_FILE_SEPARATOR);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Get a path suffix.
|
|
|
- *
|
|
|
- * @param f The file to get the suffix for.
|
|
|
- * @param prefix The prefix we're stripping off.
|
|
|
- *
|
|
|
- * @return A suffix such that prefix + suffix = path to f
|
|
|
- */
|
|
|
- private static String getSuffix(File f, String prefix) {
|
|
|
- String fullPath = getCondensedPath(f.getAbsolutePath());
|
|
|
- if (fullPath.startsWith(prefix)) {
|
|
|
- return fullPath.substring(prefix.length());
|
|
|
- }
|
|
|
- throw new RuntimeException(prefix + " is not a prefix of " + fullPath);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Create a ScanInfo object for a block. This constructor will examine
|
|
|
- * the block data and meta-data files.
|
|
|
- *
|
|
|
- * @param blockId the block ID
|
|
|
- * @param blockFile the path to the block data file
|
|
|
- * @param metaFile the path to the block meta-data file
|
|
|
- * @param vol the volume that contains the block
|
|
|
- */
|
|
|
- ScanInfo(long blockId, File blockFile, File metaFile, FsVolumeSpi vol) {
|
|
|
- this.blockId = blockId;
|
|
|
- String condensedVolPath = vol == null ? null :
|
|
|
- getCondensedPath(vol.getBasePath());
|
|
|
- this.blockSuffix = blockFile == null ? null :
|
|
|
- getSuffix(blockFile, condensedVolPath);
|
|
|
- this.blockFileLength = (blockFile != null) ? blockFile.length() : 0;
|
|
|
- if (metaFile == null) {
|
|
|
- this.metaSuffix = null;
|
|
|
- } else if (blockFile == null) {
|
|
|
- this.metaSuffix = getSuffix(metaFile, condensedVolPath);
|
|
|
- } else {
|
|
|
- this.metaSuffix = getSuffix(metaFile,
|
|
|
- condensedVolPath + blockSuffix);
|
|
|
- }
|
|
|
- this.volume = vol;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Returns the block data file.
|
|
|
- *
|
|
|
- * @return the block data file
|
|
|
- */
|
|
|
- File getBlockFile() {
|
|
|
- return (blockSuffix == null) ? null :
|
|
|
- new File(volume.getBasePath(), blockSuffix);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Return the length of the data block. The length returned is the length
|
|
|
- * cached when this object was created.
|
|
|
- *
|
|
|
- * @return the length of the data block
|
|
|
- */
|
|
|
- long getBlockFileLength() {
|
|
|
- return blockFileLength;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Returns the block meta data file or null if there isn't one.
|
|
|
- *
|
|
|
- * @return the block meta data file
|
|
|
- */
|
|
|
- File getMetaFile() {
|
|
|
- if (metaSuffix == null) {
|
|
|
- return null;
|
|
|
- } else if (blockSuffix == null) {
|
|
|
- return new File(volume.getBasePath(), metaSuffix);
|
|
|
- } else {
|
|
|
- return new File(volume.getBasePath(), blockSuffix + metaSuffix);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Returns the block ID.
|
|
|
- *
|
|
|
- * @return the block ID
|
|
|
- */
|
|
|
- long getBlockId() {
|
|
|
- return blockId;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Returns the volume that contains the block that this object describes.
|
|
|
- *
|
|
|
- * @return the volume
|
|
|
- */
|
|
|
- FsVolumeSpi getVolume() {
|
|
|
- return volume;
|
|
|
- }
|
|
|
-
|
|
|
- @Override // Comparable
|
|
|
- public int compareTo(ScanInfo b) {
|
|
|
- if (blockId < b.blockId) {
|
|
|
- return -1;
|
|
|
- } else if (blockId == b.blockId) {
|
|
|
- return 0;
|
|
|
- } else {
|
|
|
- return 1;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- @Override // Object
|
|
|
- public boolean equals(Object o) {
|
|
|
- if (this == o) {
|
|
|
- return true;
|
|
|
- }
|
|
|
- if (!(o instanceof ScanInfo)) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- return blockId == ((ScanInfo) o).blockId;
|
|
|
- }
|
|
|
-
|
|
|
- @Override // Object
|
|
|
- public int hashCode() {
|
|
|
- return (int)(blockId^(blockId>>>32));
|
|
|
- }
|
|
|
-
|
|
|
- public long getGenStamp() {
|
|
|
- return metaSuffix != null ? Block.getGenerationStamp(
|
|
|
- getMetaFile().getName()) :
|
|
|
- HdfsConstants.GRANDFATHER_GENERATION_STAMP;
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* Create a new directory scanner, but don't cycle it running yet.
|
|
@@ -644,7 +445,7 @@ public class DirectoryScanner implements Runnable {
|
|
|
// There may be multiple on-disk records for the same block, don't increment
|
|
|
// the memory record pointer if so.
|
|
|
ScanInfo nextInfo = blockpoolReport[Math.min(d, blockpoolReport.length - 1)];
|
|
|
- if (nextInfo.getBlockId() != info.blockId) {
|
|
|
+ if (nextInfo.getBlockId() != info.getBlockId()) {
|
|
|
++m;
|
|
|
}
|
|
|
} else {
|
|
@@ -762,19 +563,6 @@ public class DirectoryScanner implements Runnable {
|
|
|
return list.toSortedArrays();
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Helper method to determine if a file name is consistent with a block.
|
|
|
- * meta-data file
|
|
|
- *
|
|
|
- * @param blockId the block ID
|
|
|
- * @param metaFile the file to check
|
|
|
- * @return whether the file name is a block meta-data file name
|
|
|
- */
|
|
|
- private static boolean isBlockMetaFile(String blockId, String metaFile) {
|
|
|
- return metaFile.startsWith(blockId)
|
|
|
- && metaFile.endsWith(Block.METADATA_EXTENSION);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* The ReportCompiler class encapsulates the process of searching a datanode's
|
|
|
* disks for block information. It operates by performing a DFS of the
|
|
@@ -784,7 +572,7 @@ public class DirectoryScanner implements Runnable {
|
|
|
* ScanInfo object for it and adds that object to its report list. The report
|
|
|
* list is returned by the {@link #call()} method.
|
|
|
*/
|
|
|
- private class ReportCompiler implements Callable<ScanInfoPerBlockPool> {
|
|
|
+ public class ReportCompiler implements Callable<ScanInfoPerBlockPool> {
|
|
|
private final FsVolumeSpi volume;
|
|
|
private final DataNode datanode;
|
|
|
// Variable for tracking time spent running for throttling purposes
|
|
@@ -816,14 +604,12 @@ public class DirectoryScanner implements Runnable {
|
|
|
ScanInfoPerBlockPool result = new ScanInfoPerBlockPool(bpList.length);
|
|
|
for (String bpid : bpList) {
|
|
|
LinkedList<ScanInfo> report = new LinkedList<>();
|
|
|
- File bpFinalizedDir = volume.getFinalizedDir(bpid);
|
|
|
|
|
|
perfTimer.start();
|
|
|
throttleTimer.start();
|
|
|
|
|
|
try {
|
|
|
- result.put(bpid,
|
|
|
- compileReport(volume, bpFinalizedDir, bpFinalizedDir, report));
|
|
|
+ result.put(bpid, volume.compileReport(bpid, report, this));
|
|
|
} catch (InterruptedException ex) {
|
|
|
// Exit quickly and flag the scanner to do the same
|
|
|
result = null;
|
|
@@ -833,107 +619,13 @@ public class DirectoryScanner implements Runnable {
|
|
|
return result;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Compile a list of {@link ScanInfo} for the blocks in the directory
|
|
|
- * given by {@code dir}.
|
|
|
- *
|
|
|
- * @param vol the volume that contains the directory to scan
|
|
|
- * @param bpFinalizedDir the root directory of the directory to scan
|
|
|
- * @param dir the directory to scan
|
|
|
- * @param report the list onto which blocks reports are placed
|
|
|
- */
|
|
|
- private LinkedList<ScanInfo> compileReport(FsVolumeSpi vol,
|
|
|
- File bpFinalizedDir, File dir, LinkedList<ScanInfo> report)
|
|
|
- throws InterruptedException {
|
|
|
-
|
|
|
- throttle();
|
|
|
-
|
|
|
- List <String> fileNames;
|
|
|
- try {
|
|
|
- fileNames = IOUtils.listDirectory(dir, BlockDirFilter.INSTANCE);
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.warn("Exception occured while compiling report: ", ioe);
|
|
|
- // Initiate a check on disk failure.
|
|
|
- datanode.checkDiskErrorAsync();
|
|
|
- // Ignore this directory and proceed.
|
|
|
- return report;
|
|
|
- }
|
|
|
- Collections.sort(fileNames);
|
|
|
-
|
|
|
- /*
|
|
|
- * Assumption: In the sorted list of files block file appears immediately
|
|
|
- * before block metadata file. This is true for the current naming
|
|
|
- * convention for block file blk_<blockid> and meta file
|
|
|
- * blk_<blockid>_<genstamp>.meta
|
|
|
- */
|
|
|
- for (int i = 0; i < fileNames.size(); i++) {
|
|
|
- // Make sure this thread can make a timely exit. With a low throttle
|
|
|
- // rate, completing a run can take a looooong time.
|
|
|
- if (Thread.interrupted()) {
|
|
|
- throw new InterruptedException();
|
|
|
- }
|
|
|
-
|
|
|
- File file = new File(dir, fileNames.get(i));
|
|
|
- if (file.isDirectory()) {
|
|
|
- compileReport(vol, bpFinalizedDir, file, report);
|
|
|
- continue;
|
|
|
- }
|
|
|
- if (!Block.isBlockFilename(file)) {
|
|
|
- if (isBlockMetaFile(Block.BLOCK_FILE_PREFIX, file.getName())) {
|
|
|
- long blockId = Block.getBlockId(file.getName());
|
|
|
- verifyFileLocation(file.getParentFile(), bpFinalizedDir,
|
|
|
- blockId);
|
|
|
- report.add(new ScanInfo(blockId, null, file, vol));
|
|
|
- }
|
|
|
- continue;
|
|
|
- }
|
|
|
- File blockFile = file;
|
|
|
- long blockId = Block.filename2id(file.getName());
|
|
|
- File metaFile = null;
|
|
|
-
|
|
|
- // Skip all the files that start with block name until
|
|
|
- // getting to the metafile for the block
|
|
|
- while (i + 1 < fileNames.size()) {
|
|
|
- File blkMetaFile = new File(dir, fileNames.get(i + 1));
|
|
|
- if (!(blkMetaFile.isFile()
|
|
|
- && blkMetaFile.getName().startsWith(blockFile.getName()))) {
|
|
|
- break;
|
|
|
- }
|
|
|
- i++;
|
|
|
- if (isBlockMetaFile(blockFile.getName(), blkMetaFile.getName())) {
|
|
|
- metaFile = blkMetaFile;
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- verifyFileLocation(blockFile, bpFinalizedDir, blockId);
|
|
|
- report.add(new ScanInfo(blockId, blockFile, metaFile, vol));
|
|
|
- }
|
|
|
- return report;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Verify whether the actual directory location of block file has the
|
|
|
- * expected directory path computed using its block ID.
|
|
|
- */
|
|
|
- private void verifyFileLocation(File actualBlockFile,
|
|
|
- File bpFinalizedDir, long blockId) {
|
|
|
- File expectedBlockDir =
|
|
|
- DatanodeUtil.idToBlockDir(bpFinalizedDir, blockId);
|
|
|
- File actualBlockDir = actualBlockFile.getParentFile();
|
|
|
- if (actualBlockDir.compareTo(expectedBlockDir) != 0) {
|
|
|
- LOG.warn("Block: " + blockId +
|
|
|
- " found in invalid directory. Expected directory: " +
|
|
|
- expectedBlockDir + ". Actual directory: " + actualBlockDir);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Called by the thread before each potential disk scan so that a pause
|
|
|
* can be optionally inserted to limit the number of scans per second.
|
|
|
* The limit is controlled by
|
|
|
* {@link DFSConfigKeys#DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY}.
|
|
|
*/
|
|
|
- private void throttle() throws InterruptedException {
|
|
|
+ public void throttle() throws InterruptedException {
|
|
|
accumulateTimeRunning();
|
|
|
|
|
|
if ((throttleLimitMsPerSec < 1000) &&
|
|
@@ -963,7 +655,7 @@ public class DirectoryScanner implements Runnable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private enum BlockDirFilter implements FilenameFilter {
|
|
|
+ public enum BlockDirFilter implements FilenameFilter {
|
|
|
INSTANCE;
|
|
|
|
|
|
@Override
|