|
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
@@ -468,4 +469,68 @@ public class TestGetBlocks {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testReadSkipStaleStorage() throws Exception {
|
|
|
+ final short repFactor = (short) 1;
|
|
|
+ final int blockNum = 64;
|
|
|
+ final int storageNum = 2;
|
|
|
+ final int fileLen = BLOCK_SIZE * blockNum;
|
|
|
+ final Path path = new Path("testReadSkipStaleStorage");
|
|
|
+ final Configuration conf = new HdfsConfiguration();
|
|
|
+
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .numDataNodes(1)
|
|
|
+ .storagesPerDatanode(storageNum)
|
|
|
+ .build();
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ FileSystem fs = cluster.getFileSystem();
|
|
|
+ DFSTestUtil.createFile(fs, path, false, 1024, fileLen,
|
|
|
+ BLOCK_SIZE, repFactor, 0, true);
|
|
|
+
|
|
|
+ // get datanode info
|
|
|
+ ClientProtocol client = NameNodeProxies.createProxy(conf,
|
|
|
+ cluster.getFileSystem(0).getUri(),
|
|
|
+ ClientProtocol.class).getProxy();
|
|
|
+ DatanodeInfo[] dataNodes = client.getDatanodeReport(DatanodeReportType.ALL);
|
|
|
+
|
|
|
+ // get storage info
|
|
|
+ BlockManager bm0 = cluster.getNamesystem(0).getBlockManager();
|
|
|
+ DatanodeStorageInfo[] storageInfos = bm0.getDatanodeManager()
|
|
|
+ .getDatanode(dataNodes[0].getDatanodeUuid()).getStorageInfos();
|
|
|
+
|
|
|
+ InetSocketAddress addr = new InetSocketAddress("localhost",
|
|
|
+ cluster.getNameNodePort());
|
|
|
+ NamenodeProtocol namenode = NameNodeProxies.createProxy(conf,
|
|
|
+ DFSUtilClient.getNNUri(addr), NamenodeProtocol.class).getProxy();
|
|
|
+
|
|
|
+ // check blocks count equals to blockNum
|
|
|
+ BlockWithLocations[] blocks = namenode.getBlocks(
|
|
|
+ dataNodes[0], fileLen*2, 0, 0).getBlocks();
|
|
|
+ assertEquals(blockNum, blocks.length);
|
|
|
+
|
|
|
+ // calculate the block count on storage[0]
|
|
|
+ int count = 0;
|
|
|
+ for (BlockWithLocations b : blocks) {
|
|
|
+ for (String s : b.getStorageIDs()) {
|
|
|
+ if (s.equals(storageInfos[0].getStorageID())) {
|
|
|
+ count++;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // set storage[0] stale
|
|
|
+ storageInfos[0].setBlockContentsStale(true);
|
|
|
+ blocks = namenode.getBlocks(
|
|
|
+ dataNodes[0], fileLen*2, 0, 0).getBlocks();
|
|
|
+ assertEquals(blockNum - count, blocks.length);
|
|
|
+
|
|
|
+ // set all storage stale
|
|
|
+ bm0.getDatanodeManager().markAllDatanodesStale();
|
|
|
+ blocks = namenode.getBlocks(
|
|
|
+ dataNodes[0], fileLen*2, 0, 0).getBlocks();
|
|
|
+ assertEquals(0, blocks.length);
|
|
|
+ }
|
|
|
}
|