|
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
|
|
|
import static org.junit.Assert.*;
|
|
|
|
|
|
import java.net.InetSocketAddress;
|
|
|
+import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.Iterator;
|
|
@@ -36,6 +37,7 @@ import org.apache.hadoop.fs.LocatedFileStatus;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.SafeModeAction;
|
|
|
import org.apache.hadoop.fs.RemoteIterator;
|
|
|
+import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
@@ -239,26 +241,29 @@ 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, 0).getBlocks();
|
|
|
+ locs = namenode.getBlocks(dataNodes[0], fileLen, 0, 0,
|
|
|
+ null).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, 0).getBlocks();
|
|
|
+ locs = namenode.getBlocks(dataNodes[0], fileLen, blkSize, 0,
|
|
|
+ null).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, 0).getBlocks();
|
|
|
+ blkSize, 0, null).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, 0).getBlocks();
|
|
|
+ locs = namenode.getBlocks(dataNodes[0], 1, 1, 0,
|
|
|
+ null).getBlocks();
|
|
|
assertEquals(locs.length, 1);
|
|
|
assertEquals(locs[0].getStorageIDs().length, replicationFactor);
|
|
|
|
|
@@ -283,7 +288,8 @@ public class TestGetBlocks {
|
|
|
|
|
|
// Namenode should refuse to provide block locations to the balancer
|
|
|
// while in safemode.
|
|
|
- locs = namenode.getBlocks(dataNodes[0], fileLen, 0, 0).getBlocks();
|
|
|
+ locs = namenode.getBlocks(dataNodes[0], fileLen, 0, 0,
|
|
|
+ null).getBlocks();
|
|
|
assertEquals(blkLocsSize, locs.length);
|
|
|
assertFalse(fs.isInSafeMode());
|
|
|
LOG.info("Entering safe mode");
|
|
@@ -310,7 +316,8 @@ public class TestGetBlocks {
|
|
|
|
|
|
// Namenode should refuse should fail
|
|
|
LambdaTestUtils.intercept(exClass,
|
|
|
- msg, () -> namenode.getBlocks(datanode, size, minBlkSize, 0));
|
|
|
+ msg, () -> namenode.getBlocks(datanode, size, minBlkSize, 0,
|
|
|
+ null));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -450,7 +457,7 @@ public class TestGetBlocks {
|
|
|
.getBlockLocations(fileNew, 0, fileLen).getLocatedBlocks();
|
|
|
|
|
|
BlockWithLocations[] locsAll = namenode.getBlocks(
|
|
|
- dataNodes[0], fileLen*2, 0, hotInterval).getBlocks();
|
|
|
+ dataNodes[0], fileLen*2, 0, hotInterval, null).getBlocks();
|
|
|
assertEquals(locsAll.length, 4);
|
|
|
|
|
|
for(int i = 0; i < blockNum; i++) {
|
|
@@ -461,7 +468,7 @@ public class TestGetBlocks {
|
|
|
}
|
|
|
|
|
|
BlockWithLocations[] locs2 = namenode.getBlocks(
|
|
|
- dataNodes[0], fileLen*2, 0, hotInterval).getBlocks();
|
|
|
+ dataNodes[0], fileLen*2, 0, hotInterval, null).getBlocks();
|
|
|
for(int i = 0; i < 2; i++) {
|
|
|
assertTrue(belongToFile(locs2[i], locatedBlocksOld));
|
|
|
}
|
|
@@ -508,7 +515,7 @@ public class TestGetBlocks {
|
|
|
|
|
|
// check blocks count equals to blockNum
|
|
|
BlockWithLocations[] blocks = namenode.getBlocks(
|
|
|
- dataNodes[0], fileLen*2, 0, 0).getBlocks();
|
|
|
+ dataNodes[0], fileLen*2, 0, 0, null).getBlocks();
|
|
|
assertEquals(blockNum, blocks.length);
|
|
|
|
|
|
// calculate the block count on storage[0]
|
|
@@ -524,13 +531,94 @@ public class TestGetBlocks {
|
|
|
// set storage[0] stale
|
|
|
storageInfos[0].setBlockContentsStale(true);
|
|
|
blocks = namenode.getBlocks(
|
|
|
- dataNodes[0], fileLen*2, 0, 0).getBlocks();
|
|
|
+ dataNodes[0], fileLen*2, 0, 0, null).getBlocks();
|
|
|
assertEquals(blockNum - count, blocks.length);
|
|
|
|
|
|
// set all storage stale
|
|
|
bm0.getDatanodeManager().markAllDatanodesStale();
|
|
|
blocks = namenode.getBlocks(
|
|
|
- dataNodes[0], fileLen*2, 0, 0).getBlocks();
|
|
|
+ dataNodes[0], fileLen*2, 0, 0, null).getBlocks();
|
|
|
assertEquals(0, blocks.length);
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testChooseSpecifyStorageType() throws Exception {
|
|
|
+ final short repFactor = (short) 1;
|
|
|
+ final int fileLen = BLOCK_SIZE;
|
|
|
+ final Configuration conf = new HdfsConfiguration();
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
+
|
|
|
+ try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
|
|
|
+ .storageTypes(new StorageType[] {StorageType.DISK, StorageType.SSD}).
|
|
|
+ storagesPerDatanode(2).build()) {
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ // Get storage info.
|
|
|
+ ClientProtocol client = NameNodeProxies.createProxy(conf,
|
|
|
+ cluster.getFileSystem(0).getUri(),
|
|
|
+ ClientProtocol.class).getProxy();
|
|
|
+ DatanodeInfo[] dataNodes = client.getDatanodeReport(DatanodeReportType.ALL);
|
|
|
+ BlockManager bm0 = cluster.getNamesystem(0).getBlockManager();
|
|
|
+ DatanodeStorageInfo[] storageInfos = bm0.getDatanodeManager()
|
|
|
+ .getDatanode(dataNodes[0].getDatanodeUuid()).getStorageInfos();
|
|
|
+ assert Arrays.stream(storageInfos)
|
|
|
+ .anyMatch(datanodeStorageInfo -> {
|
|
|
+ String storageTypeName = datanodeStorageInfo.getStorageType().name();
|
|
|
+ return storageTypeName.equals("SSD") || storageTypeName.equals("DISK");
|
|
|
+ }) : "No 'SSD' or 'DISK' storage types found.";
|
|
|
+
|
|
|
+ // Create hdfs file.
|
|
|
+ Path ssdDir = new Path("/testChooseSSD");
|
|
|
+ DistributedFileSystem fs = cluster.getFileSystem();
|
|
|
+ Path ssdFile = new Path(ssdDir, "file");
|
|
|
+ fs.mkdirs(ssdDir);
|
|
|
+ fs.setStoragePolicy(ssdDir, "ALL_SSD");
|
|
|
+ DFSTestUtil.createFile(fs, ssdFile, false, 1024, fileLen,
|
|
|
+ BLOCK_SIZE, repFactor, 0, true);
|
|
|
+ DFSTestUtil.waitReplication(fs, ssdFile, repFactor);
|
|
|
+ BlockLocation[] locations = fs.getClient()
|
|
|
+ .getBlockLocations(ssdFile.toUri().getPath(), 0, Long.MAX_VALUE);
|
|
|
+ assertEquals(1, locations.length);
|
|
|
+ assertEquals("SSD", locations[0].getStorageTypes()[0].name());
|
|
|
+
|
|
|
+ Path diskDir = new Path("/testChooseDisk");
|
|
|
+ fs = cluster.getFileSystem();
|
|
|
+ Path diskFile = new Path(diskDir, "file");
|
|
|
+ fs.mkdirs(diskDir);
|
|
|
+ fs.setStoragePolicy(diskDir, "HOT");
|
|
|
+ DFSTestUtil.createFile(fs, diskFile, false, 1024, fileLen,
|
|
|
+ BLOCK_SIZE, repFactor, 0, true);
|
|
|
+ DFSTestUtil.waitReplication(fs, diskFile, repFactor);
|
|
|
+ locations = fs.getClient()
|
|
|
+ .getBlockLocations(diskFile.toUri().getPath(), 0, Long.MAX_VALUE);
|
|
|
+ assertEquals(1, locations.length);
|
|
|
+ assertEquals("DISK", locations[0].getStorageTypes()[0].name());
|
|
|
+
|
|
|
+ InetSocketAddress addr = new InetSocketAddress("localhost",
|
|
|
+ cluster.getNameNodePort());
|
|
|
+ NamenodeProtocol namenode = NameNodeProxies.createProxy(conf,
|
|
|
+ DFSUtilClient.getNNUri(addr), NamenodeProtocol.class).getProxy();
|
|
|
+
|
|
|
+ // Check blocks count equals to blockNum.
|
|
|
+ // If StorageType is not specified will get all blocks.
|
|
|
+ BlockWithLocations[] blocks = namenode.getBlocks(
|
|
|
+ dataNodes[0], fileLen * 2, 0, 0,
|
|
|
+ null).getBlocks();
|
|
|
+ assertEquals(2, blocks.length);
|
|
|
+
|
|
|
+ // Check the count of blocks with a StorageType of DISK.
|
|
|
+ blocks = namenode.getBlocks(
|
|
|
+ dataNodes[0], fileLen * 2, 0, 0,
|
|
|
+ StorageType.DISK).getBlocks();
|
|
|
+ assertEquals(1, blocks.length);
|
|
|
+ assertEquals("DISK", blocks[0].getStorageTypes()[0].name());
|
|
|
+
|
|
|
+ // Check the count of blocks with a StorageType of SSD.
|
|
|
+ blocks = namenode.getBlocks(
|
|
|
+ dataNodes[0], fileLen * 2, 0, 0,
|
|
|
+ StorageType.SSD).getBlocks();
|
|
|
+ assertEquals(1, blocks.length);
|
|
|
+ assertEquals("SSD", blocks[0].getStorageTypes()[0].name());
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|