|
@@ -19,25 +19,36 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
|
+import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
|
+import org.apache.hadoop.hdfs.net.DFSNetworkTopology;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
|
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
|
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
import org.apache.hadoop.net.StaticMapping;
|
|
import org.apache.hadoop.net.StaticMapping;
|
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
|
|
+import org.junit.Assert;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.util.*;
|
|
import java.util.*;
|
|
|
|
+import java.util.function.Supplier;
|
|
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
@@ -168,6 +179,108 @@ public class TestBlockPlacementPolicyRackFaultTolerant {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Verify decommission a dn which is an only node in its rack.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testPlacementWithOnlyOneNodeInRackDecommission() throws Exception {
|
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
|
+ final String[] racks = {"/RACK0", "/RACK0", "/RACK2", "/RACK3", "/RACK4", "/RACK5", "/RACK2"};
|
|
|
|
+ final String[] hosts = {"/host0", "/host1", "/host2", "/host3", "/host4", "/host5", "/host6"};
|
|
|
|
+
|
|
|
|
+ // enables DFSNetworkTopology
|
|
|
|
+ conf.setClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
|
|
|
|
+ BlockPlacementPolicyRackFaultTolerant.class,
|
|
|
|
+ BlockPlacementPolicy.class);
|
|
|
|
+ conf.setBoolean(DFSConfigKeys.DFS_USE_DFS_NETWORK_TOPOLOGY_KEY, true);
|
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
|
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY,
|
|
|
|
+ DEFAULT_BLOCK_SIZE / 2);
|
|
|
|
+
|
|
|
|
+ if (cluster != null) {
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(7).racks(racks)
|
|
|
|
+ .hosts(hosts).build();
|
|
|
|
+ cluster.waitActive();
|
|
|
|
+ nameNodeRpc = cluster.getNameNodeRpc();
|
|
|
|
+ namesystem = cluster.getNamesystem();
|
|
|
|
+ DistributedFileSystem fs = cluster.getFileSystem();
|
|
|
|
+ fs.enableErasureCodingPolicy("RS-3-2-1024k");
|
|
|
|
+ fs.setErasureCodingPolicy(new Path("/"), "RS-3-2-1024k");
|
|
|
|
+
|
|
|
|
+ final BlockManager bm = cluster.getNamesystem().getBlockManager();
|
|
|
|
+ final DatanodeManager dm = bm.getDatanodeManager();
|
|
|
|
+ assertTrue(dm.getNetworkTopology() instanceof DFSNetworkTopology);
|
|
|
|
+
|
|
|
|
+ String clientMachine = "/host4";
|
|
|
|
+ String clientRack = "/RACK4";
|
|
|
|
+ String src = "/test";
|
|
|
|
+
|
|
|
|
+ final DatanodeManager dnm = namesystem.getBlockManager().getDatanodeManager();
|
|
|
|
+ DatanodeDescriptor dnd4 = dnm.getDatanode(cluster.getDataNodes().get(4).getDatanodeId());
|
|
|
|
+ assertEquals(dnd4.getNetworkLocation(), clientRack);
|
|
|
|
+ dnm.getDatanodeAdminManager().startDecommission(dnd4);
|
|
|
|
+ short replication = 5;
|
|
|
|
+ short additionalReplication = 1;
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ // Create the file with client machine
|
|
|
|
+ HdfsFileStatus fileStatus = namesystem.startFile(src, perm,
|
|
|
|
+ clientMachine, clientMachine, EnumSet.of(CreateFlag.CREATE), true,
|
|
|
|
+ replication, DEFAULT_BLOCK_SIZE * 1024 * 10, null, null, false);
|
|
|
|
+
|
|
|
|
+ //test chooseTarget for new file
|
|
|
|
+ LocatedBlock locatedBlock = nameNodeRpc.addBlock(src, clientMachine,
|
|
|
|
+ null, null, fileStatus.getFileId(), null, null);
|
|
|
|
+ HashMap<String, Integer> racksCount = new HashMap<String, Integer>();
|
|
|
|
+ doTestLocatedBlockRacks(racksCount, replication, 4, locatedBlock);
|
|
|
|
+
|
|
|
|
+ //test chooseTarget for existing file.
|
|
|
|
+ LocatedBlock additionalLocatedBlock =
|
|
|
|
+ nameNodeRpc.getAdditionalDatanode(src, fileStatus.getFileId(),
|
|
|
|
+ locatedBlock.getBlock(), locatedBlock.getLocations(),
|
|
|
|
+ locatedBlock.getStorageIDs(), DatanodeInfo.EMPTY_ARRAY,
|
|
|
|
+ additionalReplication, clientMachine);
|
|
|
|
+
|
|
|
|
+ racksCount.clear();
|
|
|
|
+ doTestLocatedBlockRacks(racksCount, additionalReplication + replication,
|
|
|
|
+ 4, additionalLocatedBlock);
|
|
|
|
+ assertEquals(racksCount.get("/RACK0"), (Integer)2);
|
|
|
|
+ assertEquals(racksCount.get("/RACK2"), (Integer)2);
|
|
|
|
+ } finally {
|
|
|
|
+ dnm.getDatanodeAdminManager().stopDecommission(dnd4);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ //test if decommission succeeded
|
|
|
|
+ DatanodeDescriptor dnd3 = dnm.getDatanode(cluster.getDataNodes().get(3).getDatanodeId());
|
|
|
|
+ cluster.getNamesystem().writeLock();
|
|
|
|
+ try {
|
|
|
|
+ dm.getDatanodeAdminManager().startDecommission(dnd3);
|
|
|
|
+ } finally {
|
|
|
|
+ cluster.getNamesystem().writeUnlock();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // make sure the decommission finishes and the block in on 4 racks
|
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Boolean get() {
|
|
|
|
+ return dnd3.isDecommissioned();
|
|
|
|
+ }
|
|
|
|
+ }, 1000, 10 * 1000);
|
|
|
|
+
|
|
|
|
+ LocatedBlocks locatedBlocks =
|
|
|
|
+ cluster.getFileSystem().getClient().getLocatedBlocks(
|
|
|
|
+ src, 0, DEFAULT_BLOCK_SIZE);
|
|
|
|
+ assertEquals(4, bm.getDatanodeManager().
|
|
|
|
+ getNetworkTopology().getNumOfNonEmptyRacks());
|
|
|
|
+ for (LocatedBlock block : locatedBlocks.getLocatedBlocks()) {
|
|
|
|
+ BlockPlacementStatus status = bm.getStriptedBlockPlacementPolicy()
|
|
|
|
+ .verifyBlockPlacement(block.getLocations(), 5);
|
|
|
|
+ Assert.assertTrue(status.isPlacementPolicySatisfied());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
private void shuffle(DatanodeInfo[] locs, String[] storageIDs) {
|
|
private void shuffle(DatanodeInfo[] locs, String[] storageIDs) {
|
|
int length = locs.length;
|
|
int length = locs.length;
|
|
Object[][] pairs = new Object[length][];
|
|
Object[][] pairs = new Object[length][];
|
|
@@ -199,6 +312,17 @@ public class TestBlockPlacementPolicyRackFaultTolerant {
|
|
assertTrue(maxCount - minCount <= 1);
|
|
assertTrue(maxCount - minCount <= 1);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private void doTestLocatedBlockRacks(HashMap<String, Integer> racksCount, int replication,
|
|
|
|
+ int validracknum, LocatedBlock locatedBlock) {
|
|
|
|
+ assertEquals(replication, locatedBlock.getLocations().length);
|
|
|
|
+
|
|
|
|
+ for (DatanodeInfo node :
|
|
|
|
+ locatedBlock.getLocations()) {
|
|
|
|
+ addToRacksCount(node.getNetworkLocation(), racksCount);
|
|
|
|
+ }
|
|
|
|
+ assertEquals(validracknum, racksCount.size());
|
|
|
|
+ }
|
|
|
|
+
|
|
private void addToRacksCount(String rack, HashMap<String, Integer> racksCount) {
|
|
private void addToRacksCount(String rack, HashMap<String, Integer> racksCount) {
|
|
Integer count = racksCount.get(rack);
|
|
Integer count = racksCount.get(rack);
|
|
if (count == null) {
|
|
if (count == null) {
|