|
@@ -20,17 +20,21 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.List;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
|
|
|
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
@@ -56,14 +60,12 @@ public class TestHeartbeatHandling {
|
|
|
final HeartbeatManager hm = namesystem.getBlockManager(
|
|
|
).getDatanodeManager().getHeartbeatManager();
|
|
|
final String poolId = namesystem.getBlockPoolId();
|
|
|
- final DatanodeRegistration nodeReg =
|
|
|
+ final DatanodeRegistration nodeReg =
|
|
|
DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(0), poolId);
|
|
|
-
|
|
|
-
|
|
|
final DatanodeDescriptor dd = NameNodeAdapter.getDatanode(namesystem, nodeReg);
|
|
|
-
|
|
|
+
|
|
|
final int REMAINING_BLOCKS = 1;
|
|
|
- final int MAX_REPLICATE_LIMIT =
|
|
|
+ final int MAX_REPLICATE_LIMIT =
|
|
|
conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 2);
|
|
|
final int MAX_INVALIDATE_LIMIT = DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT;
|
|
|
final int MAX_INVALIDATE_BLOCKS = 2*MAX_INVALIDATE_LIMIT+REMAINING_BLOCKS;
|
|
@@ -83,7 +85,7 @@ public class TestHeartbeatHandling {
|
|
|
assertEquals(1, cmds.length);
|
|
|
assertEquals(DatanodeProtocol.DNA_TRANSFER, cmds[0].getAction());
|
|
|
assertEquals(MAX_REPLICATE_LIMIT, ((BlockCommand)cmds[0]).getBlocks().length);
|
|
|
-
|
|
|
+
|
|
|
ArrayList<Block> blockList = new ArrayList<Block>(MAX_INVALIDATE_BLOCKS);
|
|
|
for (int i=0; i<MAX_INVALIDATE_BLOCKS; i++) {
|
|
|
blockList.add(new Block(i, 0, GenerationStamp.LAST_RESERVED_STAMP));
|
|
@@ -122,4 +124,113 @@ public class TestHeartbeatHandling {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test if
|
|
|
+ * {@link FSNamesystem#handleHeartbeat}
|
|
|
+ * correctly selects data node targets for block recovery.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testHeartbeatBlockRecovery() throws Exception {
|
|
|
+ final Configuration conf = new HdfsConfiguration();
|
|
|
+ final MiniDFSCluster cluster =
|
|
|
+ new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
|
|
|
+ try {
|
|
|
+ cluster.waitActive();
|
|
|
+ final FSNamesystem namesystem = cluster.getNamesystem();
|
|
|
+ final HeartbeatManager hm = namesystem.getBlockManager(
|
|
|
+ ).getDatanodeManager().getHeartbeatManager();
|
|
|
+ final String poolId = namesystem.getBlockPoolId();
|
|
|
+ final DatanodeRegistration nodeReg1 =
|
|
|
+ DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(0), poolId);
|
|
|
+ final DatanodeDescriptor dd1 = NameNodeAdapter.getDatanode(namesystem, nodeReg1);
|
|
|
+ final DatanodeRegistration nodeReg2 =
|
|
|
+ DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(1), poolId);
|
|
|
+ final DatanodeDescriptor dd2 = NameNodeAdapter.getDatanode(namesystem, nodeReg2);
|
|
|
+ final DatanodeRegistration nodeReg3 =
|
|
|
+ DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(2), poolId);
|
|
|
+ final DatanodeDescriptor dd3 = NameNodeAdapter.getDatanode(namesystem, nodeReg3);
|
|
|
+
|
|
|
+ try {
|
|
|
+ namesystem.writeLock();
|
|
|
+ synchronized(hm) {
|
|
|
+ NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem);
|
|
|
+ NameNodeAdapter.sendHeartBeat(nodeReg2, dd2, namesystem);
|
|
|
+ NameNodeAdapter.sendHeartBeat(nodeReg3, dd3, namesystem);
|
|
|
+
|
|
|
+ // Test with all alive nodes.
|
|
|
+ dd1.setLastUpdate(System.currentTimeMillis());
|
|
|
+ dd2.setLastUpdate(System.currentTimeMillis());
|
|
|
+ dd3.setLastUpdate(System.currentTimeMillis());
|
|
|
+ BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
|
|
|
+ new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3,
|
|
|
+ BlockUCState.UNDER_RECOVERY,
|
|
|
+ new DatanodeDescriptor[] {dd1, dd2, dd3});
|
|
|
+ dd1.addBlockToBeRecovered(blockInfo);
|
|
|
+ DatanodeCommand[] cmds =
|
|
|
+ NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
|
|
|
+ assertEquals(1, cmds.length);
|
|
|
+ assertEquals(DatanodeProtocol.DNA_RECOVERBLOCK, cmds[0].getAction());
|
|
|
+ BlockRecoveryCommand recoveryCommand = (BlockRecoveryCommand)cmds[0];
|
|
|
+ assertEquals(1, recoveryCommand.getRecoveringBlocks().size());
|
|
|
+ DatanodeInfo[] recoveringNodes = recoveryCommand.getRecoveringBlocks()
|
|
|
+ .toArray(new BlockRecoveryCommand.RecoveringBlock[0])[0].getLocations();
|
|
|
+ assertEquals(3, recoveringNodes.length);
|
|
|
+ assertEquals(recoveringNodes[0], (DatanodeInfo)dd1);
|
|
|
+ assertEquals(recoveringNodes[1], (DatanodeInfo)dd2);
|
|
|
+ assertEquals(recoveringNodes[2], (DatanodeInfo)dd3);
|
|
|
+
|
|
|
+ // Test with one stale node.
|
|
|
+ dd1.setLastUpdate(System.currentTimeMillis());
|
|
|
+ // More than the default stale interval of 30 seconds.
|
|
|
+ dd2.setLastUpdate(System.currentTimeMillis() - 40 * 1000);
|
|
|
+ dd3.setLastUpdate(System.currentTimeMillis());
|
|
|
+ blockInfo = new BlockInfoUnderConstruction(
|
|
|
+ new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3,
|
|
|
+ BlockUCState.UNDER_RECOVERY,
|
|
|
+ new DatanodeDescriptor[] {dd1, dd2, dd3});
|
|
|
+ dd1.addBlockToBeRecovered(blockInfo);
|
|
|
+ cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
|
|
|
+ assertEquals(1, cmds.length);
|
|
|
+ assertEquals(DatanodeProtocol.DNA_RECOVERBLOCK, cmds[0].getAction());
|
|
|
+ recoveryCommand = (BlockRecoveryCommand)cmds[0];
|
|
|
+ assertEquals(1, recoveryCommand.getRecoveringBlocks().size());
|
|
|
+ recoveringNodes = recoveryCommand.getRecoveringBlocks()
|
|
|
+ .toArray(new BlockRecoveryCommand.RecoveringBlock[0])[0].getLocations();
|
|
|
+ assertEquals(2, recoveringNodes.length);
|
|
|
+ // dd2 is skipped.
|
|
|
+ assertEquals(recoveringNodes[0], (DatanodeInfo)dd1);
|
|
|
+ assertEquals(recoveringNodes[1], (DatanodeInfo)dd3);
|
|
|
+
|
|
|
+ // Test with all stale node.
|
|
|
+ dd1.setLastUpdate(System.currentTimeMillis() - 60 * 1000);
|
|
|
+ // More than the default stale interval of 30 seconds.
|
|
|
+ dd2.setLastUpdate(System.currentTimeMillis() - 40 * 1000);
|
|
|
+ dd3.setLastUpdate(System.currentTimeMillis() - 80 * 1000);
|
|
|
+ blockInfo = new BlockInfoUnderConstruction(
|
|
|
+ new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3,
|
|
|
+ BlockUCState.UNDER_RECOVERY,
|
|
|
+ new DatanodeDescriptor[] {dd1, dd2, dd3});
|
|
|
+ dd1.addBlockToBeRecovered(blockInfo);
|
|
|
+ cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
|
|
|
+ assertEquals(1, cmds.length);
|
|
|
+ assertEquals(DatanodeProtocol.DNA_RECOVERBLOCK, cmds[0].getAction());
|
|
|
+ recoveryCommand = (BlockRecoveryCommand)cmds[0];
|
|
|
+ assertEquals(1, recoveryCommand.getRecoveringBlocks().size());
|
|
|
+ recoveringNodes = recoveryCommand.getRecoveringBlocks()
|
|
|
+ .toArray(new BlockRecoveryCommand.RecoveringBlock[0])[0].getLocations();
|
|
|
+ // Only dd1 is included since it heart beated and hence its not stale
|
|
|
+ // when the list of recovery blocks is constructed.
|
|
|
+ assertEquals(3, recoveringNodes.length);
|
|
|
+ assertEquals(recoveringNodes[0], (DatanodeInfo)dd1);
|
|
|
+ assertEquals(recoveringNodes[1], (DatanodeInfo)dd2);
|
|
|
+ assertEquals(recoveringNodes[2], (DatanodeInfo)dd3);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ namesystem.writeUnlock();
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|