|
@@ -474,12 +474,12 @@ public class TestNameNodeProvidedImplementation {
|
|
|
}
|
|
|
|
|
|
private DatanodeInfo[] getAndCheckBlockLocations(DFSClient client,
|
|
|
- String filename, int expectedLocations) throws IOException {
|
|
|
- LocatedBlocks locatedBlocks = client.getLocatedBlocks(
|
|
|
- filename, 0, baseFileLen);
|
|
|
- //given the start and length in the above call,
|
|
|
- //only one LocatedBlock in LocatedBlocks
|
|
|
- assertEquals(1, locatedBlocks.getLocatedBlocks().size());
|
|
|
+ String filename, long fileLen, long expectedBlocks, int expectedLocations)
|
|
|
+ throws IOException {
|
|
|
+ LocatedBlocks locatedBlocks = client.getLocatedBlocks(filename, 0, fileLen);
|
|
|
+ // given the start and length in the above call,
|
|
|
+ // only one LocatedBlock in LocatedBlocks
|
|
|
+ assertEquals(expectedBlocks, locatedBlocks.getLocatedBlocks().size());
|
|
|
LocatedBlock locatedBlock = locatedBlocks.getLocatedBlocks().get(0);
|
|
|
assertEquals(expectedLocations, locatedBlock.getLocations().length);
|
|
|
return locatedBlock.getLocations();
|
|
@@ -513,17 +513,20 @@ public class TestNameNodeProvidedImplementation {
|
|
|
file, newReplication, 10000);
|
|
|
DFSClient client = new DFSClient(new InetSocketAddress("localhost",
|
|
|
cluster.getNameNodePort()), cluster.getConfiguration(0));
|
|
|
- getAndCheckBlockLocations(client, filename, newReplication);
|
|
|
+ getAndCheckBlockLocations(client, filename, baseFileLen, 1, newReplication);
|
|
|
|
|
|
// set the replication back to 1
|
|
|
newReplication = 1;
|
|
|
LOG.info("Setting replication of file {} back to {}",
|
|
|
filename, newReplication);
|
|
|
fs.setReplication(file, newReplication);
|
|
|
+ // defaultReplication number of replicas should be returned
|
|
|
+ int defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
|
|
|
+ DFSConfigKeys.DFS_REPLICATION_DEFAULT);
|
|
|
DFSTestUtil.waitForReplication((DistributedFileSystem) fs,
|
|
|
- file, newReplication, 10000);
|
|
|
- // the only replica left should be the PROVIDED datanode
|
|
|
- getAndCheckBlockLocations(client, filename, newReplication);
|
|
|
+ file, (short) defaultReplication, 10000);
|
|
|
+ getAndCheckBlockLocations(client, filename, baseFileLen, 1,
|
|
|
+ defaultReplication);
|
|
|
}
|
|
|
|
|
|
@Test(timeout=30000)
|
|
@@ -545,8 +548,9 @@ public class TestNameNodeProvidedImplementation {
|
|
|
|
|
|
if (numFiles >= 1) {
|
|
|
String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
|
|
|
-
|
|
|
- DatanodeInfo[] dnInfos = getAndCheckBlockLocations(client, filename, 1);
|
|
|
+ // 2 locations returned as there are 2 PROVIDED datanodes
|
|
|
+ DatanodeInfo[] dnInfos =
|
|
|
+ getAndCheckBlockLocations(client, filename, baseFileLen, 1, 2);
|
|
|
//the location should be one of the provided DNs available
|
|
|
assertTrue(
|
|
|
dnInfos[0].getDatanodeUuid().equals(
|
|
@@ -564,7 +568,7 @@ public class TestNameNodeProvidedImplementation {
|
|
|
providedDatanode1.getDatanodeId().getXferAddr());
|
|
|
|
|
|
//should find the block on the 2nd provided datanode
|
|
|
- dnInfos = getAndCheckBlockLocations(client, filename, 1);
|
|
|
+ dnInfos = getAndCheckBlockLocations(client, filename, baseFileLen, 1, 1);
|
|
|
assertEquals(providedDatanode2.getDatanodeUuid(),
|
|
|
dnInfos[0].getDatanodeUuid());
|
|
|
|
|
@@ -575,14 +579,14 @@ public class TestNameNodeProvidedImplementation {
|
|
|
BlockManagerTestUtil.noticeDeadDatanode(
|
|
|
cluster.getNameNode(),
|
|
|
providedDatanode2.getDatanodeId().getXferAddr());
|
|
|
- getAndCheckBlockLocations(client, filename, 0);
|
|
|
+ getAndCheckBlockLocations(client, filename, baseFileLen, 1, 0);
|
|
|
|
|
|
//restart the provided datanode
|
|
|
cluster.restartDataNode(providedDNProperties1, true);
|
|
|
cluster.waitActive();
|
|
|
|
|
|
//should find the block on the 1st provided datanode now
|
|
|
- dnInfos = getAndCheckBlockLocations(client, filename, 1);
|
|
|
+ dnInfos = getAndCheckBlockLocations(client, filename, baseFileLen, 1, 1);
|
|
|
//not comparing UUIDs as the datanode can now have a different one.
|
|
|
assertEquals(providedDatanode1.getDatanodeId().getXferAddr(),
|
|
|
dnInfos[0].getXferAddr());
|
|
@@ -593,20 +597,18 @@ public class TestNameNodeProvidedImplementation {
|
|
|
public void testTransientDeadDatanodes() throws Exception {
|
|
|
createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
|
|
|
FixedBlockResolver.class);
|
|
|
- // 2 Datanodes, 1 PROVIDED and other DISK
|
|
|
- startCluster(NNDIRPATH, 2, null,
|
|
|
+ // 3 Datanodes, 2 PROVIDED and other DISK
|
|
|
+ startCluster(NNDIRPATH, 3, null,
|
|
|
new StorageType[][] {
|
|
|
+ {StorageType.PROVIDED, StorageType.DISK},
|
|
|
{StorageType.PROVIDED, StorageType.DISK},
|
|
|
{StorageType.DISK}},
|
|
|
false);
|
|
|
|
|
|
DataNode providedDatanode = cluster.getDataNodes().get(0);
|
|
|
-
|
|
|
- DFSClient client = new DFSClient(new InetSocketAddress("localhost",
|
|
|
- cluster.getNameNodePort()), cluster.getConfiguration(0));
|
|
|
-
|
|
|
for (int i= 0; i < numFiles; i++) {
|
|
|
- verifyFileLocation(i);
|
|
|
+ // expect to have 2 locations as we have 2 provided Datanodes.
|
|
|
+ verifyFileLocation(i, 2);
|
|
|
// NameNode thinks the datanode is down
|
|
|
BlockManagerTestUtil.noticeDeadDatanode(
|
|
|
cluster.getNameNode(),
|
|
@@ -614,7 +616,7 @@ public class TestNameNodeProvidedImplementation {
|
|
|
cluster.waitActive();
|
|
|
cluster.triggerHeartbeats();
|
|
|
Thread.sleep(1000);
|
|
|
- verifyFileLocation(i);
|
|
|
+ verifyFileLocation(i, 2);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -622,17 +624,18 @@ public class TestNameNodeProvidedImplementation {
|
|
|
public void testNamenodeRestart() throws Exception {
|
|
|
createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
|
|
|
FixedBlockResolver.class);
|
|
|
- // 2 Datanodes, 1 PROVIDED and other DISK
|
|
|
- startCluster(NNDIRPATH, 2, null,
|
|
|
+ // 3 Datanodes, 2 PROVIDED and other DISK
|
|
|
+ startCluster(NNDIRPATH, 3, null,
|
|
|
new StorageType[][] {
|
|
|
+ {StorageType.PROVIDED, StorageType.DISK},
|
|
|
{StorageType.PROVIDED, StorageType.DISK},
|
|
|
{StorageType.DISK}},
|
|
|
false);
|
|
|
|
|
|
- verifyFileLocation(numFiles - 1);
|
|
|
+ verifyFileLocation(numFiles - 1, 2);
|
|
|
cluster.restartNameNodes();
|
|
|
cluster.waitActive();
|
|
|
- verifyFileLocation(numFiles - 1);
|
|
|
+ verifyFileLocation(numFiles - 1, 2);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -640,18 +643,21 @@ public class TestNameNodeProvidedImplementation {
|
|
|
* @param fileIndex the index of the file to verify.
|
|
|
* @throws Exception
|
|
|
*/
|
|
|
- private void verifyFileLocation(int fileIndex)
|
|
|
+ private void verifyFileLocation(int fileIndex, int replication)
|
|
|
throws Exception {
|
|
|
- DataNode providedDatanode = cluster.getDataNodes().get(0);
|
|
|
DFSClient client = new DFSClient(
|
|
|
new InetSocketAddress("localhost", cluster.getNameNodePort()),
|
|
|
cluster.getConfiguration(0));
|
|
|
- if (fileIndex <= numFiles && fileIndex >= 0) {
|
|
|
- String filename = "/" + filePrefix + fileIndex + fileSuffix;
|
|
|
- DatanodeInfo[] dnInfos = getAndCheckBlockLocations(client, filename, 1);
|
|
|
- // location should be the provided DN
|
|
|
- assertEquals(providedDatanode.getDatanodeUuid(),
|
|
|
- dnInfos[0].getDatanodeUuid());
|
|
|
+ if (fileIndex < numFiles && fileIndex >= 0) {
|
|
|
+ String filename = filePrefix + fileIndex + fileSuffix;
|
|
|
+ File file = new File(new Path(NAMEPATH, filename).toUri());
|
|
|
+ long fileLen = file.length();
|
|
|
+ long blockSize = conf.getLong(FixedBlockResolver.BLOCKSIZE,
|
|
|
+ FixedBlockResolver.BLOCKSIZE_DEFAULT);
|
|
|
+ long numLocatedBlocks =
|
|
|
+ fileLen == 0 ? 1 : (long) Math.ceil(fileLen * 1.0 / blockSize);
|
|
|
+ getAndCheckBlockLocations(client, "/" + filename, fileLen,
|
|
|
+ numLocatedBlocks, replication);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -669,4 +675,55 @@ public class TestNameNodeProvidedImplementation {
|
|
|
NameNode nn = cluster.getNameNode();
|
|
|
assertEquals(clusterID, nn.getNamesystem().getClusterId());
|
|
|
}
|
|
|
+
|
|
|
+ @Test(timeout=30000)
|
|
|
+ public void testNumberOfProvidedLocations() throws Exception {
|
|
|
+ // set default replication to 4
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 4);
|
|
|
+ createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
|
|
|
+ FixedBlockResolver.class);
|
|
|
+ // start with 4 PROVIDED location
|
|
|
+ startCluster(NNDIRPATH, 4,
|
|
|
+ new StorageType[]{
|
|
|
+ StorageType.PROVIDED, StorageType.DISK},
|
|
|
+ null,
|
|
|
+ false);
|
|
|
+ int expectedLocations = 4;
|
|
|
+ for (int i = 0; i < numFiles; i++) {
|
|
|
+ verifyFileLocation(i, expectedLocations);
|
|
|
+ }
|
|
|
+ // stop 2 datanodes, one after the other and verify number of locations.
|
|
|
+ for (int i = 1; i <= 2; i++) {
|
|
|
+ DataNode dn = cluster.getDataNodes().get(0);
|
|
|
+ cluster.stopDataNode(0);
|
|
|
+ // make NameNode detect that datanode is down
|
|
|
+ BlockManagerTestUtil.noticeDeadDatanode(cluster.getNameNode(),
|
|
|
+ dn.getDatanodeId().getXferAddr());
|
|
|
+
|
|
|
+ expectedLocations = 4 - i;
|
|
|
+ for (int j = 0; j < numFiles; j++) {
|
|
|
+ verifyFileLocation(j, expectedLocations);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test(timeout=30000)
|
|
|
+ public void testNumberOfProvidedLocationsManyBlocks() throws Exception {
|
|
|
+ // increase number of blocks per file to at least 10 blocks per file
|
|
|
+ conf.setLong(FixedBlockResolver.BLOCKSIZE, baseFileLen/10);
|
|
|
+ // set default replication to 4
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 4);
|
|
|
+ createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
|
|
|
+ FixedBlockResolver.class);
|
|
|
+ // start with 4 PROVIDED location
|
|
|
+ startCluster(NNDIRPATH, 4,
|
|
|
+ new StorageType[]{
|
|
|
+ StorageType.PROVIDED, StorageType.DISK},
|
|
|
+ null,
|
|
|
+ false);
|
|
|
+ int expectedLocations = 4;
|
|
|
+ for (int i = 0; i < numFiles; i++) {
|
|
|
+ verifyFileLocation(i, expectedLocations);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|