|
@@ -29,13 +29,13 @@ import java.util.concurrent.locks.ReadWriteLock;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
-import org.apache.commons.logging.Log;
|
|
|
-import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import com.google.common.collect.Lists;
|
|
@@ -54,8 +54,8 @@ import com.google.common.collect.Lists;
|
|
|
public class NetworkTopology {
|
|
|
public final static String DEFAULT_RACK = "/default-rack";
|
|
|
public final static int DEFAULT_HOST_LEVEL = 2;
|
|
|
- public static final Log LOG =
|
|
|
- LogFactory.getLog(NetworkTopology.class);
|
|
|
+ public static final Logger LOG =
|
|
|
+ LoggerFactory.getLogger(NetworkTopology.class);
|
|
|
|
|
|
public static class InvalidTopologyException extends RuntimeException {
|
|
|
private static final long serialVersionUID = 1L;
|
|
@@ -442,9 +442,7 @@ public class NetworkTopology {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("NetworkTopology became:\n" + this.toString());
|
|
|
- }
|
|
|
+ LOG.debug("NetworkTopology became:\n{}", this.toString());
|
|
|
} finally {
|
|
|
netlock.writeLock().unlock();
|
|
|
}
|
|
@@ -517,9 +515,7 @@ public class NetworkTopology {
|
|
|
numOfRacks--;
|
|
|
}
|
|
|
}
|
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("NetworkTopology became:\n" + this.toString());
|
|
|
- }
|
|
|
+ LOG.debug("NetworkTopology became:\n{}", this.toString());
|
|
|
} finally {
|
|
|
netlock.writeLock().unlock();
|
|
|
}
|
|
@@ -717,26 +713,45 @@ public class NetworkTopology {
|
|
|
r.setSeed(seed);
|
|
|
}
|
|
|
|
|
|
- /** randomly choose one node from <i>scope</i>
|
|
|
- * if scope starts with ~, choose one from the all nodes except for the
|
|
|
- * ones in <i>scope</i>; otherwise, choose one from <i>scope</i>
|
|
|
+ /**
|
|
|
+ * Randomly choose a node.
|
|
|
+ *
|
|
|
* @param scope range of nodes from which a node will be chosen
|
|
|
* @return the chosen node
|
|
|
+ *
|
|
|
+ * @see #chooseRandom(String, Collection)
|
|
|
*/
|
|
|
- public Node chooseRandom(String scope) {
|
|
|
+ public Node chooseRandom(final String scope) {
|
|
|
+ return chooseRandom(scope, null);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Randomly choose one node from <i>scope</i>.
|
|
|
+ *
|
|
|
+ * If scope starts with ~, choose one from the all nodes except for the
|
|
|
+ * ones in <i>scope</i>; otherwise, choose one from <i>scope</i>.
|
|
|
+ * If excludedNodes is given, choose a node that's not in excludedNodes.
|
|
|
+ *
|
|
|
+ * @param scope range of nodes from which a node will be chosen
|
|
|
+ * @param excludedNodes nodes to be excluded from
|
|
|
+ * @return the chosen node
|
|
|
+ */
|
|
|
+ public Node chooseRandom(final String scope,
|
|
|
+ final Collection<Node> excludedNodes) {
|
|
|
netlock.readLock().lock();
|
|
|
try {
|
|
|
if (scope.startsWith("~")) {
|
|
|
- return chooseRandom(NodeBase.ROOT, scope.substring(1));
|
|
|
+ return chooseRandom(NodeBase.ROOT, scope.substring(1), excludedNodes);
|
|
|
} else {
|
|
|
- return chooseRandom(scope, null);
|
|
|
+ return chooseRandom(scope, null, excludedNodes);
|
|
|
}
|
|
|
} finally {
|
|
|
netlock.readLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private Node chooseRandom(String scope, String excludedScope){
|
|
|
+ private Node chooseRandom(final String scope, String excludedScope,
|
|
|
+ final Collection<Node> excludedNodes) {
|
|
|
if (excludedScope != null) {
|
|
|
if (scope.startsWith(excludedScope)) {
|
|
|
return null;
|
|
@@ -747,7 +762,8 @@ public class NetworkTopology {
|
|
|
}
|
|
|
Node node = getNode(scope);
|
|
|
if (!(node instanceof InnerNode)) {
|
|
|
- return node;
|
|
|
+ return excludedNodes != null && excludedNodes.contains(node) ?
|
|
|
+ null : node;
|
|
|
}
|
|
|
InnerNode innerNode = (InnerNode)node;
|
|
|
int numOfDatanodes = innerNode.getNumOfLeaves();
|
|
@@ -762,12 +778,36 @@ public class NetworkTopology {
|
|
|
}
|
|
|
}
|
|
|
if (numOfDatanodes == 0) {
|
|
|
- throw new InvalidTopologyException(
|
|
|
- "Failed to find datanode (scope=\"" + String.valueOf(scope) +
|
|
|
- "\" excludedScope=\"" + String.valueOf(excludedScope) + "\").");
|
|
|
+ LOG.warn("Failed to find datanode (scope=\"{}\" excludedScope=\"{}\").",
|
|
|
+ String.valueOf(scope), String.valueOf(excludedScope));
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ Node ret = null;
|
|
|
+ final int availableNodes;
|
|
|
+ if (excludedScope == null) {
|
|
|
+ availableNodes = countNumOfAvailableNodes(scope, excludedNodes);
|
|
|
+ } else {
|
|
|
+ availableNodes =
|
|
|
+ countNumOfAvailableNodes("~" + excludedScope, excludedNodes);
|
|
|
+ }
|
|
|
+ LOG.debug("Choosing random from {} available nodes on node {},"
|
|
|
+ + " scope={}, excludedScope={}, excludeNodes={}", availableNodes,
|
|
|
+ innerNode.toString(), scope, excludedScope, excludedNodes);
|
|
|
+ if (availableNodes > 0) {
|
|
|
+ do {
|
|
|
+ int leaveIndex = r.nextInt(numOfDatanodes);
|
|
|
+ ret = innerNode.getLeaf(leaveIndex, node);
|
|
|
+ if (excludedNodes == null || !excludedNodes.contains(ret)) {
|
|
|
+ break;
|
|
|
+ } else {
|
|
|
+ LOG.debug("Node {} is excluded, continuing.", ret);
|
|
|
+ }
|
|
|
+ // We've counted numOfAvailableNodes inside the lock, so there must be
|
|
|
+ // at least 1 satisfying node. Keep trying until we found it.
|
|
|
+ } while (true);
|
|
|
}
|
|
|
- int leaveIndex = r.nextInt(numOfDatanodes);
|
|
|
- return innerNode.getLeaf(leaveIndex, node);
|
|
|
+ LOG.debug("chooseRandom returning {}", ret);
|
|
|
+ return ret;
|
|
|
}
|
|
|
|
|
|
/** return leaves in <i>scope</i>
|
|
@@ -795,6 +835,7 @@ public class NetworkTopology {
|
|
|
* @param excludedNodes a list of nodes
|
|
|
* @return number of available nodes
|
|
|
*/
|
|
|
+ @VisibleForTesting
|
|
|
public int countNumOfAvailableNodes(String scope,
|
|
|
Collection<Node> excludedNodes) {
|
|
|
boolean isExcluded=false;
|
|
@@ -807,16 +848,18 @@ public class NetworkTopology {
|
|
|
int excludedCountOffScope = 0; // the number of nodes outside scope & excludedNodes
|
|
|
netlock.readLock().lock();
|
|
|
try {
|
|
|
- for (Node node : excludedNodes) {
|
|
|
- node = getNode(NodeBase.getPath(node));
|
|
|
- if (node == null) {
|
|
|
- continue;
|
|
|
- }
|
|
|
- if ((NodeBase.getPath(node) + NodeBase.PATH_SEPARATOR_STR)
|
|
|
- .startsWith(scope + NodeBase.PATH_SEPARATOR_STR)) {
|
|
|
- excludedCountInScope++;
|
|
|
- } else {
|
|
|
- excludedCountOffScope++;
|
|
|
+ if (excludedNodes != null) {
|
|
|
+ for (Node node : excludedNodes) {
|
|
|
+ node = getNode(NodeBase.getPath(node));
|
|
|
+ if (node == null) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ if ((NodeBase.getPath(node) + NodeBase.PATH_SEPARATOR_STR)
|
|
|
+ .startsWith(scope + NodeBase.PATH_SEPARATOR_STR)) {
|
|
|
+ excludedCountInScope++;
|
|
|
+ } else {
|
|
|
+ excludedCountOffScope++;
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
Node n = getNode(scope);
|