|
@@ -49,6 +49,7 @@ import org.apache.commons.io.FileUtils;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.DF;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
+import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.DFSClient;
|
|
@@ -58,6 +59,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
@@ -71,6 +73,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
|
|
|
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.namenode.NameNode;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.util.AutoCloseableLock;
|
|
@@ -507,6 +510,53 @@ public class TestDirectoryScanner {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 600000)
|
|
|
+ public void testRegularBlock() throws Exception {
|
|
|
+ Configuration conf = getConfiguration();
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).build();
|
|
|
+ try {
|
|
|
+ cluster.waitActive();
|
|
|
+ bpid = cluster.getNamesystem().getBlockPoolId();
|
|
|
+ fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
|
|
|
+ client = cluster.getFileSystem().getClient();
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
|
|
|
+ // log trace
|
|
|
+ GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer.
|
|
|
+ captureLogs(NameNode.stateChangeLog);
|
|
|
+ // Add files with 5 blocks
|
|
|
+ createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH * 5, false);
|
|
|
+
|
|
|
+ List<ReplicaInfo> infos = new ArrayList<>(FsDatasetTestUtil.getReplicas(fds, bpid));
|
|
|
+ ReplicaInfo lastReplica = infos.get(infos.size() - 1);
|
|
|
+ ReplicaInfo penultimateReplica = infos.get(infos.size() - 2);
|
|
|
+
|
|
|
+ String blockParent = new File(lastReplica.getBlockURI().getPath()).getParent();
|
|
|
+ File lastBlockFile = new File(blockParent, getBlockFile(lastReplica.getBlockId()));
|
|
|
+ File penultimateBlockFile = new File(blockParent,
|
|
|
+ getBlockFile(penultimateReplica.getBlockId()));
|
|
|
+ FileUtil.symLink(lastBlockFile.toString(), penultimateBlockFile.toString());
|
|
|
+ ExtendedBlock block = new ExtendedBlock(bpid, penultimateReplica.getBlockId());
|
|
|
+
|
|
|
+ scanner = new DirectoryScanner(fds, conf);
|
|
|
+ scanner.setRetainDiffs(true);
|
|
|
+ scanner.reconcile();
|
|
|
+ DirectoryScanner.Stats stats = scanner.stats.get(bpid);
|
|
|
+ assertNotNull(stats);
|
|
|
+ assertEquals(1, stats.mismatchBlocks);
|
|
|
+
|
|
|
+ // check nn log
|
|
|
+ String msg = "*DIR* reportBadBlocks for block: " + bpid + ":" +
|
|
|
+ getBlockFile(block.getBlockId());
|
|
|
+ assertTrue(logCapturer.getOutput().contains(msg));
|
|
|
+ } finally {
|
|
|
+ if (scanner != null) {
|
|
|
+ scanner.shutdown();
|
|
|
+ scanner = null;
|
|
|
+ }
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test(timeout = 600000)
|
|
|
public void testDirectoryScanner() throws Exception {
|
|
|
// Run the test with and without parallel scanning
|