|
@@ -33,6 +33,7 @@ import java.net.URI;
|
|
import java.nio.channels.ClosedChannelException;
|
|
import java.nio.channels.ClosedChannelException;
|
|
import java.nio.channels.FileChannel;
|
|
import java.nio.channels.FileChannel;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
|
|
+import java.util.Collection;
|
|
import java.util.Iterator;
|
|
import java.util.Iterator;
|
|
import java.util.LinkedList;
|
|
import java.util.LinkedList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
@@ -44,26 +45,23 @@ import java.util.concurrent.TimeoutException;
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
|
|
import org.apache.commons.io.FileUtils;
|
|
import org.apache.commons.io.FileUtils;
|
|
-import org.slf4j.Logger;
|
|
|
|
-import org.slf4j.LoggerFactory;
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.DF;
|
|
import org.apache.hadoop.fs.DF;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.StorageType;
|
|
import org.apache.hadoop.fs.StorageType;
|
|
-import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
|
import org.apache.hadoop.hdfs.DFSClient;
|
|
import org.apache.hadoop.hdfs.DFSClient;
|
|
|
|
+import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
-import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
|
|
|
|
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.DataNodeVolumeMetrics;
|
|
|
|
-import org.apache.hadoop.util.AutoCloseableLock;
|
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
|
|
import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.DataNodeVolumeMetrics;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
|
|
@@ -73,14 +71,17 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
|
+import org.apache.hadoop.util.AutoCloseableLock;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
import org.mockito.Mockito;
|
|
import org.mockito.Mockito;
|
|
|
|
+import org.slf4j.Logger;
|
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Tests {@link DirectoryScanner} handling of differences
|
|
|
|
- * between blocks on the disk and block in memory.
|
|
|
|
|
|
+ * Tests {@link DirectoryScanner} handling of differences between blocks on the
|
|
|
|
+ * disk and block in memory.
|
|
*/
|
|
*/
|
|
public class TestDirectoryScanner {
|
|
public class TestDirectoryScanner {
|
|
private static final Logger LOG =
|
|
private static final Logger LOG =
|
|
@@ -102,7 +103,7 @@ public class TestDirectoryScanner {
|
|
CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1);
|
|
CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1);
|
|
CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
|
|
CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
|
|
CONF.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
|
CONF.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
|
- getMemlockLimit(Long.MAX_VALUE));
|
|
|
|
|
|
+ getMemlockLimit(Long.MAX_VALUE));
|
|
}
|
|
}
|
|
|
|
|
|
@Before
|
|
@Before
|
|
@@ -110,21 +111,20 @@ public class TestDirectoryScanner {
|
|
LazyPersistTestCase.initCacheManipulator();
|
|
LazyPersistTestCase.initCacheManipulator();
|
|
}
|
|
}
|
|
|
|
|
|
- /** create a file with a length of <code>fileLen</code> */
|
|
|
|
- private List<LocatedBlock> createFile(String fileNamePrefix,
|
|
|
|
- long fileLen,
|
|
|
|
- boolean isLazyPersist) throws IOException {
|
|
|
|
|
|
+ /** create a file with a length of <code>fileLen</code>. */
|
|
|
|
+ private List<LocatedBlock> createFile(String fileNamePrefix, long fileLen,
|
|
|
|
+ boolean isLazyPersist) throws IOException {
|
|
FileSystem fs = cluster.getFileSystem();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
Path filePath = new Path("/" + fileNamePrefix + ".dat");
|
|
Path filePath = new Path("/" + fileNamePrefix + ".dat");
|
|
- DFSTestUtil.createFile(
|
|
|
|
- fs, filePath, isLazyPersist, 1024, fileLen,
|
|
|
|
|
|
+ DFSTestUtil.createFile(fs, filePath, isLazyPersist, 1024, fileLen,
|
|
BLOCK_LENGTH, (short) 1, r.nextLong(), false);
|
|
BLOCK_LENGTH, (short) 1, r.nextLong(), false);
|
|
- return client.getLocatedBlocks(filePath.toString(), 0, fileLen).getLocatedBlocks();
|
|
|
|
|
|
+ return client.getLocatedBlocks(filePath.toString(), 0, fileLen)
|
|
|
|
+ .getLocatedBlocks();
|
|
}
|
|
}
|
|
|
|
|
|
- /** Truncate a block file */
|
|
|
|
|
|
+ /** Truncate a block file. */
|
|
private long truncateBlockFile() throws IOException {
|
|
private long truncateBlockFile() throws IOException {
|
|
- try(AutoCloseableLock lock = fds.acquireDatasetLock()) {
|
|
|
|
|
|
+ try (AutoCloseableLock lock = fds.acquireDatasetLock()) {
|
|
for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
|
|
for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
|
|
File f = new File(b.getBlockURI());
|
|
File f = new File(b.getBlockURI());
|
|
File mf = new File(b.getMetadataURI());
|
|
File mf = new File(b.getMetadataURI());
|
|
@@ -149,7 +149,7 @@ public class TestDirectoryScanner {
|
|
|
|
|
|
/** Delete a block file */
|
|
/** Delete a block file */
|
|
private long deleteBlockFile() {
|
|
private long deleteBlockFile() {
|
|
- try(AutoCloseableLock lock = fds.acquireDatasetLock()) {
|
|
|
|
|
|
+ try (AutoCloseableLock lock = fds.acquireDatasetLock()) {
|
|
for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
|
|
for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
|
|
File f = new File(b.getBlockURI());
|
|
File f = new File(b.getBlockURI());
|
|
File mf = new File(b.getMetadataURI());
|
|
File mf = new File(b.getMetadataURI());
|
|
@@ -165,7 +165,7 @@ public class TestDirectoryScanner {
|
|
|
|
|
|
/** Delete block meta file */
|
|
/** Delete block meta file */
|
|
private long deleteMetaFile() {
|
|
private long deleteMetaFile() {
|
|
- try(AutoCloseableLock lock = fds.acquireDatasetLock()) {
|
|
|
|
|
|
+ try (AutoCloseableLock lock = fds.acquireDatasetLock()) {
|
|
for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
|
|
for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
|
|
// Delete a metadata file
|
|
// Delete a metadata file
|
|
if (b.metadataExists() && b.deleteMetadata()) {
|
|
if (b.metadataExists() && b.deleteMetadata()) {
|
|
@@ -179,11 +179,12 @@ public class TestDirectoryScanner {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Duplicate the given block on all volumes.
|
|
* Duplicate the given block on all volumes.
|
|
|
|
+ *
|
|
* @param blockId
|
|
* @param blockId
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
private void duplicateBlock(long blockId) throws IOException {
|
|
private void duplicateBlock(long blockId) throws IOException {
|
|
- try(AutoCloseableLock lock = fds.acquireDatasetLock()) {
|
|
|
|
|
|
+ try (AutoCloseableLock lock = fds.acquireDatasetLock()) {
|
|
ReplicaInfo b = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId);
|
|
ReplicaInfo b = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId);
|
|
try (FsDatasetSpi.FsVolumeReferences volumes =
|
|
try (FsDatasetSpi.FsVolumeReferences volumes =
|
|
fds.getFsVolumeReferences()) {
|
|
fds.getFsVolumeReferences()) {
|
|
@@ -199,16 +200,14 @@ public class TestDirectoryScanner {
|
|
URI destRoot = v.getStorageLocation().getUri();
|
|
URI destRoot = v.getStorageLocation().getUri();
|
|
|
|
|
|
String relativeBlockPath =
|
|
String relativeBlockPath =
|
|
- sourceRoot.relativize(sourceBlock.toURI())
|
|
|
|
- .getPath();
|
|
|
|
|
|
+ sourceRoot.relativize(sourceBlock.toURI()).getPath();
|
|
String relativeMetaPath =
|
|
String relativeMetaPath =
|
|
- sourceRoot.relativize(sourceMeta.toURI())
|
|
|
|
- .getPath();
|
|
|
|
|
|
+ sourceRoot.relativize(sourceMeta.toURI()).getPath();
|
|
|
|
|
|
- File destBlock = new File(new File(destRoot).toString(),
|
|
|
|
- relativeBlockPath);
|
|
|
|
- File destMeta = new File(new File(destRoot).toString(),
|
|
|
|
- relativeMetaPath);
|
|
|
|
|
|
+ File destBlock =
|
|
|
|
+ new File(new File(destRoot).toString(), relativeBlockPath);
|
|
|
|
+ File destMeta =
|
|
|
|
+ new File(new File(destRoot).toString(), relativeMetaPath);
|
|
|
|
|
|
destBlock.getParentFile().mkdirs();
|
|
destBlock.getParentFile().mkdirs();
|
|
FileUtils.copyFile(sourceBlock, destBlock);
|
|
FileUtils.copyFile(sourceBlock, destBlock);
|
|
@@ -223,7 +222,7 @@ public class TestDirectoryScanner {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /** Get a random blockId that is not used already */
|
|
|
|
|
|
+ /** Get a random blockId that is not used already. */
|
|
private long getFreeBlockId() {
|
|
private long getFreeBlockId() {
|
|
long id = rand.nextLong();
|
|
long id = rand.nextLong();
|
|
while (true) {
|
|
while (true) {
|
|
@@ -244,14 +243,15 @@ public class TestDirectoryScanner {
|
|
+ Block.METADATA_EXTENSION;
|
|
+ Block.METADATA_EXTENSION;
|
|
}
|
|
}
|
|
|
|
|
|
- /** Create a block file in a random volume*/
|
|
|
|
|
|
+ /** Create a block file in a random volume. */
|
|
private long createBlockFile() throws IOException {
|
|
private long createBlockFile() throws IOException {
|
|
long id = getFreeBlockId();
|
|
long id = getFreeBlockId();
|
|
- try (FsDatasetSpi.FsVolumeReferences volumes = fds.getFsVolumeReferences()) {
|
|
|
|
|
|
+ try (
|
|
|
|
+ FsDatasetSpi.FsVolumeReferences volumes = fds.getFsVolumeReferences()) {
|
|
int numVolumes = volumes.size();
|
|
int numVolumes = volumes.size();
|
|
int index = rand.nextInt(numVolumes - 1);
|
|
int index = rand.nextInt(numVolumes - 1);
|
|
- File finalizedDir = ((FsVolumeImpl) volumes.get(index))
|
|
|
|
- .getFinalizedDir(bpid);
|
|
|
|
|
|
+ File finalizedDir =
|
|
|
|
+ ((FsVolumeImpl) volumes.get(index)).getFinalizedDir(bpid);
|
|
File file = new File(finalizedDir, getBlockFile(id));
|
|
File file = new File(finalizedDir, getBlockFile(id));
|
|
if (file.createNewFile()) {
|
|
if (file.createNewFile()) {
|
|
LOG.info("Created block file " + file.getName());
|
|
LOG.info("Created block file " + file.getName());
|
|
@@ -260,14 +260,14 @@ public class TestDirectoryScanner {
|
|
return id;
|
|
return id;
|
|
}
|
|
}
|
|
|
|
|
|
- /** Create a metafile in a random volume*/
|
|
|
|
|
|
+ /** Create a metafile in a random volume */
|
|
private long createMetaFile() throws IOException {
|
|
private long createMetaFile() throws IOException {
|
|
long id = getFreeBlockId();
|
|
long id = getFreeBlockId();
|
|
try (FsDatasetSpi.FsVolumeReferences refs = fds.getFsVolumeReferences()) {
|
|
try (FsDatasetSpi.FsVolumeReferences refs = fds.getFsVolumeReferences()) {
|
|
int numVolumes = refs.size();
|
|
int numVolumes = refs.size();
|
|
int index = rand.nextInt(numVolumes - 1);
|
|
int index = rand.nextInt(numVolumes - 1);
|
|
- File finalizedDir = ((FsVolumeImpl) refs.get(index))
|
|
|
|
- .getFinalizedDir(bpid);
|
|
|
|
|
|
+ File finalizedDir =
|
|
|
|
+ ((FsVolumeImpl) refs.get(index)).getFinalizedDir(bpid);
|
|
File file = new File(finalizedDir, getMetaFile(id));
|
|
File file = new File(finalizedDir, getMetaFile(id));
|
|
if (file.createNewFile()) {
|
|
if (file.createNewFile()) {
|
|
LOG.info("Created metafile " + file.getName());
|
|
LOG.info("Created metafile " + file.getName());
|
|
@@ -276,7 +276,7 @@ public class TestDirectoryScanner {
|
|
return id;
|
|
return id;
|
|
}
|
|
}
|
|
|
|
|
|
- /** Create block file and corresponding metafile in a rondom volume */
|
|
|
|
|
|
+ /** Create block file and corresponding metafile in a rondom volume. */
|
|
private long createBlockMetaFile() throws IOException {
|
|
private long createBlockMetaFile() throws IOException {
|
|
long id = getFreeBlockId();
|
|
long id = getFreeBlockId();
|
|
|
|
|
|
@@ -318,7 +318,7 @@ public class TestDirectoryScanner {
|
|
long missingBlockFile, long missingMemoryBlocks, long mismatchBlocks)
|
|
long missingBlockFile, long missingMemoryBlocks, long mismatchBlocks)
|
|
throws IOException, InterruptedException, TimeoutException {
|
|
throws IOException, InterruptedException, TimeoutException {
|
|
scan(totalBlocks, diffsize, missingMetaFile, missingBlockFile,
|
|
scan(totalBlocks, diffsize, missingMetaFile, missingBlockFile,
|
|
- missingMemoryBlocks, mismatchBlocks, 0);
|
|
|
|
|
|
+ missingMemoryBlocks, mismatchBlocks, 0);
|
|
}
|
|
}
|
|
|
|
|
|
private void scan(long totalBlocks, int diffsize, long missingMetaFile,
|
|
private void scan(long totalBlocks, int diffsize, long missingMetaFile,
|
|
@@ -332,22 +332,22 @@ public class TestDirectoryScanner {
|
|
verifyStats(totalBlocks, diffsize, missingMetaFile, missingBlockFile,
|
|
verifyStats(totalBlocks, diffsize, missingMetaFile, missingBlockFile,
|
|
missingMemoryBlocks, mismatchBlocks, duplicateBlocks);
|
|
missingMemoryBlocks, mismatchBlocks, duplicateBlocks);
|
|
} catch (AssertionError ex) {
|
|
} catch (AssertionError ex) {
|
|
|
|
+ LOG.warn("Assertion Error", ex);
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
|
|
|
|
return true;
|
|
return true;
|
|
- }, 50, 2000);
|
|
|
|
|
|
+ }, 100, 2000);
|
|
}
|
|
}
|
|
|
|
|
|
private void verifyStats(long totalBlocks, int diffsize, long missingMetaFile,
|
|
private void verifyStats(long totalBlocks, int diffsize, long missingMetaFile,
|
|
long missingBlockFile, long missingMemoryBlocks, long mismatchBlocks,
|
|
long missingBlockFile, long missingMemoryBlocks, long mismatchBlocks,
|
|
long duplicateBlocks) {
|
|
long duplicateBlocks) {
|
|
- assertTrue(scanner.diffs.containsKey(bpid));
|
|
|
|
- LinkedList<FsVolumeSpi.ScanInfo> diff = scanner.diffs.get(bpid);
|
|
|
|
- assertTrue(scanner.stats.containsKey(bpid));
|
|
|
|
- DirectoryScanner.Stats stats = scanner.stats.get(bpid);
|
|
|
|
-
|
|
|
|
|
|
+ Collection<FsVolumeSpi.ScanInfo> diff = scanner.diffs.getScanInfo(bpid);
|
|
assertEquals(diffsize, diff.size());
|
|
assertEquals(diffsize, diff.size());
|
|
|
|
+
|
|
|
|
+ DirectoryScanner.Stats stats = scanner.stats.get(bpid);
|
|
|
|
+ assertNotNull(stats);
|
|
assertEquals(totalBlocks, stats.totalBlocks);
|
|
assertEquals(totalBlocks, stats.totalBlocks);
|
|
assertEquals(missingMetaFile, stats.missingMetaFile);
|
|
assertEquals(missingMetaFile, stats.missingMetaFile);
|
|
assertEquals(missingBlockFile, stats.missingBlockFile);
|
|
assertEquals(missingBlockFile, stats.missingBlockFile);
|
|
@@ -356,20 +356,18 @@ public class TestDirectoryScanner {
|
|
assertEquals(duplicateBlocks, stats.duplicateBlocks);
|
|
assertEquals(duplicateBlocks, stats.duplicateBlocks);
|
|
}
|
|
}
|
|
|
|
|
|
- @Test (timeout=300000)
|
|
|
|
|
|
+ @Test(timeout = 300000)
|
|
public void testRetainBlockOnPersistentStorage() throws Exception {
|
|
public void testRetainBlockOnPersistentStorage() throws Exception {
|
|
- cluster = new MiniDFSCluster
|
|
|
|
- .Builder(CONF)
|
|
|
|
- .storageTypes(new StorageType[] { StorageType.RAM_DISK, StorageType.DEFAULT })
|
|
|
|
- .numDataNodes(1)
|
|
|
|
- .build();
|
|
|
|
|
|
+ cluster = new MiniDFSCluster.Builder(CONF)
|
|
|
|
+ .storageTypes(
|
|
|
|
+ new StorageType[] { StorageType.RAM_DISK, StorageType.DEFAULT })
|
|
|
|
+ .numDataNodes(1).build();
|
|
try {
|
|
try {
|
|
cluster.waitActive();
|
|
cluster.waitActive();
|
|
- DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
|
bpid = cluster.getNamesystem().getBlockPoolId();
|
|
bpid = cluster.getNamesystem().getBlockPoolId();
|
|
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
|
|
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
|
|
client = cluster.getFileSystem().getClient();
|
|
client = cluster.getFileSystem().getClient();
|
|
- scanner = new DirectoryScanner(dataNode, fds, CONF);
|
|
|
|
|
|
+ scanner = new DirectoryScanner(fds, CONF);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.setRetainDiffs(true);
|
|
FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
|
|
FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
|
|
|
|
|
|
@@ -397,24 +395,22 @@ public class TestDirectoryScanner {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- @Test (timeout=300000)
|
|
|
|
|
|
+ @Test(timeout = 300000)
|
|
public void testDeleteBlockOnTransientStorage() throws Exception {
|
|
public void testDeleteBlockOnTransientStorage() throws Exception {
|
|
- cluster = new MiniDFSCluster
|
|
|
|
- .Builder(CONF)
|
|
|
|
- .storageTypes(new StorageType[] { StorageType.RAM_DISK, StorageType.DEFAULT })
|
|
|
|
- .numDataNodes(1)
|
|
|
|
- .build();
|
|
|
|
|
|
+ cluster = new MiniDFSCluster.Builder(CONF)
|
|
|
|
+ .storageTypes(
|
|
|
|
+ new StorageType[] { StorageType.RAM_DISK, StorageType.DEFAULT })
|
|
|
|
+ .numDataNodes(1).build();
|
|
try {
|
|
try {
|
|
cluster.waitActive();
|
|
cluster.waitActive();
|
|
bpid = cluster.getNamesystem().getBlockPoolId();
|
|
bpid = cluster.getNamesystem().getBlockPoolId();
|
|
- DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
|
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
|
|
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
|
|
client = cluster.getFileSystem().getClient();
|
|
client = cluster.getFileSystem().getClient();
|
|
- scanner = new DirectoryScanner(dataNode, fds, CONF);
|
|
|
|
|
|
+ scanner = new DirectoryScanner(fds, CONF);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.setRetainDiffs(true);
|
|
FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
|
|
FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
|
|
|
|
|
|
- // Create a file file on RAM_DISK
|
|
|
|
|
|
+ // Create a file on RAM_DISK
|
|
List<LocatedBlock> blocks =
|
|
List<LocatedBlock> blocks =
|
|
createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH, true);
|
|
createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH, true);
|
|
|
|
|
|
@@ -440,14 +436,14 @@ public class TestDirectoryScanner {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- @Test (timeout=600000)
|
|
|
|
|
|
+ @Test(timeout = 600000)
|
|
public void testDirectoryScanner() throws Exception {
|
|
public void testDirectoryScanner() throws Exception {
|
|
// Run the test with and without parallel scanning
|
|
// Run the test with and without parallel scanning
|
|
for (int parallelism = 1; parallelism < 3; parallelism++) {
|
|
for (int parallelism = 1; parallelism < 3; parallelism++) {
|
|
runTest(parallelism);
|
|
runTest(parallelism);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
public void runTest(int parallelism) throws Exception {
|
|
public void runTest(int parallelism) throws Exception {
|
|
cluster = new MiniDFSCluster.Builder(CONF).build();
|
|
cluster = new MiniDFSCluster.Builder(CONF).build();
|
|
try {
|
|
try {
|
|
@@ -456,9 +452,9 @@ public class TestDirectoryScanner {
|
|
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
|
|
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
|
|
client = cluster.getFileSystem().getClient();
|
|
client = cluster.getFileSystem().getClient();
|
|
CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY,
|
|
CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY,
|
|
- parallelism);
|
|
|
|
- DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
|
- scanner = new DirectoryScanner(dataNode, fds, CONF);
|
|
|
|
|
|
+ parallelism);
|
|
|
|
+
|
|
|
|
+ scanner = new DirectoryScanner(fds, CONF);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.setRetainDiffs(true);
|
|
|
|
|
|
// Add files with 100 blocks
|
|
// Add files with 100 blocks
|
|
@@ -492,7 +488,7 @@ public class TestDirectoryScanner {
|
|
// Test5: A metafile exists for which there is no block file and
|
|
// Test5: A metafile exists for which there is no block file and
|
|
// a block in memory
|
|
// a block in memory
|
|
blockId = createMetaFile();
|
|
blockId = createMetaFile();
|
|
- scan(totalBlocks+1, 1, 0, 1, 1, 0);
|
|
|
|
|
|
+ scan(totalBlocks + 1, 1, 0, 1, 1, 0);
|
|
File metafile = new File(getMetaFile(blockId));
|
|
File metafile = new File(getMetaFile(blockId));
|
|
assertTrue(!metafile.exists());
|
|
assertTrue(!metafile.exists());
|
|
scan(totalBlocks, 0, 0, 0, 0, 0);
|
|
scan(totalBlocks, 0, 0, 0, 0, 0);
|
|
@@ -521,7 +517,7 @@ public class TestDirectoryScanner {
|
|
scan(totalBlocks, 0, 0, 0, 0, 0);
|
|
scan(totalBlocks, 0, 0, 0, 0, 0);
|
|
|
|
|
|
// Test9: create a bunch of blocks files
|
|
// Test9: create a bunch of blocks files
|
|
- for (int i = 0; i < 10 ; i++) {
|
|
|
|
|
|
+ for (int i = 0; i < 10; i++) {
|
|
blockId = createBlockFile();
|
|
blockId = createBlockFile();
|
|
}
|
|
}
|
|
totalBlocks += 10;
|
|
totalBlocks += 10;
|
|
@@ -529,14 +525,14 @@ public class TestDirectoryScanner {
|
|
scan(totalBlocks, 0, 0, 0, 0, 0);
|
|
scan(totalBlocks, 0, 0, 0, 0, 0);
|
|
|
|
|
|
// Test10: create a bunch of metafiles
|
|
// Test10: create a bunch of metafiles
|
|
- for (int i = 0; i < 10 ; i++) {
|
|
|
|
|
|
+ for (int i = 0; i < 10; i++) {
|
|
blockId = createMetaFile();
|
|
blockId = createMetaFile();
|
|
}
|
|
}
|
|
- scan(totalBlocks+10, 10, 0, 10, 10, 0);
|
|
|
|
|
|
+ scan(totalBlocks + 10, 10, 0, 10, 10, 0);
|
|
scan(totalBlocks, 0, 0, 0, 0, 0);
|
|
scan(totalBlocks, 0, 0, 0, 0, 0);
|
|
|
|
|
|
// Test11: create a bunch block files and meta files
|
|
// Test11: create a bunch block files and meta files
|
|
- for (int i = 0; i < 10 ; i++) {
|
|
|
|
|
|
+ for (int i = 0; i < 10; i++) {
|
|
blockId = createBlockMetaFile();
|
|
blockId = createBlockMetaFile();
|
|
}
|
|
}
|
|
totalBlocks += 10;
|
|
totalBlocks += 10;
|
|
@@ -544,7 +540,7 @@ public class TestDirectoryScanner {
|
|
scan(totalBlocks, 0, 0, 0, 0, 0);
|
|
scan(totalBlocks, 0, 0, 0, 0, 0);
|
|
|
|
|
|
// Test12: truncate block files to test block length mismatch
|
|
// Test12: truncate block files to test block length mismatch
|
|
- for (int i = 0; i < 10 ; i++) {
|
|
|
|
|
|
+ for (int i = 0; i < 10; i++) {
|
|
truncateBlockFile();
|
|
truncateBlockFile();
|
|
}
|
|
}
|
|
scan(totalBlocks, 10, 0, 0, 0, 10);
|
|
scan(totalBlocks, 10, 0, 0, 0, 10);
|
|
@@ -557,9 +553,9 @@ public class TestDirectoryScanner {
|
|
deleteMetaFile();
|
|
deleteMetaFile();
|
|
deleteBlockFile();
|
|
deleteBlockFile();
|
|
truncateBlockFile();
|
|
truncateBlockFile();
|
|
- scan(totalBlocks+3, 6, 2, 2, 3, 2);
|
|
|
|
- scan(totalBlocks+1, 0, 0, 0, 0, 0);
|
|
|
|
-
|
|
|
|
|
|
+ scan(totalBlocks + 3, 6, 2, 2, 3, 2);
|
|
|
|
+ scan(totalBlocks + 1, 0, 0, 0, 0, 0);
|
|
|
|
+
|
|
// Test14: make sure no throttling is happening
|
|
// Test14: make sure no throttling is happening
|
|
assertTrue("Throttle appears to be engaged",
|
|
assertTrue("Throttle appears to be engaged",
|
|
scanner.timeWaitingMs.get() < 10L);
|
|
scanner.timeWaitingMs.get() < 10L);
|
|
@@ -567,10 +563,11 @@ public class TestDirectoryScanner {
|
|
scanner.timeRunningMs.get() > 0L);
|
|
scanner.timeRunningMs.get() > 0L);
|
|
|
|
|
|
// Test15: validate clean shutdown of DirectoryScanner
|
|
// Test15: validate clean shutdown of DirectoryScanner
|
|
- ////assertTrue(scanner.getRunStatus()); //assumes "real" FSDataset, not sim
|
|
|
|
|
|
+ //// assertTrue(scanner.getRunStatus()); //assumes "real" FSDataset, not
|
|
|
|
+ // sim
|
|
scanner.shutdown();
|
|
scanner.shutdown();
|
|
assertFalse(scanner.getRunStatus());
|
|
assertFalse(scanner.getRunStatus());
|
|
-
|
|
|
|
|
|
+
|
|
} finally {
|
|
} finally {
|
|
if (scanner != null) {
|
|
if (scanner != null) {
|
|
scanner.shutdown();
|
|
scanner.shutdown();
|
|
@@ -582,17 +579,17 @@ public class TestDirectoryScanner {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Test that the timeslice throttle limits the report compiler thread's
|
|
* Test that the timeslice throttle limits the report compiler thread's
|
|
- * execution time correctly. We test by scanning a large block pool and
|
|
|
|
|
|
+ * execution time correctly. We test by scanning a large block pool and
|
|
* comparing the time spent waiting to the time spent running.
|
|
* comparing the time spent waiting to the time spent running.
|
|
*
|
|
*
|
|
- * The block pool has to be large, or the ratio will be off. The throttle
|
|
|
|
- * allows the report compiler thread to finish its current cycle when
|
|
|
|
- * blocking it, so the ratio will always be a little lower than expected.
|
|
|
|
- * The smaller the block pool, the further off the ratio will be.
|
|
|
|
|
|
+ * The block pool has to be large, or the ratio will be off. The throttle
|
|
|
|
+ * allows the report compiler thread to finish its current cycle when blocking
|
|
|
|
+ * it, so the ratio will always be a little lower than expected. The smaller
|
|
|
|
+ * the block pool, the further off the ratio will be.
|
|
*
|
|
*
|
|
* @throws Exception thrown on unexpected failure
|
|
* @throws Exception thrown on unexpected failure
|
|
*/
|
|
*/
|
|
- @Test (timeout=600000)
|
|
|
|
|
|
+ @Test(timeout = 600000)
|
|
public void testThrottling() throws Exception {
|
|
public void testThrottling() throws Exception {
|
|
Configuration conf = new Configuration(CONF);
|
|
Configuration conf = new Configuration(CONF);
|
|
|
|
|
|
@@ -611,10 +608,9 @@ public class TestDirectoryScanner {
|
|
conf.setInt(
|
|
conf.setInt(
|
|
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
|
|
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
|
|
100);
|
|
100);
|
|
- DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
|
|
|
|
|
- final int maxBlocksPerFile = (int) DFSConfigKeys
|
|
|
|
- .DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT;
|
|
|
|
|
|
+ final int maxBlocksPerFile =
|
|
|
|
+ (int) DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT;
|
|
int numBlocksToCreate = blocks;
|
|
int numBlocksToCreate = blocks;
|
|
while (numBlocksToCreate > 0) {
|
|
while (numBlocksToCreate > 0) {
|
|
final int toCreate = Math.min(maxBlocksPerFile, numBlocksToCreate);
|
|
final int toCreate = Math.min(maxBlocksPerFile, numBlocksToCreate);
|
|
@@ -627,7 +623,7 @@ public class TestDirectoryScanner {
|
|
int retries = maxRetries;
|
|
int retries = maxRetries;
|
|
|
|
|
|
while ((retries > 0) && ((ratio < 7f) || (ratio > 10f))) {
|
|
while ((retries > 0) && ((ratio < 7f) || (ratio > 10f))) {
|
|
- scanner = new DirectoryScanner(dataNode, fds, conf);
|
|
|
|
|
|
+ scanner = new DirectoryScanner(fds, conf);
|
|
ratio = runThrottleTest(blocks);
|
|
ratio = runThrottleTest(blocks);
|
|
retries -= 1;
|
|
retries -= 1;
|
|
}
|
|
}
|
|
@@ -645,7 +641,7 @@ public class TestDirectoryScanner {
|
|
retries = maxRetries;
|
|
retries = maxRetries;
|
|
|
|
|
|
while ((retries > 0) && ((ratio < 2.75f) || (ratio > 4.5f))) {
|
|
while ((retries > 0) && ((ratio < 2.75f) || (ratio > 4.5f))) {
|
|
- scanner = new DirectoryScanner(dataNode, fds, conf);
|
|
|
|
|
|
+ scanner = new DirectoryScanner(fds, conf);
|
|
ratio = runThrottleTest(blocks);
|
|
ratio = runThrottleTest(blocks);
|
|
retries -= 1;
|
|
retries -= 1;
|
|
}
|
|
}
|
|
@@ -664,7 +660,7 @@ public class TestDirectoryScanner {
|
|
retries = maxRetries;
|
|
retries = maxRetries;
|
|
|
|
|
|
while ((retries > 0) && ((ratio < 7f) || (ratio > 10f))) {
|
|
while ((retries > 0) && ((ratio < 7f) || (ratio > 10f))) {
|
|
- scanner = new DirectoryScanner(dataNode, fds, conf);
|
|
|
|
|
|
+ scanner = new DirectoryScanner(fds, conf);
|
|
ratio = runThrottleTest(blocks);
|
|
ratio = runThrottleTest(blocks);
|
|
retries -= 1;
|
|
retries -= 1;
|
|
}
|
|
}
|
|
@@ -675,7 +671,7 @@ public class TestDirectoryScanner {
|
|
assertTrue("Throttle is too permissive", ratio >= 7f);
|
|
assertTrue("Throttle is too permissive", ratio >= 7f);
|
|
|
|
|
|
// Test with no limit
|
|
// Test with no limit
|
|
- scanner = new DirectoryScanner(dataNode, fds, CONF);
|
|
|
|
|
|
+ scanner = new DirectoryScanner(fds, CONF);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.setRetainDiffs(true);
|
|
scan(blocks, 0, 0, 0, 0, 0);
|
|
scan(blocks, 0, 0, 0, 0, 0);
|
|
scanner.shutdown();
|
|
scanner.shutdown();
|
|
@@ -686,7 +682,7 @@ public class TestDirectoryScanner {
|
|
assertTrue("Report complier threads logged no execution time",
|
|
assertTrue("Report complier threads logged no execution time",
|
|
scanner.timeRunningMs.get() > 0L);
|
|
scanner.timeRunningMs.get() > 0L);
|
|
|
|
|
|
- // Test with a 1ms limit. This also tests whether the scanner can be
|
|
|
|
|
|
+ // Test with a 1ms limit. This also tests whether the scanner can be
|
|
// shutdown cleanly in mid stride.
|
|
// shutdown cleanly in mid stride.
|
|
conf.setInt(
|
|
conf.setInt(
|
|
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
|
|
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
|
|
@@ -698,7 +694,7 @@ public class TestDirectoryScanner {
|
|
|
|
|
|
try {
|
|
try {
|
|
while ((retries > 0) && (ratio < 10)) {
|
|
while ((retries > 0) && (ratio < 10)) {
|
|
- scanner = new DirectoryScanner(dataNode, fds, conf);
|
|
|
|
|
|
+ scanner = new DirectoryScanner(fds, conf);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.setRetainDiffs(true);
|
|
|
|
|
|
final AtomicLong nowMs = new AtomicLong();
|
|
final AtomicLong nowMs = new AtomicLong();
|
|
@@ -728,7 +724,7 @@ public class TestDirectoryScanner {
|
|
}
|
|
}
|
|
|
|
|
|
ratio =
|
|
ratio =
|
|
- (float)scanner.timeWaitingMs.get() / scanner.timeRunningMs.get();
|
|
|
|
|
|
+ (float) scanner.timeWaitingMs.get() / scanner.timeRunningMs.get();
|
|
retries -= 1;
|
|
retries -= 1;
|
|
}
|
|
}
|
|
} finally {
|
|
} finally {
|
|
@@ -737,8 +733,7 @@ public class TestDirectoryScanner {
|
|
|
|
|
|
// We just want to test that it waits a lot, but it also runs some
|
|
// We just want to test that it waits a lot, but it also runs some
|
|
LOG.info("RATIO: " + ratio);
|
|
LOG.info("RATIO: " + ratio);
|
|
- assertTrue("Throttle is too permissive",
|
|
|
|
- ratio > 10);
|
|
|
|
|
|
+ assertTrue("Throttle is too permissive", ratio > 8);
|
|
assertTrue("Report complier threads logged no execution time",
|
|
assertTrue("Report complier threads logged no execution time",
|
|
scanner.timeRunningMs.get() > 0L);
|
|
scanner.timeRunningMs.get() > 0L);
|
|
|
|
|
|
@@ -746,7 +741,7 @@ public class TestDirectoryScanner {
|
|
conf.setInt(
|
|
conf.setInt(
|
|
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
|
|
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
|
|
0);
|
|
0);
|
|
- scanner = new DirectoryScanner(dataNode, fds, conf);
|
|
|
|
|
|
+ scanner = new DirectoryScanner(fds, conf);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.setRetainDiffs(true);
|
|
scan(blocks, 0, 0, 0, 0, 0);
|
|
scan(blocks, 0, 0, 0, 0, 0);
|
|
scanner.shutdown();
|
|
scanner.shutdown();
|
|
@@ -761,7 +756,7 @@ public class TestDirectoryScanner {
|
|
conf.setInt(
|
|
conf.setInt(
|
|
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
|
|
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
|
|
1000);
|
|
1000);
|
|
- scanner = new DirectoryScanner(dataNode, fds, conf);
|
|
|
|
|
|
+ scanner = new DirectoryScanner(fds, conf);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.setRetainDiffs(true);
|
|
scan(blocks, 0, 0, 0, 0, 0);
|
|
scan(blocks, 0, 0, 0, 0, 0);
|
|
scanner.shutdown();
|
|
scanner.shutdown();
|
|
@@ -777,9 +772,8 @@ public class TestDirectoryScanner {
|
|
conf.setInt(
|
|
conf.setInt(
|
|
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
|
|
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
|
|
10);
|
|
10);
|
|
- conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY,
|
|
|
|
- 1);
|
|
|
|
- scanner = new DirectoryScanner(dataNode, fds, conf);
|
|
|
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1);
|
|
|
|
+ scanner = new DirectoryScanner(fds, conf);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.start();
|
|
scanner.start();
|
|
|
|
|
|
@@ -805,7 +799,7 @@ public class TestDirectoryScanner {
|
|
scanner.shutdown();
|
|
scanner.shutdown();
|
|
assertFalse(scanner.getRunStatus());
|
|
assertFalse(scanner.getRunStatus());
|
|
|
|
|
|
- return (float)scanner.timeWaitingMs.get() / scanner.timeRunningMs.get();
|
|
|
|
|
|
+ return (float) scanner.timeWaitingMs.get() / scanner.timeRunningMs.get();
|
|
}
|
|
}
|
|
|
|
|
|
private void verifyAddition(long blockId, long genStamp, long size) {
|
|
private void verifyAddition(long blockId, long genStamp, long size) {
|
|
@@ -836,7 +830,7 @@ public class TestDirectoryScanner {
|
|
assertNotNull(memBlock);
|
|
assertNotNull(memBlock);
|
|
assertEquals(genStamp, memBlock.getGenerationStamp());
|
|
assertEquals(genStamp, memBlock.getGenerationStamp());
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private void verifyStorageType(long blockId, boolean expectTransient) {
|
|
private void verifyStorageType(long blockId, boolean expectTransient) {
|
|
final ReplicaInfo memBlock;
|
|
final ReplicaInfo memBlock;
|
|
memBlock = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId);
|
|
memBlock = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId);
|
|
@@ -859,7 +853,7 @@ public class TestDirectoryScanner {
|
|
public long getAvailable() throws IOException {
|
|
public long getAvailable() throws IOException {
|
|
return 0;
|
|
return 0;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
public File getFinalizedDir(String bpid) throws IOException {
|
|
public File getFinalizedDir(String bpid) throws IOException {
|
|
return new File("/base/current/" + bpid + "/finalized");
|
|
return new File("/base/current/" + bpid + "/finalized");
|
|
}
|
|
}
|
|
@@ -898,10 +892,11 @@ public class TestDirectoryScanner {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public BlockIterator loadBlockIterator(String bpid, String name)
|
|
public BlockIterator loadBlockIterator(String bpid, String name)
|
|
- throws IOException {
|
|
|
|
|
|
+ throws IOException {
|
|
throw new UnsupportedOperationException();
|
|
throw new UnsupportedOperationException();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @SuppressWarnings("rawtypes")
|
|
@Override
|
|
@Override
|
|
public FsDatasetSpi getDataset() {
|
|
public FsDatasetSpi getDataset() {
|
|
throw new UnsupportedOperationException();
|
|
throw new UnsupportedOperationException();
|
|
@@ -923,8 +918,8 @@ public class TestDirectoryScanner {
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public byte[] loadLastPartialChunkChecksum(
|
|
|
|
- File blockFile, File metaFile) throws IOException {
|
|
|
|
|
|
+ public byte[] loadLastPartialChunkChecksum(File blockFile, File metaFile)
|
|
|
|
+ throws IOException {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -945,7 +940,6 @@ public class TestDirectoryScanner {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
-
|
|
|
|
@Override
|
|
@Override
|
|
public VolumeCheckResult check(VolumeCheckContext context)
|
|
public VolumeCheckResult check(VolumeCheckContext context)
|
|
throws Exception {
|
|
throws Exception {
|
|
@@ -954,11 +948,11 @@ public class TestDirectoryScanner {
|
|
}
|
|
}
|
|
|
|
|
|
private final static TestFsVolumeSpi TEST_VOLUME = new TestFsVolumeSpi();
|
|
private final static TestFsVolumeSpi TEST_VOLUME = new TestFsVolumeSpi();
|
|
-
|
|
|
|
|
|
+
|
|
private final static String BPID_1 = "BP-783049782-127.0.0.1-1370971773491";
|
|
private final static String BPID_1 = "BP-783049782-127.0.0.1-1370971773491";
|
|
-
|
|
|
|
|
|
+
|
|
private final static String BPID_2 = "BP-367845636-127.0.0.1-5895645674231";
|
|
private final static String BPID_2 = "BP-367845636-127.0.0.1-5895645674231";
|
|
-
|
|
|
|
|
|
+
|
|
void testScanInfoObject(long blockId, File blockFile, File metaFile)
|
|
void testScanInfoObject(long blockId, File blockFile, File metaFile)
|
|
throws Exception {
|
|
throws Exception {
|
|
FsVolumeSpi.ScanInfo scanInfo =
|
|
FsVolumeSpi.ScanInfo scanInfo =
|
|
@@ -978,7 +972,7 @@ public class TestDirectoryScanner {
|
|
}
|
|
}
|
|
assertEquals(TEST_VOLUME, scanInfo.getVolume());
|
|
assertEquals(TEST_VOLUME, scanInfo.getVolume());
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
void testScanInfoObject(long blockId) throws Exception {
|
|
void testScanInfoObject(long blockId) throws Exception {
|
|
FsVolumeSpi.ScanInfo scanInfo =
|
|
FsVolumeSpi.ScanInfo scanInfo =
|
|
new FsVolumeSpi.ScanInfo(blockId, null, null, null);
|
|
new FsVolumeSpi.ScanInfo(blockId, null, null, null);
|
|
@@ -987,7 +981,7 @@ public class TestDirectoryScanner {
|
|
assertNull(scanInfo.getMetaFile());
|
|
assertNull(scanInfo.getMetaFile());
|
|
}
|
|
}
|
|
|
|
|
|
- @Test(timeout=120000)
|
|
|
|
|
|
+ @Test(timeout = 120000)
|
|
public void TestScanInfo() throws Exception {
|
|
public void TestScanInfo() throws Exception {
|
|
testScanInfoObject(123,
|
|
testScanInfoObject(123,
|
|
new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath(),
|
|
new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath(),
|
|
@@ -998,13 +992,10 @@ public class TestDirectoryScanner {
|
|
new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath(),
|
|
new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath(),
|
|
"blk_123"),
|
|
"blk_123"),
|
|
null);
|
|
null);
|
|
- testScanInfoObject(523,
|
|
|
|
- null,
|
|
|
|
|
|
+ testScanInfoObject(523, null,
|
|
new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath(),
|
|
new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath(),
|
|
"blk_123__1009.meta"));
|
|
"blk_123__1009.meta"));
|
|
- testScanInfoObject(789,
|
|
|
|
- null,
|
|
|
|
- null);
|
|
|
|
|
|
+ testScanInfoObject(789, null, null);
|
|
testScanInfoObject(456);
|
|
testScanInfoObject(456);
|
|
testScanInfoObject(123,
|
|
testScanInfoObject(123,
|
|
new File(TEST_VOLUME.getFinalizedDir(BPID_2).getAbsolutePath(),
|
|
new File(TEST_VOLUME.getFinalizedDir(BPID_2).getAbsolutePath(),
|
|
@@ -1027,7 +1018,6 @@ public class TestDirectoryScanner {
|
|
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
|
|
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
|
|
client = cluster.getFileSystem().getClient();
|
|
client = cluster.getFileSystem().getClient();
|
|
CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
|
|
CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
|
|
- DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
|
|
|
|
|
// Add files with 2 blocks
|
|
// Add files with 2 blocks
|
|
createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH * 2, false);
|
|
createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH * 2, false);
|
|
@@ -1047,7 +1037,7 @@ public class TestDirectoryScanner {
|
|
FsDatasetSpi<? extends FsVolumeSpi> spyFds = Mockito.spy(fds);
|
|
FsDatasetSpi<? extends FsVolumeSpi> spyFds = Mockito.spy(fds);
|
|
Mockito.doReturn(volReferences).when(spyFds).getFsVolumeReferences();
|
|
Mockito.doReturn(volReferences).when(spyFds).getFsVolumeReferences();
|
|
|
|
|
|
- scanner = new DirectoryScanner(dataNode, spyFds, CONF);
|
|
|
|
|
|
+ scanner = new DirectoryScanner(spyFds, CONF);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.reconcile();
|
|
scanner.reconcile();
|
|
} finally {
|
|
} finally {
|
|
@@ -1061,28 +1051,27 @@ public class TestDirectoryScanner {
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testDirectoryScannerInFederatedCluster() throws Exception {
|
|
public void testDirectoryScannerInFederatedCluster() throws Exception {
|
|
- //Create Federated cluster with two nameservices and one DN
|
|
|
|
|
|
+ // Create Federated cluster with two nameservices and one DN
|
|
try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF)
|
|
try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF)
|
|
.nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(2))
|
|
.nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(2))
|
|
.numDataNodes(1).build()) {
|
|
.numDataNodes(1).build()) {
|
|
cluster.waitActive();
|
|
cluster.waitActive();
|
|
cluster.transitionToActive(1);
|
|
cluster.transitionToActive(1);
|
|
cluster.transitionToActive(3);
|
|
cluster.transitionToActive(3);
|
|
- DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
|
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
|
|
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
|
|
- //Create one block in first nameservice
|
|
|
|
|
|
+ // Create one block in first nameservice
|
|
FileSystem fs = cluster.getFileSystem(1);
|
|
FileSystem fs = cluster.getFileSystem(1);
|
|
int bp1Files = 1;
|
|
int bp1Files = 1;
|
|
writeFile(fs, bp1Files);
|
|
writeFile(fs, bp1Files);
|
|
- //Create two blocks in second nameservice
|
|
|
|
|
|
+ // Create two blocks in second nameservice
|
|
FileSystem fs2 = cluster.getFileSystem(3);
|
|
FileSystem fs2 = cluster.getFileSystem(3);
|
|
int bp2Files = 2;
|
|
int bp2Files = 2;
|
|
writeFile(fs2, bp2Files);
|
|
writeFile(fs2, bp2Files);
|
|
- //Call the Directory scanner
|
|
|
|
- scanner = new DirectoryScanner(dataNode, fds, CONF);
|
|
|
|
|
|
+ // Call the Directory scanner
|
|
|
|
+ scanner = new DirectoryScanner(fds, CONF);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.setRetainDiffs(true);
|
|
scanner.reconcile();
|
|
scanner.reconcile();
|
|
- //Check blocks in corresponding BP
|
|
|
|
|
|
+ // Check blocks in corresponding BP
|
|
|
|
|
|
GenericTestUtils.waitFor(() -> {
|
|
GenericTestUtils.waitFor(() -> {
|
|
try {
|
|
try {
|