HDFS-1480. All replicas of a block can end up on the same rack when some datanodes are decommissioning. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1160897 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2011-08-23 22:04:15 +00:00
parent 61cbfa5fde
commit ec559db68e
5 changed files with 426 additions and 18 deletions

View File

@ -989,6 +989,9 @@ Trunk (unreleased changes)
HDFS-2267. DataXceiver thread name incorrect while waiting on op during
keepalive. (todd)
HDFS-1480. All replicas of a block can end up on the same rack when
some datanodes are decommissioning. (todd)
BREAKDOWN OF HDFS-1073 SUBTASKS
HDFS-1521. Persist transaction ID on disk between NN restarts.

View File

@ -66,6 +66,8 @@ import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.util.Daemon;
import com.google.common.annotations.VisibleForTesting;
/**
* Keeps information related to the blocks stored in the Hadoop cluster.
* This class is a helper class for {@link FSNamesystem} and requires several
@ -147,7 +149,8 @@ public class BlockManager {
// We also store pending replication-orders.
//
public final UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
private final PendingReplicationBlocks pendingReplications;
@VisibleForTesting
final PendingReplicationBlocks pendingReplications;
/** The maximum number of replicas allowed for a block */
public final short maxReplication;
@ -312,9 +315,14 @@ public class BlockManager {
for (Block block : neededReplications) {
List<DatanodeDescriptor> containingNodes =
new ArrayList<DatanodeDescriptor>();
List<DatanodeDescriptor> containingLiveReplicasNodes =
new ArrayList<DatanodeDescriptor>();
NumberReplicas numReplicas = new NumberReplicas();
// source node returned is not used
chooseSourceDatanode(block, containingNodes, numReplicas);
chooseSourceDatanode(block, containingNodes,
containingLiveReplicasNodes, numReplicas);
assert containingLiveReplicasNodes.size() == numReplicas.liveReplicas();
int usableReplicas = numReplicas.liveReplicas() +
numReplicas.decommissionedReplicas();
@ -993,9 +1001,10 @@ public class BlockManager {
* @param priority a hint of its priority in the neededReplication queue
* @return if the block gets replicated or not
*/
private boolean computeReplicationWorkForBlock(Block block, int priority) {
@VisibleForTesting
boolean computeReplicationWorkForBlock(Block block, int priority) {
int requiredReplication, numEffectiveReplicas;
List<DatanodeDescriptor> containingNodes;
List<DatanodeDescriptor> containingNodes, liveReplicaNodes;
DatanodeDescriptor srcNode;
INodeFile fileINode = null;
int additionalReplRequired;
@ -1016,11 +1025,14 @@ public class BlockManager {
// get a source data-node
containingNodes = new ArrayList<DatanodeDescriptor>();
liveReplicaNodes = new ArrayList<DatanodeDescriptor>();
NumberReplicas numReplicas = new NumberReplicas();
srcNode = chooseSourceDatanode(block, containingNodes, numReplicas);
srcNode = chooseSourceDatanode(
block, containingNodes, liveReplicaNodes, numReplicas);
if(srcNode == null) // block can not be replicated from any node
return false;
assert liveReplicaNodes.size() == numReplicas.liveReplicas();
// do not schedule more if enough replicas is already pending
numEffectiveReplicas = numReplicas.liveReplicas() +
pendingReplications.getNumReplicas(block);
@ -1047,13 +1059,20 @@ public class BlockManager {
} finally {
namesystem.writeUnlock();
}
// Exclude all of the containing nodes from being targets.
// This list includes decommissioning or corrupt nodes.
HashMap<Node, Node> excludedNodes = new HashMap<Node, Node>();
for (DatanodeDescriptor dn : containingNodes) {
excludedNodes.put(dn, dn);
}
// choose replication targets: NOT HOLDING THE GLOBAL LOCK
// It is costly to extract the filename for which chooseTargets is called,
// so for now we pass in the Inode itself.
DatanodeDescriptor targets[] =
blockplacement.chooseTarget(fileINode, additionalReplRequired,
srcNode, containingNodes, block.getNumBytes());
srcNode, liveReplicaNodes, excludedNodes, block.getNumBytes());
if(targets.length == 0)
return false;
@ -1182,8 +1201,10 @@ public class BlockManager {
private DatanodeDescriptor chooseSourceDatanode(
Block block,
List<DatanodeDescriptor> containingNodes,
List<DatanodeDescriptor> nodesContainingLiveReplicas,
NumberReplicas numReplicas) {
containingNodes.clear();
nodesContainingLiveReplicas.clear();
DatanodeDescriptor srcNode = null;
int live = 0;
int decommissioned = 0;
@ -1202,6 +1223,7 @@ public class BlockManager {
else if (excessBlocks != null && excessBlocks.contains(block)) {
excess++;
} else {
nodesContainingLiveReplicas.add(node);
live++;
}
containingNodes.add(node);
@ -2049,7 +2071,8 @@ public class BlockManager {
/**
* The given node is reporting that it received a certain block.
*/
private void addBlock(DatanodeDescriptor node, Block block, String delHint)
@VisibleForTesting
void addBlock(DatanodeDescriptor node, Block block, String delHint)
throws IOException {
// decrement number of blocks scheduled to this datanode.
node.decBlocksScheduled();

View File

@ -127,9 +127,10 @@ public abstract class BlockPlacementPolicy {
int numOfReplicas,
DatanodeDescriptor writer,
List<DatanodeDescriptor> chosenNodes,
HashMap<Node, Node> excludedNodes,
long blocksize) {
return chooseTarget(srcInode.getFullPathName(), numOfReplicas, writer,
chosenNodes, blocksize);
chosenNodes, excludedNodes, blocksize);
}
/**

View File

@ -102,16 +102,6 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
excludedNodes, blocksize);
}
/** {@inheritDoc} */
@Override
public DatanodeDescriptor[] chooseTarget(FSInodeInfo srcInode,
int numOfReplicas,
DatanodeDescriptor writer,
List<DatanodeDescriptor> chosenNodes,
long blocksize) {
return chooseTarget(numOfReplicas, writer, chosenNodes, false,
null, blocksize);
}
/** This is the implementation. */
DatanodeDescriptor[] chooseTarget(int numOfReplicas,

View File

@ -0,0 +1,391 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
import static org.junit.Assert.*;
import java.io.IOException;
import java.util.List;
import java.util.Map.Entry;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.net.NetworkTopology;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.LinkedListMultimap;
import com.google.common.collect.Lists;
public class TestBlockManager {
private final List<DatanodeDescriptor> nodes = ImmutableList.of(
new DatanodeDescriptor(new DatanodeID("h1:5020"), "/rackA"),
new DatanodeDescriptor(new DatanodeID("h2:5020"), "/rackA"),
new DatanodeDescriptor(new DatanodeID("h3:5020"), "/rackA"),
new DatanodeDescriptor(new DatanodeID("h4:5020"), "/rackB"),
new DatanodeDescriptor(new DatanodeID("h5:5020"), "/rackB"),
new DatanodeDescriptor(new DatanodeID("h6:5020"), "/rackB")
);
private final List<DatanodeDescriptor> rackA = nodes.subList(0, 3);
private final List<DatanodeDescriptor> rackB = nodes.subList(3, 6);
/**
* Some of these tests exercise code which has some randomness involved -
* ie even if there's a bug, they may pass because the random node selection
* chooses the correct result.
*
* Since they're true unit tests and run quickly, we loop them a number
* of times trying to trigger the incorrect behavior.
*/
private static final int NUM_TEST_ITERS = 30;
private static final int BLOCK_SIZE = 64*1024;
private Configuration conf;
private FSNamesystem fsn;
private BlockManager bm;
@Before
public void setupMockCluster() throws IOException {
conf = new HdfsConfiguration();
conf.set(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY,
"need to set a dummy value here so it assumes a multi-rack cluster");
fsn = Mockito.mock(FSNamesystem.class);
Mockito.doReturn(true).when(fsn).hasWriteLock();
bm = new BlockManager(fsn, conf);
}
private void addNodes(Iterable<DatanodeDescriptor> nodesToAdd) {
NetworkTopology cluster = bm.getDatanodeManager().getNetworkTopology();
// construct network topology
for (DatanodeDescriptor dn : nodesToAdd) {
cluster.add(dn);
dn.updateHeartbeat(
2*FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
2*FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
}
private void removeNode(DatanodeDescriptor deadNode) {
NetworkTopology cluster = bm.getDatanodeManager().getNetworkTopology();
cluster.remove(deadNode);
bm.removeBlocksAssociatedTo(deadNode);
}
/**
* Test that replication of under-replicated blocks is detected
* and basically works
*/
@Test
public void testBasicReplication() throws Exception {
addNodes(nodes);
for (int i = 0; i < NUM_TEST_ITERS; i++) {
doBasicTest(i);
}
}
private void doBasicTest(int testIndex) {
List<DatanodeDescriptor> origNodes = nodes(0, 1);
BlockInfo blockInfo = addBlockOnNodes((long)testIndex, origNodes);
DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo);
assertEquals(2, pipeline.length);
assertTrue("Source of replication should be one of the nodes the block " +
"was on. Was: " + pipeline[0],
origNodes.contains(pipeline[0]));
assertTrue("Destination of replication should be on the other rack. " +
"Was: " + pipeline[1],
rackB.contains(pipeline[1]));
}
/**
* Regression test for HDFS-1480
* - Cluster has 2 racks, A and B, each with three nodes.
* - Block initially written on A1, A2, B1
* - Admin decommissions two of these nodes (let's say A1 and A2 but it doesn't matter)
* - Re-replication should respect rack policy
*/
@Test
public void testTwoOfThreeNodesDecommissioned() throws Exception {
addNodes(nodes);
for (int i = 0; i < NUM_TEST_ITERS; i++) {
doTestTwoOfThreeNodesDecommissioned(i);
}
}
private void doTestTwoOfThreeNodesDecommissioned(int testIndex) throws Exception {
// Block originally on A1, A2, B1
List<DatanodeDescriptor> origNodes = nodes(0, 1, 3);
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
// Decommission two of the nodes (A1, A2)
List<DatanodeDescriptor> decomNodes = startDecommission(0, 1);
DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo);
assertTrue("Source of replication should be one of the nodes the block " +
"was on. Was: " + pipeline[0],
origNodes.contains(pipeline[0]));
assertEquals("Should have two targets", 3, pipeline.length);
boolean foundOneOnRackA = false;
for (int i = 1; i < pipeline.length; i++) {
DatanodeDescriptor target = pipeline[i];
if (rackA.contains(target)) {
foundOneOnRackA = true;
}
assertFalse(decomNodes.contains(target));
assertFalse(origNodes.contains(target));
}
assertTrue("Should have at least one target on rack A. Pipeline: " +
Joiner.on(",").join(pipeline),
foundOneOnRackA);
}
/**
* Test what happens when a block is on three nodes, and all three of those
* nodes are decommissioned. It should properly re-replicate to three new
* nodes.
*/
@Test
public void testAllNodesHoldingReplicasDecommissioned() throws Exception {
addNodes(nodes);
for (int i = 0; i < NUM_TEST_ITERS; i++) {
doTestAllNodesHoldingReplicasDecommissioned(i);
}
}
private void doTestAllNodesHoldingReplicasDecommissioned(int testIndex) throws Exception {
// Block originally on A1, A2, B1
List<DatanodeDescriptor> origNodes = nodes(0, 1, 3);
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
// Decommission all of the nodes
List<DatanodeDescriptor> decomNodes = startDecommission(0, 1, 3);
DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo);
assertTrue("Source of replication should be one of the nodes the block " +
"was on. Was: " + pipeline[0],
origNodes.contains(pipeline[0]));
assertEquals("Should have three targets", 4, pipeline.length);
boolean foundOneOnRackA = false;
boolean foundOneOnRackB = false;
for (int i = 1; i < pipeline.length; i++) {
DatanodeDescriptor target = pipeline[i];
if (rackA.contains(target)) {
foundOneOnRackA = true;
} else if (rackB.contains(target)) {
foundOneOnRackB = true;
}
assertFalse(decomNodes.contains(target));
assertFalse(origNodes.contains(target));
}
assertTrue("Should have at least one target on rack A. Pipeline: " +
Joiner.on(",").join(pipeline),
foundOneOnRackA);
assertTrue("Should have at least one target on rack B. Pipeline: " +
Joiner.on(",").join(pipeline),
foundOneOnRackB);
}
/**
* Test what happens when there are two racks, and an entire rack is
* decommissioned.
*
* Since the cluster is multi-rack, it will consider the block
* under-replicated rather than create a third replica on the
* same rack. Adding a new node on a third rack should cause re-replication
* to that node.
*/
@Test
public void testOneOfTwoRacksDecommissioned() throws Exception {
addNodes(nodes);
for (int i = 0; i < NUM_TEST_ITERS; i++) {
doTestOneOfTwoRacksDecommissioned(i);
}
}
private void doTestOneOfTwoRacksDecommissioned(int testIndex) throws Exception {
// Block originally on A1, A2, B1
List<DatanodeDescriptor> origNodes = nodes(0, 1, 3);
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
// Decommission all of the nodes in rack A
List<DatanodeDescriptor> decomNodes = startDecommission(0, 1, 2);
DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo);
assertTrue("Source of replication should be one of the nodes the block " +
"was on. Was: " + pipeline[0],
origNodes.contains(pipeline[0]));
assertEquals("Should have 2 targets", 3, pipeline.length);
boolean foundOneOnRackB = false;
for (int i = 1; i < pipeline.length; i++) {
DatanodeDescriptor target = pipeline[i];
if (rackB.contains(target)) {
foundOneOnRackB = true;
}
assertFalse(decomNodes.contains(target));
assertFalse(origNodes.contains(target));
}
assertTrue("Should have at least one target on rack B. Pipeline: " +
Joiner.on(",").join(pipeline),
foundOneOnRackB);
// Mark the block as received on the target nodes in the pipeline
fulfillPipeline(blockInfo, pipeline);
// the block is still under-replicated. Add a new node. This should allow
// the third off-rack replica.
DatanodeDescriptor rackCNode = new DatanodeDescriptor(new DatanodeID("h7:5020"), "/rackC");
addNodes(ImmutableList.of(rackCNode));
try {
DatanodeDescriptor[] pipeline2 = scheduleSingleReplication(blockInfo);
assertEquals(2, pipeline2.length);
assertEquals(rackCNode, pipeline2[1]);
} finally {
removeNode(rackCNode);
}
}
/**
* Unit test version of testSufficientlyReplBlocksUsesNewRack from
* {@link TestBlocksWithNotEnoughRacks}.
**/
@Test
public void testSufficientlyReplBlocksUsesNewRack() throws Exception {
addNodes(nodes);
for (int i = 0; i < NUM_TEST_ITERS; i++) {
doTestSufficientlyReplBlocksUsesNewRack(i);
}
}
private void doTestSufficientlyReplBlocksUsesNewRack(int testIndex) {
// Originally on only nodes in rack A.
List<DatanodeDescriptor> origNodes = rackA;
BlockInfo blockInfo = addBlockOnNodes((long)testIndex, origNodes);
DatanodeDescriptor pipeline[] = scheduleSingleReplication(blockInfo);
assertEquals(2, pipeline.length); // single new copy
assertTrue("Source of replication should be one of the nodes the block " +
"was on. Was: " + pipeline[0],
origNodes.contains(pipeline[0]));
assertTrue("Destination of replication should be on the other rack. " +
"Was: " + pipeline[1],
rackB.contains(pipeline[1]));
}
/**
* Tell the block manager that replication is completed for the given
* pipeline.
*/
private void fulfillPipeline(BlockInfo blockInfo,
DatanodeDescriptor[] pipeline) throws IOException {
for (int i = 1; i < pipeline.length; i++) {
bm.addBlock(pipeline[i], blockInfo, null);
}
}
private BlockInfo blockOnNodes(long blkId, List<DatanodeDescriptor> nodes) {
Block block = new Block(blkId);
BlockInfo blockInfo = new BlockInfo(block, 3);
for (DatanodeDescriptor dn : nodes) {
blockInfo.addNode(dn);
}
return blockInfo;
}
private List<DatanodeDescriptor> nodes(int ... indexes) {
List<DatanodeDescriptor> ret = Lists.newArrayList();
for (int idx : indexes) {
ret.add(nodes.get(idx));
}
return ret;
}
private List<DatanodeDescriptor> startDecommission(int ... indexes) {
List<DatanodeDescriptor> nodes = nodes(indexes);
for (DatanodeDescriptor node : nodes) {
node.startDecommission();
}
return nodes;
}
private BlockInfo addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
INodeFile iNode = Mockito.mock(INodeFile.class);
Mockito.doReturn((short)3).when(iNode).getReplication();
BlockInfo blockInfo = blockOnNodes(blockId, nodes);
bm.blocksMap.addINode(blockInfo, iNode);
return blockInfo;
}
private DatanodeDescriptor[] scheduleSingleReplication(Block block) {
assertEquals("Block not initially pending replication",
0, bm.pendingReplications.getNumReplicas(block));
assertTrue("computeReplicationWork should indicate replication is needed",
bm.computeReplicationWorkForBlock(block, 1));
assertTrue("replication is pending after work is computed",
bm.pendingReplications.getNumReplicas(block) > 0);
LinkedListMultimap<DatanodeDescriptor, BlockTargetPair> repls =
getAllPendingReplications();
assertEquals(1, repls.size());
Entry<DatanodeDescriptor, BlockTargetPair> repl = repls.entries().iterator().next();
DatanodeDescriptor[] targets = repl.getValue().targets;
DatanodeDescriptor[] pipeline = new DatanodeDescriptor[1 + targets.length];
pipeline[0] = repl.getKey();
System.arraycopy(targets, 0, pipeline, 1, targets.length);
return pipeline;
}
private LinkedListMultimap<DatanodeDescriptor, BlockTargetPair> getAllPendingReplications() {
LinkedListMultimap<DatanodeDescriptor, BlockTargetPair> repls =
LinkedListMultimap.create();
for (DatanodeDescriptor dn : nodes) {
List<BlockTargetPair> thisRepls = dn.getReplicationCommand(10);
if (thisRepls != null) {
repls.putAll(dn, thisRepls);
}
}
return repls;
}
}