|
@@ -238,26 +238,26 @@ public class TestGetBlocks {
|
|
|
DFSUtilClient.getNNUri(addr), NamenodeProtocol.class).getProxy();
|
|
|
|
|
|
// Should return all 13 blocks, as minBlockSize is not passed
|
|
|
- locs = namenode.getBlocks(dataNodes[0], fileLen, 0).getBlocks();
|
|
|
+ locs = namenode.getBlocks(dataNodes[0], fileLen, 0, 0).getBlocks();
|
|
|
assertEquals(blkLocsSize, locs.length);
|
|
|
|
|
|
assertEquals(locs[0].getStorageIDs().length, replicationFactor);
|
|
|
assertEquals(locs[1].getStorageIDs().length, replicationFactor);
|
|
|
|
|
|
// Should return 12 blocks, as minBlockSize is blkSize
|
|
|
- locs = namenode.getBlocks(dataNodes[0], fileLen, blkSize).getBlocks();
|
|
|
+ locs = namenode.getBlocks(dataNodes[0], fileLen, blkSize, 0).getBlocks();
|
|
|
assertEquals(blkLocsSize - 1, locs.length);
|
|
|
assertEquals(locs[0].getStorageIDs().length, replicationFactor);
|
|
|
assertEquals(locs[1].getStorageIDs().length, replicationFactor);
|
|
|
|
|
|
// get blocks of size BlockSize from dataNodes[0]
|
|
|
locs = namenode.getBlocks(dataNodes[0], blkSize,
|
|
|
- blkSize).getBlocks();
|
|
|
+ blkSize, 0).getBlocks();
|
|
|
assertEquals(locs.length, 1);
|
|
|
assertEquals(locs[0].getStorageIDs().length, replicationFactor);
|
|
|
|
|
|
// get blocks of size 1 from dataNodes[0]
|
|
|
- locs = namenode.getBlocks(dataNodes[0], 1, 1).getBlocks();
|
|
|
+ locs = namenode.getBlocks(dataNodes[0], 1, 1, 0).getBlocks();
|
|
|
assertEquals(locs.length, 1);
|
|
|
assertEquals(locs[0].getStorageIDs().length, replicationFactor);
|
|
|
|
|
@@ -282,7 +282,7 @@ public class TestGetBlocks {
|
|
|
|
|
|
// Namenode should refuse to provide block locations to the balancer
|
|
|
// while in safemode.
|
|
|
- locs = namenode.getBlocks(dataNodes[0], fileLen, 0).getBlocks();
|
|
|
+ locs = namenode.getBlocks(dataNodes[0], fileLen, 0, 0).getBlocks();
|
|
|
assertEquals(blkLocsSize, locs.length);
|
|
|
assertFalse(fs.isInSafeMode());
|
|
|
LOG.info("Entering safe mode");
|
|
@@ -309,7 +309,7 @@ public class TestGetBlocks {
|
|
|
|
|
|
// Namenode should refuse should fail
|
|
|
LambdaTestUtils.intercept(exClass,
|
|
|
- msg, () -> namenode.getBlocks(datanode, size, minBlkSize));
|
|
|
+ msg, () -> namenode.getBlocks(datanode, size, minBlkSize, 0));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -396,4 +396,76 @@ public class TestGetBlocks {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-}
|
|
|
+ private boolean belongToFile(BlockWithLocations blockWithLocations,
|
|
|
+ List<LocatedBlock> blocks) {
|
|
|
+ for(LocatedBlock block : blocks) {
|
|
|
+ if (block.getBlock().getLocalBlock().equals(
|
|
|
+ blockWithLocations.getBlock())) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * test GetBlocks with dfs.namenode.hot.block.interval.
|
|
|
+ * Balancer prefer to get blocks which are belong to the cold files
|
|
|
+ * created before this time period.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testGetBlocksWithHotBlockTimeInterval() throws Exception {
|
|
|
+ final Configuration conf = new HdfsConfiguration();
|
|
|
+ final short repFactor = (short) 1;
|
|
|
+ final int blockNum = 2;
|
|
|
+ final int fileLen = BLOCK_SIZE * blockNum;
|
|
|
+ final long hotInterval = 2000;
|
|
|
+
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).
|
|
|
+ numDataNodes(repFactor).build();
|
|
|
+ try {
|
|
|
+ cluster.waitActive();
|
|
|
+ FileSystem fs = cluster.getFileSystem();
|
|
|
+ final DFSClient dfsclient = ((DistributedFileSystem) fs).getClient();
|
|
|
+
|
|
|
+ String fileOld = "/f.old";
|
|
|
+ DFSTestUtil.createFile(fs, new Path(fileOld), fileLen, repFactor, 0);
|
|
|
+
|
|
|
+ List<LocatedBlock> locatedBlocksOld = dfsclient.getNamenode().
|
|
|
+ getBlockLocations(fileOld, 0, fileLen).getLocatedBlocks();
|
|
|
+ DatanodeInfo[] dataNodes = locatedBlocksOld.get(0).getLocations();
|
|
|
+
|
|
|
+ InetSocketAddress addr = new InetSocketAddress("localhost",
|
|
|
+ cluster.getNameNodePort());
|
|
|
+ NamenodeProtocol namenode = NameNodeProxies.createProxy(conf,
|
|
|
+ DFSUtilClient.getNNUri(addr), NamenodeProtocol.class).getProxy();
|
|
|
+
|
|
|
+ // make the file as old.
|
|
|
+ dfsclient.getNamenode().setTimes(fileOld, 0, 0);
|
|
|
+
|
|
|
+ String fileNew = "/f.new";
|
|
|
+ DFSTestUtil.createFile(fs, new Path(fileNew), fileLen, repFactor, 0);
|
|
|
+ List<LocatedBlock> locatedBlocksNew = dfsclient.getNamenode()
|
|
|
+ .getBlockLocations(fileNew, 0, fileLen).getLocatedBlocks();
|
|
|
+
|
|
|
+ BlockWithLocations[] locsAll = namenode.getBlocks(
|
|
|
+ dataNodes[0], fileLen*2, 0, hotInterval).getBlocks();
|
|
|
+ assertEquals(locsAll.length, 4);
|
|
|
+
|
|
|
+ for(int i = 0; i < blockNum; i++) {
|
|
|
+ assertTrue(belongToFile(locsAll[i], locatedBlocksOld));
|
|
|
+ }
|
|
|
+ for(int i = blockNum; i < blockNum*2; i++) {
|
|
|
+ assertTrue(belongToFile(locsAll[i], locatedBlocksNew));
|
|
|
+ }
|
|
|
+
|
|
|
+ BlockWithLocations[] locs2 = namenode.getBlocks(
|
|
|
+ dataNodes[0], fileLen*2, 0, hotInterval).getBlocks();
|
|
|
+ for(int i = 0; i < 2; i++) {
|
|
|
+ assertTrue(belongToFile(locs2[i], locatedBlocksOld));
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+}
|