|
@@ -48,6 +48,7 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
+import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.DFSClient;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
@@ -1309,6 +1310,68 @@ public class TestBalancer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Test special case. Two replicas belong to same block should not in same node.
|
|
|
+ * We have 2 nodes.
|
|
|
+ * We have a block in (DN0,SSD) and (DN1,DISK).
|
|
|
+ * Replica in (DN0,SSD) should not be moved to (DN1,SSD).
|
|
|
+ * Otherwise DN1 has 2 replicas.
|
|
|
+ */
|
|
|
+ @Test(timeout=100000)
|
|
|
+ public void testTwoReplicaShouldNotInSameDN() throws Exception {
|
|
|
+ final Configuration conf = new HdfsConfiguration();
|
|
|
+
|
|
|
+ int blockSize = 5 * 1024 * 1024 ;
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L);
|
|
|
+
|
|
|
+ int numOfDatanodes =2;
|
|
|
+ final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .numDataNodes(2)
|
|
|
+ .racks(new String[]{"/default/rack0", "/default/rack0"})
|
|
|
+ .storagesPerDatanode(2)
|
|
|
+ .storageTypes(new StorageType[][]{
|
|
|
+ {StorageType.SSD, StorageType.DISK},
|
|
|
+ {StorageType.SSD, StorageType.DISK}})
|
|
|
+ .storageCapacities(new long[][]{
|
|
|
+ {100 * blockSize, 20 * blockSize},
|
|
|
+ {20 * blockSize, 100 * blockSize}})
|
|
|
+ .build();
|
|
|
+
|
|
|
+ try {
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ //set "/bar" directory with ONE_SSD storage policy.
|
|
|
+ DistributedFileSystem fs = cluster.getFileSystem();
|
|
|
+ Path barDir = new Path("/bar");
|
|
|
+ fs.mkdir(barDir,new FsPermission((short)777));
|
|
|
+ fs.setStoragePolicy(barDir, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
|
|
|
+
|
|
|
+ // Insert 30 blocks. So (DN0,SSD) and (DN1,DISK) are about half full,
|
|
|
+ // and (DN0,SSD) and (DN1,DISK) are about 15% full.
|
|
|
+ long fileLen = 30 * blockSize;
|
|
|
+ // fooFile has ONE_SSD policy. So
|
|
|
+ // (DN0,SSD) and (DN1,DISK) have 2 replicas belong to same block.
|
|
|
+ // (DN0,DISK) and (DN1,SSD) have 2 replicas belong to same block.
|
|
|
+ Path fooFile = new Path(barDir, "foo");
|
|
|
+ createFile(cluster, fooFile, fileLen, (short) numOfDatanodes, 0);
|
|
|
+ // update space info
|
|
|
+ cluster.triggerHeartbeats();
|
|
|
+
|
|
|
+ Balancer.Parameters p = Balancer.Parameters.DEFAULT;
|
|
|
+ Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
|
|
|
+ final int r = Balancer.run(namenodes, p, conf);
|
|
|
+
|
|
|
+ // Replica in (DN0,SSD) was not moved to (DN1,SSD), because (DN1,DISK)
|
|
|
+ // already has one. Otherwise DN1 will have 2 replicas.
|
|
|
+ // For same reason, no replicas were moved.
|
|
|
+ assertEquals(ExitStatus.NO_MOVE_PROGRESS.getExitCode(), r);
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
/**
|
|
|
* @param args
|
|
|
*/
|