|
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage;
|
|
|
import org.apache.hadoop.hdfs.server.common.StorageInfo;
|
|
@@ -98,6 +99,7 @@ import static org.mockito.Mockito.never;
|
|
|
import static org.mockito.Mockito.spy;
|
|
|
import static org.mockito.Mockito.verify;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
+
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
@@ -691,4 +693,45 @@ public class TestFsDatasetImpl {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test(timeout = 30000)
|
|
|
+ public void testReportBadBlocks() throws Exception {
|
|
|
+ boolean threwException = false;
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
+ try {
|
|
|
+ Configuration config = new HdfsConfiguration();
|
|
|
+ cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ Assert.assertEquals(0, cluster.getNamesystem().getCorruptReplicaBlocks());
|
|
|
+ DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
+ ExtendedBlock block =
|
|
|
+ new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(), 0);
|
|
|
+ try {
|
|
|
+ // Test the reportBadBlocks when the volume is null
|
|
|
+ dataNode.reportBadBlocks(block);
|
|
|
+ } catch (NullPointerException npe) {
|
|
|
+ threwException = true;
|
|
|
+ }
|
|
|
+ Thread.sleep(3000);
|
|
|
+ Assert.assertFalse(threwException);
|
|
|
+ Assert.assertEquals(0, cluster.getNamesystem().getCorruptReplicaBlocks());
|
|
|
+
|
|
|
+ FileSystem fs = cluster.getFileSystem();
|
|
|
+ Path filePath = new Path("testData");
|
|
|
+ DFSTestUtil.createFile(fs, filePath, 1, (short) 1, 0);
|
|
|
+
|
|
|
+ block = DFSTestUtil.getFirstBlock(fs, filePath);
|
|
|
+ // Test for the overloaded method reportBadBlocks
|
|
|
+ dataNode.reportBadBlocks(block, dataNode.getFSDataset()
|
|
|
+ .getFsVolumeReferences().get(0));
|
|
|
+ Thread.sleep(3000);
|
|
|
+ BlockManagerTestUtil.updateState(cluster.getNamesystem()
|
|
|
+ .getBlockManager());
|
|
|
+ // Verify the bad block has been reported to namenode
|
|
|
+ Assert.assertEquals(1, cluster.getNamesystem().getCorruptReplicaBlocks());
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|