HDFS-5207. In BlockPlacementPolicy.chooseTarget(..), change the writer and the excludedNodes parameter types respectively to Node and Set. Contributed by Junping Du

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1523875 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2013-09-17 02:38:47 +00:00
parent 90f43b1c18
commit f98c343c7f
10 changed files with 121 additions and 105 deletions

View File

@ -279,6 +279,10 @@ Release 2.3.0 - UNRELEASED
methods; replace HashMap with Map in parameter declarations and cleanup
some related code. (szetszwo)
HDFS-5207. In BlockPlacementPolicy.chooseTarget(..), change the writer
and the excludedNodes parameter types respectively to Node and Set.
(Junping Du via szetszwo)
OPTIMIZATIONS
BUG FIXES

View File

@ -26,6 +26,7 @@
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
@ -1256,13 +1257,13 @@ int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
namesystem.writeUnlock();
}
final Map<Node, Node> excludedNodes = new HashMap<Node, Node>();
final Set<Node> excludedNodes = new HashSet<Node>();
for(ReplicationWork rw : work){
// Exclude all of the containing nodes from being targets.
// This list includes decommissioning or corrupt nodes.
excludedNodes.clear();
for (DatanodeDescriptor dn : rw.containingNodes) {
excludedNodes.put(dn, dn);
excludedNodes.add(dn);
}
// choose replication targets: NOT HOLDING THE GLOBAL LOCK
@ -1375,12 +1376,12 @@ int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
*
* @throws IOException
* if the number of targets < minimum replication.
* @see BlockPlacementPolicy#chooseTarget(String, int, DatanodeDescriptor,
* List, boolean, HashMap, long)
* @see BlockPlacementPolicy#chooseTarget(String, int, Node,
* List, boolean, Set, long)
*/
public DatanodeDescriptor[] chooseTarget(final String src,
final int numOfReplicas, final DatanodeDescriptor client,
final HashMap<Node, Node> excludedNodes,
final Set<Node> excludedNodes,
final long blocksize, List<String> favoredNodes) throws IOException {
List<DatanodeDescriptor> favoredDatanodeDescriptors =
getDatanodeDescriptors(favoredNodes);
@ -3248,7 +3249,7 @@ public ReplicationWork(Block block,
}
private void chooseTargets(BlockPlacementPolicy blockplacement,
Map<Node, Node> excludedNodes) {
Set<Node> excludedNodes) {
targets = blockplacement.chooseTarget(bc.getName(),
additionalReplRequired, srcNode, liveReplicaNodes, false,
excludedNodes, block.getNumBytes());

View File

@ -21,6 +21,7 @@
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -68,22 +69,22 @@ public static class NotEnoughReplicasException extends Exception {
*/
public abstract DatanodeDescriptor[] chooseTarget(String srcPath,
int numOfReplicas,
DatanodeDescriptor writer,
Node writer,
List<DatanodeDescriptor> chosenNodes,
boolean returnChosenNodes,
Map<Node, Node> excludedNodes,
Set<Node> excludedNodes,
long blocksize);
/**
* Same as {@link #chooseTarget(String, int, DatanodeDescriptor, List, boolean,
* HashMap, long)} with added parameter {@code favoredDatanodes}
* Same as {@link #chooseTarget(String, int, Node, List, boolean,
* Set, long)} with added parameter {@code favoredDatanodes}
* @param favoredNodes datanodes that should be favored as targets. This
* is only a hint and due to cluster state, namenode may not be
* able to place the blocks on these datanodes.
*/
DatanodeDescriptor[] chooseTarget(String src,
int numOfReplicas, DatanodeDescriptor writer,
Map<Node, Node> excludedNodes,
int numOfReplicas, Node writer,
Set<Node> excludedNodes,
long blocksize, List<DatanodeDescriptor> favoredNodes) {
// This class does not provide the functionality of placing
// a block in favored datanodes. The implementations of this class

View File

@ -21,9 +21,8 @@
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
@ -106,10 +105,10 @@ public void initialize(Configuration conf, FSClusterStats stats,
@Override
public DatanodeDescriptor[] chooseTarget(String srcPath,
int numOfReplicas,
DatanodeDescriptor writer,
Node writer,
List<DatanodeDescriptor> chosenNodes,
boolean returnChosenNodes,
Map<Node, Node> excludedNodes,
Set<Node> excludedNodes,
long blocksize) {
return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes,
excludedNodes, blocksize);
@ -118,8 +117,8 @@ public DatanodeDescriptor[] chooseTarget(String srcPath,
@Override
DatanodeDescriptor[] chooseTarget(String src,
int numOfReplicas,
DatanodeDescriptor writer,
Map<Node, Node> excludedNodes,
Node writer,
Set<Node> excludedNodes,
long blocksize,
List<DatanodeDescriptor> favoredNodes) {
try {
@ -130,8 +129,8 @@ DatanodeDescriptor[] chooseTarget(String src,
excludedNodes, blocksize);
}
Map<Node, Node> favoriteAndExcludedNodes = excludedNodes == null ?
new HashMap<Node, Node>() : new HashMap<Node, Node>(excludedNodes);
Set<Node> favoriteAndExcludedNodes = excludedNodes == null ?
new HashSet<Node>() : new HashSet<Node>(excludedNodes);
// Choose favored nodes
List<DatanodeDescriptor> results = new ArrayList<DatanodeDescriptor>();
@ -150,7 +149,7 @@ DatanodeDescriptor[] chooseTarget(String src,
+ " with favored node " + favoredNode);
continue;
}
favoriteAndExcludedNodes.put(target, target);
favoriteAndExcludedNodes.add(target);
}
if (results.size() < numOfReplicas) {
@ -175,17 +174,17 @@ DatanodeDescriptor[] chooseTarget(String src,
/** This is the implementation. */
private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
DatanodeDescriptor writer,
Node writer,
List<DatanodeDescriptor> chosenNodes,
boolean returnChosenNodes,
Map<Node, Node> excludedNodes,
Set<Node> excludedNodes,
long blocksize) {
if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
return DatanodeDescriptor.EMPTY_ARRAY;
}
if (excludedNodes == null) {
excludedNodes = new HashMap<Node, Node>();
excludedNodes = new HashSet<Node>();
}
int[] result = getMaxNodesPerRack(chosenNodes, numOfReplicas);
@ -205,7 +204,7 @@ private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
boolean avoidStaleNodes = (stats != null
&& stats.isAvoidingStaleDataNodesForWrite());
DatanodeDescriptor localNode = chooseTarget(numOfReplicas, writer,
Node localNode = chooseTarget(numOfReplicas, writer,
excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes);
if (!returnChosenNodes) {
results.removeAll(chosenNodes);
@ -228,10 +227,20 @@ private int[] getMaxNodesPerRack(List<DatanodeDescriptor> chosenNodes,
return new int[] {numOfReplicas, maxNodesPerRack};
}
/* choose <i>numOfReplicas</i> from all data nodes */
private DatanodeDescriptor chooseTarget(int numOfReplicas,
DatanodeDescriptor writer,
Map<Node, Node> excludedNodes,
/**
* choose <i>numOfReplicas</i> from all data nodes
* @param numOfReplicas additional number of replicas wanted
* @param writer the writer's machine, could be a non-DatanodeDescriptor node
* @param excludedNodes datanodes that should not be considered as targets
* @param blocksize size of the data to be written
* @param maxNodesPerRack max nodes allowed per rack
* @param results the target nodes already chosen
* @param avoidStaleNodes avoid stale nodes in replica choosing
* @return local node of writer (not chosen node)
*/
private Node chooseTarget(int numOfReplicas,
Node writer,
Set<Node> excludedNodes,
long blocksize,
int maxNodesPerRack,
List<DatanodeDescriptor> results,
@ -243,13 +252,13 @@ private DatanodeDescriptor chooseTarget(int numOfReplicas,
int numOfResults = results.size();
boolean newBlock = (numOfResults==0);
if (writer == null && !newBlock) {
if ((writer == null || !(writer instanceof DatanodeDescriptor)) && !newBlock) {
writer = results.get(0);
}
// Keep a copy of original excludedNodes
final Map<Node, Node> oldExcludedNodes = avoidStaleNodes ?
new HashMap<Node, Node>(excludedNodes) : null;
final Set<Node> oldExcludedNodes = avoidStaleNodes ?
new HashSet<Node>(excludedNodes) : null;
try {
if (numOfResults == 0) {
writer = chooseLocalNode(writer, excludedNodes, blocksize,
@ -296,7 +305,7 @@ private DatanodeDescriptor chooseTarget(int numOfReplicas,
// We need to additionally exclude the nodes that were added to the
// result list in the successful calls to choose*() above.
for (Node node : results) {
oldExcludedNodes.put(node, node);
oldExcludedNodes.add(node);
}
// Set numOfReplicas, since it can get out of sync with the result list
// if the NotEnoughReplicasException was thrown in chooseRandom().
@ -314,8 +323,8 @@ private DatanodeDescriptor chooseTarget(int numOfReplicas,
* choose a node on the same rack
* @return the chosen node
*/
protected DatanodeDescriptor chooseLocalNode(DatanodeDescriptor localMachine,
Map<Node, Node> excludedNodes,
protected DatanodeDescriptor chooseLocalNode(Node localMachine,
Set<Node> excludedNodes,
long blocksize,
int maxNodesPerRack,
List<DatanodeDescriptor> results,
@ -325,13 +334,13 @@ protected DatanodeDescriptor chooseLocalNode(DatanodeDescriptor localMachine,
if (localMachine == null)
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes);
if (preferLocalNode) {
if (preferLocalNode && localMachine instanceof DatanodeDescriptor) {
DatanodeDescriptor localDatanode = (DatanodeDescriptor) localMachine;
// otherwise try local machine first
Node oldNode = excludedNodes.put(localMachine, localMachine);
if (oldNode == null) { // was not in the excluded list
if (addIfIsGoodTarget(localMachine, excludedNodes, blocksize,
if (excludedNodes.add(localMachine)) { // was not in the excluded list
if (addIfIsGoodTarget(localDatanode, excludedNodes, blocksize,
maxNodesPerRack, false, results, avoidStaleNodes) >= 0) {
return localMachine;
return localDatanode;
}
}
}
@ -347,9 +356,8 @@ protected DatanodeDescriptor chooseLocalNode(DatanodeDescriptor localMachine,
* @return number of new excluded nodes
*/
protected int addToExcludedNodes(DatanodeDescriptor localMachine,
Map<Node, Node> excludedNodes) {
Node node = excludedNodes.put(localMachine, localMachine);
return node == null?1:0;
Set<Node> excludedNodes) {
return excludedNodes.add(localMachine) ? 1 : 0;
}
/**
@ -360,8 +368,8 @@ protected int addToExcludedNodes(DatanodeDescriptor localMachine,
* in the cluster.
* @return the chosen node
*/
protected DatanodeDescriptor chooseLocalRack(DatanodeDescriptor localMachine,
Map<Node, Node> excludedNodes,
protected DatanodeDescriptor chooseLocalRack(Node localMachine,
Set<Node> excludedNodes,
long blocksize,
int maxNodesPerRack,
List<DatanodeDescriptor> results,
@ -412,7 +420,7 @@ protected DatanodeDescriptor chooseLocalRack(DatanodeDescriptor localMachine,
protected void chooseRemoteRack(int numOfReplicas,
DatanodeDescriptor localMachine,
Map<Node, Node> excludedNodes,
Set<Node> excludedNodes,
long blocksize,
int maxReplicasPerRack,
List<DatanodeDescriptor> results,
@ -436,7 +444,7 @@ protected void chooseRemoteRack(int numOfReplicas,
* @return the chosen node, if there is any.
*/
protected DatanodeDescriptor chooseRandom(String scope,
Map<Node, Node> excludedNodes,
Set<Node> excludedNodes,
long blocksize,
int maxNodesPerRack,
List<DatanodeDescriptor> results,
@ -452,7 +460,7 @@ protected DatanodeDescriptor chooseRandom(String scope,
*/
protected DatanodeDescriptor chooseRandom(int numOfReplicas,
String scope,
Map<Node, Node> excludedNodes,
Set<Node> excludedNodes,
long blocksize,
int maxNodesPerRack,
List<DatanodeDescriptor> results,
@ -460,7 +468,7 @@ protected DatanodeDescriptor chooseRandom(int numOfReplicas,
throws NotEnoughReplicasException {
int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes(
scope, excludedNodes.keySet());
scope, excludedNodes);
StringBuilder builder = null;
if (LOG.isDebugEnabled()) {
builder = debugLoggingBuilder.get();
@ -472,8 +480,7 @@ protected DatanodeDescriptor chooseRandom(int numOfReplicas,
while(numOfReplicas > 0 && numOfAvailableNodes > 0) {
DatanodeDescriptor chosenNode =
(DatanodeDescriptor)clusterMap.chooseRandom(scope);
Node oldNode = excludedNodes.put(chosenNode, chosenNode);
if (oldNode == null) {
if (excludedNodes.add(chosenNode)) { //was not in the excluded list
numOfAvailableNodes--;
int newExcludedNodes = addIfIsGoodTarget(chosenNode, excludedNodes,
@ -506,12 +513,12 @@ protected DatanodeDescriptor chooseRandom(int numOfReplicas,
/**
* If the given node is a good target, add it to the result list and
* update the excluded node map.
* update the set of excluded nodes.
* @return -1 if the given is not a good target;
* otherwise, return the number of excluded nodes added to the map.
* otherwise, return the number of nodes added to excludedNodes set.
*/
int addIfIsGoodTarget(DatanodeDescriptor node,
Map<Node, Node> excludedNodes,
Set<Node> excludedNodes,
long blockSize,
int maxNodesPerRack,
boolean considerLoad,
@ -614,7 +621,7 @@ private boolean isGoodTarget(DatanodeDescriptor node,
* starts from the writer and traverses all <i>nodes</i>
* This is basically a traveling salesman problem.
*/
private DatanodeDescriptor[] getPipeline(DatanodeDescriptor writer,
private DatanodeDescriptor[] getPipeline(Node writer,
DatanodeDescriptor[] nodes) {
if (nodes.length==0) return nodes;

View File

@ -22,6 +22,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -63,8 +64,8 @@ public void initialize(Configuration conf, FSClusterStats stats,
* @return the chosen node
*/
@Override
protected DatanodeDescriptor chooseLocalNode(DatanodeDescriptor localMachine,
Map<Node, Node> excludedNodes, long blocksize, int maxNodesPerRack,
protected DatanodeDescriptor chooseLocalNode(Node localMachine,
Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
List<DatanodeDescriptor> results, boolean avoidStaleNodes)
throws NotEnoughReplicasException {
// if no local machine, randomly choose one node
@ -72,12 +73,14 @@ protected DatanodeDescriptor chooseLocalNode(DatanodeDescriptor localMachine,
return chooseRandom(NodeBase.ROOT, excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes);
if (localMachine instanceof DatanodeDescriptor) {
DatanodeDescriptor localDataNode = (DatanodeDescriptor)localMachine;
// otherwise try local machine first
Node oldNode = excludedNodes.put(localMachine, localMachine);
if (oldNode == null) { // was not in the excluded list
if (addIfIsGoodTarget(localMachine, excludedNodes, blocksize,
if (excludedNodes.add(localMachine)) { // was not in the excluded list
if (addIfIsGoodTarget(localDataNode, excludedNodes, blocksize,
maxNodesPerRack, false, results, avoidStaleNodes) >= 0) {
return localMachine;
return localDataNode;
}
}
}
@ -95,8 +98,8 @@ protected DatanodeDescriptor chooseLocalNode(DatanodeDescriptor localMachine,
@Override
protected DatanodeDescriptor chooseLocalRack(DatanodeDescriptor localMachine,
Map<Node, Node> excludedNodes, long blocksize, int maxNodesPerRack,
protected DatanodeDescriptor chooseLocalRack(Node localMachine,
Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
List<DatanodeDescriptor> results, boolean avoidStaleNodes)
throws NotEnoughReplicasException {
// no local machine, so choose a random machine
@ -142,7 +145,7 @@ protected DatanodeDescriptor chooseLocalRack(DatanodeDescriptor localMachine,
@Override
protected void chooseRemoteRack(int numOfReplicas,
DatanodeDescriptor localMachine, Map<Node, Node> excludedNodes,
DatanodeDescriptor localMachine, Set<Node> excludedNodes,
long blocksize, int maxReplicasPerRack, List<DatanodeDescriptor> results,
boolean avoidStaleNodes) throws NotEnoughReplicasException {
int oldNumOfReplicas = results.size();
@ -168,8 +171,8 @@ protected void chooseRemoteRack(int numOfReplicas,
* @return the chosen node
*/
private DatanodeDescriptor chooseLocalNodeGroup(
NetworkTopologyWithNodeGroup clusterMap, DatanodeDescriptor localMachine,
Map<Node, Node> excludedNodes, long blocksize, int maxNodesPerRack,
NetworkTopologyWithNodeGroup clusterMap, Node localMachine,
Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
List<DatanodeDescriptor> results, boolean avoidStaleNodes)
throws NotEnoughReplicasException {
// no local machine, so choose a random machine
@ -225,13 +228,12 @@ protected String getRack(final DatanodeInfo cur) {
*/
@Override
protected int addToExcludedNodes(DatanodeDescriptor chosenNode,
Map<Node, Node> excludedNodes) {
Set<Node> excludedNodes) {
int countOfExcludedNodes = 0;
String nodeGroupScope = chosenNode.getNetworkLocation();
List<Node> leafNodes = clusterMap.getLeaves(nodeGroupScope);
for (Node leafNode : leafNodes) {
Node node = excludedNodes.put(leafNode, leafNode);
if (node == null) {
if (excludedNodes.add(leafNode)) {
// not a existing node in excludedNodes
countOfExcludedNodes++;
}

View File

@ -223,7 +223,6 @@
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
import org.apache.hadoop.security.token.delegation.DelegationKey;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum;
@ -2443,7 +2442,7 @@ void setBlockPoolId(String bpid) {
* client to "try again later".
*/
LocatedBlock getAdditionalBlock(String src, long fileId, String clientName,
ExtendedBlock previous, HashMap<Node, Node> excludedNodes,
ExtendedBlock previous, Set<Node> excludedNodes,
List<String> favoredNodes)
throws LeaseExpiredException, NotReplicatedYetException,
QuotaExceededException, SafeModeException, UnresolvedLinkException,
@ -2642,7 +2641,7 @@ LocatedBlock makeLocatedBlock(Block blk,
/** @see NameNode#getAdditionalDatanode(String, ExtendedBlock, DatanodeInfo[], DatanodeInfo[], int, String) */
LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk,
final DatanodeInfo[] existings, final HashMap<Node, Node> excludes,
final DatanodeInfo[] existings, final Set<Node> excludes,
final int numAdditionalNodes, final String clientName
) throws IOException {
//check if the feature is enabled

View File

@ -29,8 +29,9 @@
import java.net.InetSocketAddress;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.hadoop.HadoopIllegalArgumentException;
@ -547,11 +548,11 @@ public LocatedBlock addBlock(String src, String clientName,
stateChangeLog.debug("*BLOCK* NameNode.addBlock: file " + src
+ " fileId=" + fileId + " for " + clientName);
}
HashMap<Node, Node> excludedNodesSet = null;
Set<Node> excludedNodesSet = null;
if (excludedNodes != null) {
excludedNodesSet = new HashMap<Node, Node>(excludedNodes.length);
excludedNodesSet = new HashSet<Node>(excludedNodes.length);
for (Node node : excludedNodes) {
excludedNodesSet.put(node, node);
excludedNodesSet.add(node);
}
}
List<String> favoredNodesList = (favoredNodes == null) ? null
@ -579,11 +580,11 @@ public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock
metrics.incrGetAdditionalDatanodeOps();
HashMap<Node, Node> excludeSet = null;
Set<Node> excludeSet = null;
if (excludes != null) {
excludeSet = new HashMap<Node, Node>(excludes.length);
excludeSet = new HashSet<Node>(excludes.length);
for (Node node : excludes) {
excludeSet.put(node, node);
excludeSet.add(node);
}
}
return namesystem.getAdditionalDatanode(src, blk,

View File

@ -29,9 +29,11 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ContentSummary;
@ -187,7 +189,7 @@ private static DatanodeDescriptor[] chooseTarget(int numOfReplicas,
}
private static DatanodeDescriptor[] chooseTarget(int numOfReplicas,
List<DatanodeDescriptor> chosenNodes, Map<Node, Node> excludedNodes) {
List<DatanodeDescriptor> chosenNodes, Set<Node> excludedNodes) {
return chooseTarget(numOfReplicas, dataNodes[0], chosenNodes, excludedNodes);
}
@ -195,7 +197,7 @@ private static DatanodeDescriptor[] chooseTarget(
int numOfReplicas,
DatanodeDescriptor writer,
List<DatanodeDescriptor> chosenNodes,
Map<Node, Node> excludedNodes) {
Set<Node> excludedNodes) {
return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes,
false, excludedNodes, BLOCK_SIZE);
}
@ -210,25 +212,25 @@ private static DatanodeDescriptor[] chooseTarget(
*/
@Test
public void testChooseTarget2() throws Exception {
HashMap<Node, Node> excludedNodes;
Set<Node> excludedNodes;
DatanodeDescriptor[] targets;
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
excludedNodes = new HashMap<Node, Node>();
excludedNodes.put(dataNodes[1], dataNodes[1]);
excludedNodes = new HashSet<Node>();
excludedNodes.add(dataNodes[1]);
targets = chooseTarget(0, chosenNodes, excludedNodes);
assertEquals(targets.length, 0);
excludedNodes.clear();
chosenNodes.clear();
excludedNodes.put(dataNodes[1], dataNodes[1]);
excludedNodes.add(dataNodes[1]);
targets = chooseTarget(1, chosenNodes, excludedNodes);
assertEquals(targets.length, 1);
assertEquals(targets[0], dataNodes[0]);
excludedNodes.clear();
chosenNodes.clear();
excludedNodes.put(dataNodes[1], dataNodes[1]);
excludedNodes.add(dataNodes[1]);
targets = chooseTarget(2, chosenNodes, excludedNodes);
assertEquals(targets.length, 2);
assertEquals(targets[0], dataNodes[0]);
@ -236,7 +238,7 @@ public void testChooseTarget2() throws Exception {
excludedNodes.clear();
chosenNodes.clear();
excludedNodes.put(dataNodes[1], dataNodes[1]);
excludedNodes.add(dataNodes[1]);
targets = chooseTarget(3, chosenNodes, excludedNodes);
assertEquals(targets.length, 3);
assertEquals(targets[0], dataNodes[0]);
@ -245,7 +247,7 @@ public void testChooseTarget2() throws Exception {
excludedNodes.clear();
chosenNodes.clear();
excludedNodes.put(dataNodes[1], dataNodes[1]);
excludedNodes.add(dataNodes[1]);
targets = chooseTarget(4, chosenNodes, excludedNodes);
assertEquals(targets.length, 4);
assertEquals(targets[0], dataNodes[0]);
@ -258,7 +260,7 @@ public void testChooseTarget2() throws Exception {
excludedNodes.clear();
chosenNodes.clear();
excludedNodes.put(dataNodes[1], dataNodes[1]);
excludedNodes.add(dataNodes[1]);
chosenNodes.add(dataNodes[2]);
targets = replicator.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true,
excludedNodes, BLOCK_SIZE);
@ -479,8 +481,8 @@ public void testChooseTargetWithStaleNodes() throws Exception {
assertEquals(targets.length, 1);
assertEquals(targets[0], dataNodes[1]);
HashMap<Node, Node> excludedNodes = new HashMap<Node, Node>();
excludedNodes.put(dataNodes[1], dataNodes[1]);
Set<Node> excludedNodes = new HashSet<Node>();
excludedNodes.add(dataNodes[1]);
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
targets = chooseTarget(1, chosenNodes, excludedNodes);
assertEquals(targets.length, 1);

View File

@ -181,7 +181,7 @@ private DatanodeDescriptor[] chooseTarget(
int numOfReplicas,
DatanodeDescriptor writer,
List<DatanodeDescriptor> chosenNodes,
Map<Node, Node> excludedNodes) {
Set<Node> excludedNodes) {
return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes,
false, excludedNodes, BLOCK_SIZE);
}
@ -253,13 +253,12 @@ private void verifyNoTwoTargetsOnSameNodeGroup(DatanodeDescriptor[] targets) {
*/
@Test
public void testChooseTarget2() throws Exception {
HashMap<Node, Node> excludedNodes;
DatanodeDescriptor[] targets;
BlockPlacementPolicyDefault repl = (BlockPlacementPolicyDefault)replicator;
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
excludedNodes = new HashMap<Node, Node>();
excludedNodes.put(dataNodes[1], dataNodes[1]);
Set<Node> excludedNodes = new HashSet<Node>();
excludedNodes.add(dataNodes[1]);
targets = repl.chooseTarget(filename, 4, dataNodes[0], chosenNodes, false,
excludedNodes, BLOCK_SIZE);
assertEquals(targets.length, 4);
@ -275,7 +274,7 @@ public void testChooseTarget2() throws Exception {
excludedNodes.clear();
chosenNodes.clear();
excludedNodes.put(dataNodes[1], dataNodes[1]);
excludedNodes.add(dataNodes[1]);
chosenNodes.add(dataNodes[2]);
targets = repl.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true,
excludedNodes, BLOCK_SIZE);

View File

@ -25,7 +25,7 @@
import java.lang.reflect.Field;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import org.apache.commons.logging.Log;
@ -119,7 +119,7 @@ public DatanodeDescriptor[] answer(InvocationOnMock invocation)
return ret;
}
}).when(spyBM).chooseTarget(Mockito.anyString(), Mockito.anyInt(),
Mockito.<DatanodeDescriptor>any(), Mockito.<HashMap<Node, Node>>any(),
Mockito.<DatanodeDescriptor>any(), Mockito.<HashSet<Node>>any(),
Mockito.anyLong(), Mockito.<List<String>>any());
// create file