HDDS-1879. Support multiple excluded scopes when choosing datanodes in NetworkTopology (#1194)
This commit is contained in:
parent
c8d61ffef6
commit
4222b62f2b
|
@ -18,6 +18,7 @@
|
|||
package org.apache.hadoop.hdds.scm.net;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* The interface defines an inner node in a network topology.
|
||||
|
@ -72,13 +73,13 @@ public interface InnerNode extends Node {
|
|||
*
|
||||
* @param leafIndex ode's index, start from 0, skip the nodes in
|
||||
* excludedScope and excludedNodes with ancestorGen
|
||||
* @param excludedScope the excluded scope
|
||||
* @param excludedScopes the excluded scopes
|
||||
* @param excludedNodes nodes to be excluded. If ancestorGen is not 0,
|
||||
* the chosen node will not share same ancestor with
|
||||
* those in excluded nodes at the specified generation
|
||||
* @param ancestorGen ignored with value is 0
|
||||
* @return the leaf node corresponding to the given index
|
||||
*/
|
||||
Node getLeaf(int leafIndex, String excludedScope,
|
||||
Node getLeaf(int leafIndex, List<String> excludedScopes,
|
||||
Collection<Node> excludedNodes, int ancestorGen);
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
@ -276,7 +277,7 @@ public class InnerNodeImpl extends NodeImpl implements InnerNode {
|
|||
*
|
||||
* @param leafIndex node's index, start from 0, skip the nodes in
|
||||
* excludedScope and excludedNodes with ancestorGen
|
||||
* @param excludedScope the exclude scope
|
||||
* @param excludedScopes the exclude scopes
|
||||
* @param excludedNodes nodes to be excluded from. If ancestorGen is not 0,
|
||||
* the chosen node will not share same ancestor with
|
||||
* those in excluded nodes at the specified generation
|
||||
|
@ -300,7 +301,7 @@ public class InnerNodeImpl extends NodeImpl implements InnerNode {
|
|||
*
|
||||
* Input:
|
||||
* leafIndex = 2
|
||||
* excludedScope = /dc2
|
||||
* excludedScope = /dc2/rack2
|
||||
* excludedNodes = {/dc1/rack1/n1}
|
||||
* ancestorGen = 1
|
||||
*
|
||||
|
@ -313,12 +314,12 @@ public class InnerNodeImpl extends NodeImpl implements InnerNode {
|
|||
* means picking the 3th available node, which is n5.
|
||||
*
|
||||
*/
|
||||
public Node getLeaf(int leafIndex, String excludedScope,
|
||||
public Node getLeaf(int leafIndex, List<String> excludedScopes,
|
||||
Collection<Node> excludedNodes, int ancestorGen) {
|
||||
Preconditions.checkArgument(leafIndex >= 0 && ancestorGen >= 0);
|
||||
// come to leaf parent layer
|
||||
if (isLeafParent()) {
|
||||
return getLeafOnLeafParent(leafIndex, excludedScope, excludedNodes);
|
||||
return getLeafOnLeafParent(leafIndex, excludedScopes, excludedNodes);
|
||||
}
|
||||
|
||||
int maxLevel = NodeSchemaManager.getInstance().getMaxLevel();
|
||||
|
@ -328,14 +329,16 @@ public class InnerNodeImpl extends NodeImpl implements InnerNode {
|
|||
Map<Node, Integer> countMap =
|
||||
getAncestorCountMap(excludedNodes, ancestorGen, currentGen);
|
||||
// nodes covered by excluded scope
|
||||
int excludedNodeCount = getExcludedScopeNodeCount(excludedScope);
|
||||
Map<String, Integer> excludedNodeCount =
|
||||
getExcludedScopeNodeCount(excludedScopes);
|
||||
|
||||
for(Node child : childrenMap.values()) {
|
||||
for (Node child : childrenMap.values()) {
|
||||
int leafCount = child.getNumOfLeaves();
|
||||
// skip nodes covered by excluded scope
|
||||
if (excludedScope != null &&
|
||||
excludedScope.startsWith(child.getNetworkFullPath())) {
|
||||
leafCount -= excludedNodeCount;
|
||||
// skip nodes covered by excluded scopes
|
||||
for (Map.Entry<String, Integer> entry: excludedNodeCount.entrySet()) {
|
||||
if (entry.getKey().startsWith(child.getNetworkFullPath())) {
|
||||
leafCount -= entry.getValue();
|
||||
}
|
||||
}
|
||||
// skip nodes covered by excluded nodes and ancestorGen
|
||||
Integer count = countMap.get(child);
|
||||
|
@ -343,7 +346,7 @@ public class InnerNodeImpl extends NodeImpl implements InnerNode {
|
|||
leafCount -= count;
|
||||
}
|
||||
if (leafIndex < leafCount) {
|
||||
return ((InnerNode)child).getLeaf(leafIndex, excludedScope,
|
||||
return ((InnerNode)child).getLeaf(leafIndex, excludedScopes,
|
||||
excludedNodes, ancestorGen);
|
||||
} else {
|
||||
leafIndex -= leafCount;
|
||||
|
@ -424,18 +427,22 @@ public class InnerNodeImpl extends NodeImpl implements InnerNode {
|
|||
* Get the node with leafIndex, considering skip nodes in excludedScope
|
||||
* and in excludeNodes list.
|
||||
*/
|
||||
private Node getLeafOnLeafParent(int leafIndex, String excludedScope,
|
||||
private Node getLeafOnLeafParent(int leafIndex, List<String> excludedScopes,
|
||||
Collection<Node> excludedNodes) {
|
||||
Preconditions.checkArgument(isLeafParent() && leafIndex >= 0);
|
||||
if (leafIndex >= getNumOfChildren()) {
|
||||
return null;
|
||||
}
|
||||
for(Node node : childrenMap.values()) {
|
||||
if ((excludedNodes != null && (excludedNodes.contains(node))) ||
|
||||
(excludedScope != null &&
|
||||
(node.getNetworkFullPath().startsWith(excludedScope)))) {
|
||||
if (excludedNodes != null && excludedNodes.contains(node)) {
|
||||
continue;
|
||||
}
|
||||
if (excludedScopes != null && excludedScopes.size() > 0) {
|
||||
if (excludedScopes.stream().anyMatch(scope ->
|
||||
node.getNetworkFullPath().startsWith(scope))) {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
if (leafIndex == 0) {
|
||||
return node;
|
||||
}
|
||||
|
@ -484,12 +491,19 @@ public class InnerNodeImpl extends NodeImpl implements InnerNode {
|
|||
return node;
|
||||
}
|
||||
|
||||
/** Get how many leaf nodes are covered by the excludedScope. */
|
||||
private int getExcludedScopeNodeCount(String excludedScope) {
|
||||
if (excludedScope == null) {
|
||||
return 0;
|
||||
/** Get how many leaf nodes are covered by the excludedScopes(no overlap). */
|
||||
private Map<String, Integer> getExcludedScopeNodeCount(
|
||||
List<String> excludedScopes) {
|
||||
HashMap<String, Integer> nodeCounts = new HashMap<>();
|
||||
if (excludedScopes == null || excludedScopes.isEmpty()) {
|
||||
return nodeCounts;
|
||||
}
|
||||
Node excludedScopeNode = getNode(excludedScope);
|
||||
return excludedScopeNode == null ? 0 : excludedScopeNode.getNumOfLeaves();
|
||||
|
||||
for (String scope: excludedScopes) {
|
||||
Node excludedScopeNode = getNode(scope);
|
||||
nodeCounts.put(scope, excludedScopeNode == null ? 0 :
|
||||
excludedScopeNode.getNumOfLeaves());
|
||||
}
|
||||
return nodeCounts;
|
||||
}
|
||||
}
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdds.scm.net;
|
||||
|
||||
import org.apache.commons.collections.CollectionUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -24,6 +25,7 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Utility class to facilitate network topology functions.
|
||||
|
@ -71,18 +73,17 @@ public final class NetUtils {
|
|||
* Remove node from mutableExcludedNodes if it's covered by excludedScope.
|
||||
* Please noted that mutableExcludedNodes content might be changed after the
|
||||
* function call.
|
||||
* @return the new excludedScope
|
||||
*/
|
||||
public static String removeDuplicate(NetworkTopology topology,
|
||||
Collection<Node> mutableExcludedNodes, String excludedScope,
|
||||
public static void removeDuplicate(NetworkTopology topology,
|
||||
Collection<Node> mutableExcludedNodes, List<String> mutableExcludedScopes,
|
||||
int ancestorGen) {
|
||||
if (mutableExcludedNodes == null || mutableExcludedNodes.size() == 0 ||
|
||||
excludedScope == null || topology == null) {
|
||||
return excludedScope;
|
||||
if (CollectionUtils.isEmpty(mutableExcludedNodes) ||
|
||||
CollectionUtils.isEmpty(mutableExcludedScopes) || topology == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
Iterator<Node> iterator = mutableExcludedNodes.iterator();
|
||||
while (iterator.hasNext()) {
|
||||
while (iterator.hasNext() && (!mutableExcludedScopes.isEmpty())) {
|
||||
Node node = iterator.next();
|
||||
Node ancestor = topology.getAncestor(node, ancestorGen);
|
||||
if (ancestor == null) {
|
||||
|
@ -90,16 +91,20 @@ public final class NetUtils {
|
|||
" of node :" + node);
|
||||
continue;
|
||||
}
|
||||
if (excludedScope.startsWith(ancestor.getNetworkFullPath())) {
|
||||
// reset excludedScope if it's covered by exclude node's ancestor
|
||||
return null;
|
||||
}
|
||||
if (ancestor.getNetworkFullPath().startsWith(excludedScope)) {
|
||||
// remove exclude node if it's covered by excludedScope
|
||||
iterator.remove();
|
||||
}
|
||||
// excludedScope is child of ancestor
|
||||
List<String> duplicateList = mutableExcludedScopes.stream()
|
||||
.filter(scope -> scope.startsWith(ancestor.getNetworkFullPath()))
|
||||
.collect(Collectors.toList());
|
||||
mutableExcludedScopes.removeAll(duplicateList);
|
||||
|
||||
// ancestor is covered by excludedScope
|
||||
mutableExcludedScopes.stream().forEach(scope -> {
|
||||
if (ancestor.getNetworkFullPath().startsWith(scope)) {
|
||||
// remove exclude node if it's covered by excludedScope
|
||||
iterator.remove();
|
||||
}
|
||||
});
|
||||
}
|
||||
return excludedScope;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -109,7 +114,7 @@ public final class NetUtils {
|
|||
*/
|
||||
public static void removeOutscope(Collection<Node> mutableExcludedNodes,
|
||||
String scope) {
|
||||
if (mutableExcludedNodes == null || scope == null) {
|
||||
if (CollectionUtils.isEmpty(mutableExcludedNodes) || scope == null) {
|
||||
return;
|
||||
}
|
||||
synchronized (mutableExcludedNodes) {
|
||||
|
@ -134,7 +139,7 @@ public final class NetUtils {
|
|||
public static List<Node> getAncestorList(NetworkTopology topology,
|
||||
Collection<Node> nodes, int generation) {
|
||||
List<Node> ancestorList = new ArrayList<>();
|
||||
if (topology == null ||nodes == null || nodes.size() == 0 ||
|
||||
if (topology == null || CollectionUtils.isEmpty(nodes) ||
|
||||
generation == 0) {
|
||||
return ancestorList;
|
||||
}
|
||||
|
|
|
@ -39,7 +39,6 @@ public interface NetworkTopology {
|
|||
*/
|
||||
void add(Node node);
|
||||
|
||||
|
||||
/**
|
||||
* Remove a node from the network topology. This will be called when a
|
||||
* existing datanode is removed from the system.
|
||||
|
@ -47,7 +46,6 @@ public interface NetworkTopology {
|
|||
*/
|
||||
void remove(Node node);
|
||||
|
||||
|
||||
/**
|
||||
* Check if the tree already contains node <i>node</i>.
|
||||
* @param node a node
|
||||
|
@ -68,7 +66,6 @@ public interface NetworkTopology {
|
|||
*/
|
||||
boolean isSameAncestor(Node node1, Node node2, int ancestorGen);
|
||||
|
||||
|
||||
/**
|
||||
* Get the ancestor for node on generation <i>ancestorGen</i>.
|
||||
*
|
||||
|
@ -119,11 +116,11 @@ public interface NetworkTopology {
|
|||
* Randomly choose a node in the scope, ano not in the exclude scope.
|
||||
* @param scope range of nodes from which a node will be chosen. cannot start
|
||||
* with ~
|
||||
* @param excludedScope the chosen node cannot be in this range. cannot
|
||||
* @param excludedScopes the chosen nodes cannot be in these ranges. cannot
|
||||
* starts with ~
|
||||
* @return the chosen node
|
||||
*/
|
||||
Node chooseRandom(String scope, String excludedScope);
|
||||
Node chooseRandom(String scope, List<String> excludedScopes);
|
||||
|
||||
/**
|
||||
* Randomly choose a leaf node from <i>scope</i>.
|
||||
|
@ -160,26 +157,6 @@ public interface NetworkTopology {
|
|||
Node chooseRandom(String scope, Collection<Node> excludedNodes,
|
||||
int ancestorGen);
|
||||
|
||||
|
||||
/**
|
||||
* Randomly choose a leaf node.
|
||||
*
|
||||
* @param scope range from which a node will be chosen, cannot start with ~
|
||||
* @param excludedNodes nodes to be excluded
|
||||
* @param excludedScope excluded node range. Cannot start with ~
|
||||
* @param ancestorGen matters when excludeNodes is not null. It means the
|
||||
* ancestor generation that's not allowed to share between chosen node and the
|
||||
* excludedNodes. For example, if ancestorGen is 1, means chosen node
|
||||
* cannot share the same parent with excludeNodes. If value is 2, cannot
|
||||
* share the same grand parent, and so on. If ancestorGen is 0, then no
|
||||
* effect.
|
||||
*
|
||||
* @return the chosen node
|
||||
*/
|
||||
Node chooseRandom(String scope, String excludedScope,
|
||||
Collection<Node> excludedNodes, int ancestorGen);
|
||||
|
||||
|
||||
/**
|
||||
* Randomly choose one node from <i>scope</i>, share the same generation
|
||||
* ancestor with <i>affinityNode</i>, and exclude nodes in
|
||||
|
@ -187,7 +164,7 @@ public interface NetworkTopology {
|
|||
*
|
||||
* @param scope range of nodes from which a node will be chosen, cannot start
|
||||
* with ~
|
||||
* @param excludedScope range of nodes to be excluded, cannot start with ~
|
||||
* @param excludedScopes ranges of nodes to be excluded, cannot start with ~
|
||||
* @param excludedNodes nodes to be excluded
|
||||
* @param affinityNode when not null, the chosen node should share the same
|
||||
* ancestor with this node at generation ancestorGen.
|
||||
|
@ -198,7 +175,7 @@ public interface NetworkTopology {
|
|||
* excludedNodes if affinityNode is null
|
||||
* @return the chosen node
|
||||
*/
|
||||
Node chooseRandom(String scope, String excludedScope,
|
||||
Node chooseRandom(String scope, List<String> excludedScopes,
|
||||
Collection<Node> excludedNodes, Node affinityNode, int ancestorGen);
|
||||
|
||||
/**
|
||||
|
@ -210,7 +187,7 @@ public interface NetworkTopology {
|
|||
* excludedNodes
|
||||
* @param scope range of nodes from which a node will be chosen, cannot start
|
||||
* with ~
|
||||
* @param excludedScope range of nodes to be excluded, cannot start with ~
|
||||
* @param excludedScopes ranges of nodes to be excluded, cannot start with ~
|
||||
* @param excludedNodes nodes to be excluded
|
||||
* @param affinityNode when not null, the chosen node should share the same
|
||||
* ancestor with this node at generation ancestorGen.
|
||||
|
@ -221,7 +198,7 @@ public interface NetworkTopology {
|
|||
* excludedNodes if affinityNode is null
|
||||
* @return the chosen node
|
||||
*/
|
||||
Node getNode(int leafIndex, String scope, String excludedScope,
|
||||
Node getNode(int leafIndex, String scope, List<String> excludedScopes,
|
||||
Collection<Node> excludedNodes, Node affinityNode, int ancestorGen);
|
||||
|
||||
/** Return the distance cost between two nodes
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hdds.scm.net;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.commons.collections.CollectionUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -283,7 +284,9 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
scope = ROOT;
|
||||
}
|
||||
if (scope.startsWith(SCOPE_REVERSE_STR)) {
|
||||
return chooseRandom(ROOT, scope.substring(1), null, null,
|
||||
ArrayList<String> excludedScopes = new ArrayList();
|
||||
excludedScopes.add(scope.substring(1));
|
||||
return chooseRandom(ROOT, excludedScopes, null, null,
|
||||
ANCESTOR_GENERATION_DEFAULT);
|
||||
} else {
|
||||
return chooseRandom(scope, null, null, null, ANCESTOR_GENERATION_DEFAULT);
|
||||
|
@ -294,12 +297,12 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
* Randomly choose a node in the scope, ano not in the exclude scope.
|
||||
* @param scope range of nodes from which a node will be chosen. cannot start
|
||||
* with ~
|
||||
* @param excludedScope the chosen node cannot be in this range. cannot
|
||||
* @param excludedScopes the chosen node cannot be in these ranges. cannot
|
||||
* starts with ~
|
||||
* @return the chosen node
|
||||
*/
|
||||
public Node chooseRandom(String scope, String excludedScope) {
|
||||
return chooseRandom(scope, excludedScope, null, null,
|
||||
public Node chooseRandom(String scope, List<String> excludedScopes) {
|
||||
return chooseRandom(scope, excludedScopes, null, null,
|
||||
ANCESTOR_GENERATION_DEFAULT);
|
||||
}
|
||||
|
||||
|
@ -320,7 +323,9 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
scope = ROOT;
|
||||
}
|
||||
if (scope.startsWith(SCOPE_REVERSE_STR)) {
|
||||
return chooseRandom(ROOT, scope.substring(1), excludedNodes, null,
|
||||
ArrayList<String> excludedScopes = new ArrayList();
|
||||
excludedScopes.add(scope.substring(1));
|
||||
return chooseRandom(ROOT, excludedScopes, excludedNodes, null,
|
||||
ANCESTOR_GENERATION_DEFAULT);
|
||||
} else {
|
||||
return chooseRandom(scope, null, excludedNodes, null,
|
||||
|
@ -352,33 +357,15 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
scope = ROOT;
|
||||
}
|
||||
if (scope.startsWith(SCOPE_REVERSE_STR)) {
|
||||
return chooseRandom(ROOT, scope.substring(1), excludedNodes, null,
|
||||
ArrayList<String> excludedScopes = new ArrayList();
|
||||
excludedScopes.add(scope.substring(1));
|
||||
return chooseRandom(ROOT, excludedScopes, excludedNodes, null,
|
||||
ancestorGen);
|
||||
} else {
|
||||
return chooseRandom(scope, null, excludedNodes, null, ancestorGen);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Randomly choose a leaf node.
|
||||
*
|
||||
* @param scope range from which a node will be chosen, cannot start with ~
|
||||
* @param excludedNodes nodes to be excluded
|
||||
* @param excludedScope excluded node range. Cannot start with ~
|
||||
* @param ancestorGen matters when excludeNodes is not null. It means the
|
||||
* ancestor generation that's not allowed to share between chosen node and the
|
||||
* excludedNodes. For example, if ancestorGen is 1, means chosen node
|
||||
* cannot share the same parent with excludeNodes. If value is 2, cannot
|
||||
* share the same grand parent, and so on. If ancestorGen is 0, then no
|
||||
* effect.
|
||||
*
|
||||
* @return the chosen node
|
||||
*/
|
||||
public Node chooseRandom(String scope, String excludedScope,
|
||||
Collection<Node> excludedNodes, int ancestorGen) {
|
||||
return chooseRandom(scope, excludedScope, excludedNodes, null, ancestorGen);
|
||||
}
|
||||
|
||||
/**
|
||||
* Randomly choose one leaf node from <i>scope</i>, share the same generation
|
||||
* ancestor with <i>affinityNode</i>, and exclude nodes in
|
||||
|
@ -386,7 +373,7 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
*
|
||||
* @param scope range of nodes from which a node will be chosen, cannot start
|
||||
* with ~
|
||||
* @param excludedScope range of nodes to be excluded, cannot start with ~
|
||||
* @param excludedScopes ranges of nodes to be excluded, cannot start with ~
|
||||
* @param excludedNodes nodes to be excluded
|
||||
* @param affinityNode when not null, the chosen node should share the same
|
||||
* ancestor with this node at generation ancestorGen.
|
||||
|
@ -397,20 +384,20 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
* excludedNodes if affinityNode is null
|
||||
* @return the chosen node
|
||||
*/
|
||||
public Node chooseRandom(String scope, String excludedScope,
|
||||
public Node chooseRandom(String scope, List<String> excludedScopes,
|
||||
Collection<Node> excludedNodes, Node affinityNode, int ancestorGen) {
|
||||
if (scope == null) {
|
||||
scope = ROOT;
|
||||
}
|
||||
|
||||
checkScope(scope);
|
||||
checkExcludedScope(excludedScope);
|
||||
checkExcludedScopes(excludedScopes);
|
||||
checkAffinityNode(affinityNode);
|
||||
checkAncestorGen(ancestorGen);
|
||||
|
||||
netlock.readLock().lock();
|
||||
try {
|
||||
return chooseNodeInternal(scope, -1, excludedScope,
|
||||
return chooseNodeInternal(scope, -1, excludedScopes,
|
||||
excludedNodes, affinityNode, ancestorGen);
|
||||
} finally {
|
||||
netlock.readLock().unlock();
|
||||
|
@ -426,7 +413,7 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
* excludedNodes
|
||||
* @param scope range of nodes from which a node will be chosen, cannot start
|
||||
* with ~
|
||||
* @param excludedScope range of nodes to be excluded, cannot start with ~
|
||||
* @param excludedScopes ranges of nodes to be excluded, cannot start with ~
|
||||
* @param excludedNodes nodes to be excluded
|
||||
* @param affinityNode when not null, the chosen node should share the same
|
||||
* ancestor with this node at generation ancestorGen.
|
||||
|
@ -466,20 +453,20 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
* from subtree /dc1. LeafIndex 1, so we pick the 2nd available node n4.
|
||||
*
|
||||
*/
|
||||
public Node getNode(int leafIndex, String scope, String excludedScope,
|
||||
public Node getNode(int leafIndex, String scope, List<String> excludedScopes,
|
||||
Collection<Node> excludedNodes, Node affinityNode, int ancestorGen) {
|
||||
Preconditions.checkArgument(leafIndex >= 0);
|
||||
if (scope == null) {
|
||||
scope = ROOT;
|
||||
}
|
||||
checkScope(scope);
|
||||
checkExcludedScope(excludedScope);
|
||||
checkExcludedScopes(excludedScopes);
|
||||
checkAffinityNode(affinityNode);
|
||||
checkAncestorGen(ancestorGen);
|
||||
|
||||
netlock.readLock().lock();
|
||||
try {
|
||||
return chooseNodeInternal(scope, leafIndex, excludedScope,
|
||||
return chooseNodeInternal(scope, leafIndex, excludedScopes,
|
||||
excludedNodes, affinityNode, ancestorGen);
|
||||
} finally {
|
||||
netlock.readLock().unlock();
|
||||
|
@ -487,8 +474,8 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
}
|
||||
|
||||
private Node chooseNodeInternal(String scope, int leafIndex,
|
||||
String excludedScope, Collection<Node> excludedNodes, Node affinityNode,
|
||||
int ancestorGen) {
|
||||
List<String> excludedScopes, Collection<Node> excludedNodes,
|
||||
Node affinityNode, int ancestorGen) {
|
||||
Preconditions.checkArgument(scope != null);
|
||||
|
||||
String finalScope = scope;
|
||||
|
@ -509,40 +496,48 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
ancestorGen = 0;
|
||||
}
|
||||
|
||||
// check overlap of excludedScope and finalScope
|
||||
if (excludedScope != null) {
|
||||
// excludeScope covers finalScope
|
||||
if (finalScope.startsWith(excludedScope)) {
|
||||
return null;
|
||||
}
|
||||
// excludeScope and finalScope share nothing
|
||||
if (!excludedScope.startsWith(finalScope)) {
|
||||
excludedScope = null;
|
||||
// check overlap of excludedScopes and finalScope
|
||||
List<String> mutableExcludedScopes = null;
|
||||
if (excludedScopes != null && !excludedScopes.isEmpty()) {
|
||||
mutableExcludedScopes = new ArrayList<>();
|
||||
for (String s: excludedScopes) {
|
||||
// excludeScope covers finalScope
|
||||
if (finalScope.startsWith(s)) {
|
||||
return null;
|
||||
}
|
||||
// excludeScope and finalScope share nothing case
|
||||
if (s.startsWith(finalScope)) {
|
||||
if (!mutableExcludedScopes.stream().anyMatch(
|
||||
e -> s.startsWith(e))) {
|
||||
mutableExcludedScopes.add(s);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// clone excludedNodes before remove duplicate in it
|
||||
Collection<Node> mutableExNodes = null;
|
||||
|
||||
// Remove duplicate in excludedNodes
|
||||
if (excludedNodes != null) {
|
||||
// Remove duplicate in excludedNodes
|
||||
mutableExNodes =
|
||||
excludedNodes.stream().distinct().collect(Collectors.toList());
|
||||
}
|
||||
|
||||
// remove duplicate in mutableExNodes and excludedScope, given ancestorGen
|
||||
excludedScope = NetUtils.removeDuplicate(this, mutableExNodes,
|
||||
excludedScope, ancestorGen);
|
||||
// remove duplicate in mutableExNodes and mutableExcludedScopes
|
||||
NetUtils.removeDuplicate(this, mutableExNodes, mutableExcludedScopes,
|
||||
ancestorGen);
|
||||
|
||||
// calculate available node count
|
||||
Node scopeNode = getNode(finalScope);
|
||||
int availableNodes = getAvailableNodesCount(
|
||||
scopeNode.getNetworkFullPath(), excludedScope, mutableExNodes,
|
||||
scopeNode.getNetworkFullPath(), mutableExcludedScopes, mutableExNodes,
|
||||
ancestorGen);
|
||||
|
||||
if (availableNodes <= 0) {
|
||||
LOG.warn("No available node in (scope=\"{}\" excludedScope=\"{}\" " +
|
||||
"excludedNodes=\"{}\" ancestorGen=\"{}\").",
|
||||
scopeNode.getNetworkFullPath(), excludedScope, excludedNodes,
|
||||
scopeNode.getNetworkFullPath(), excludedScopes, excludedNodes,
|
||||
ancestorGen);
|
||||
return null;
|
||||
}
|
||||
|
@ -556,17 +551,17 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
int nodeIndex;
|
||||
if (leafIndex >= 0) {
|
||||
nodeIndex = leafIndex % availableNodes;
|
||||
ret = ((InnerNode)scopeNode).getLeaf(nodeIndex, excludedScope,
|
||||
ret = ((InnerNode)scopeNode).getLeaf(nodeIndex, mutableExcludedScopes,
|
||||
mutableExNodes, ancestorGen);
|
||||
} else {
|
||||
nodeIndex = ThreadLocalRandom.current().nextInt(availableNodes);
|
||||
ret = ((InnerNode)scopeNode).getLeaf(nodeIndex, excludedScope,
|
||||
ret = ((InnerNode)scopeNode).getLeaf(nodeIndex, mutableExcludedScopes,
|
||||
mutableExNodes, ancestorGen);
|
||||
}
|
||||
LOG.debug("Choosing node[index={},random={}] from \"{}\" available nodes" +
|
||||
" scope=\"{}\", excludedScope=\"{}\", excludeNodes=\"{}\".",
|
||||
nodeIndex, (leafIndex == -1 ? "true" : "false"), availableNodes,
|
||||
scopeNode.getNetworkFullPath(), excludedScope, excludedNodes);
|
||||
scopeNode.getNetworkFullPath(), excludedScopes, excludedNodes);
|
||||
LOG.debug("Chosen node = {}", (ret == null ? "not found" : ret.toString()));
|
||||
return ret;
|
||||
}
|
||||
|
@ -678,13 +673,13 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
* Return the number of leaves in <i>scope</i> but not in
|
||||
* <i>excludedNodes</i> and <i>excludeScope</i>.
|
||||
* @param scope the scope
|
||||
* @param excludedScope excluded scope
|
||||
* @param excludedScopes excluded scopes
|
||||
* @param mutableExcludedNodes a list of excluded nodes, content might be
|
||||
* changed after the call
|
||||
* @param ancestorGen same generation ancestor prohibit on excludedNodes
|
||||
* @return number of available nodes
|
||||
*/
|
||||
private int getAvailableNodesCount(String scope, String excludedScope,
|
||||
private int getAvailableNodesCount(String scope, List<String> excludedScopes,
|
||||
Collection<Node> mutableExcludedNodes, int ancestorGen) {
|
||||
Preconditions.checkArgument(scope != null);
|
||||
|
||||
|
@ -702,13 +697,15 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
}
|
||||
// number of nodes to exclude
|
||||
int excludedCount = 0;
|
||||
if (excludedScope != null) {
|
||||
Node excludedScopeNode = getNode(excludedScope);
|
||||
if (excludedScopeNode != null) {
|
||||
if (excludedScope.startsWith(scope)) {
|
||||
excludedCount += excludedScopeNode.getNumOfLeaves();
|
||||
} else if (scope.startsWith(excludedScope)) {
|
||||
return 0;
|
||||
if (excludedScopes != null) {
|
||||
for (String excludedScope: excludedScopes) {
|
||||
Node excludedScopeNode = getNode(excludedScope);
|
||||
if (excludedScopeNode != null) {
|
||||
if (excludedScope.startsWith(scope)) {
|
||||
excludedCount += excludedScopeNode.getNumOfLeaves();
|
||||
} else if (scope.startsWith(excludedScope)) {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -766,11 +763,14 @@ public class NetworkTopologyImpl implements NetworkTopology{
|
|||
}
|
||||
}
|
||||
|
||||
private void checkExcludedScope(String excludedScope) {
|
||||
if (excludedScope != null &&
|
||||
(excludedScope.startsWith(SCOPE_REVERSE_STR))) {
|
||||
throw new IllegalArgumentException("excludedScope " + excludedScope +
|
||||
" cannot start with " + SCOPE_REVERSE_STR);
|
||||
private void checkExcludedScopes(List<String> excludedScopes) {
|
||||
if (!CollectionUtils.isEmpty(excludedScopes)) {
|
||||
excludedScopes.stream().forEach(scope -> {
|
||||
if (scope.startsWith(SCOPE_REVERSE_STR)) {
|
||||
throw new IllegalArgumentException("excludedScope " + scope +
|
||||
" cannot start with " + SCOPE_REVERSE_STR);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -357,9 +357,11 @@ public class TestNetworkTopologyImpl {
|
|||
|
||||
// test chooseRandom(String scope, String excludedScope)
|
||||
path = dataNodes[random.nextInt(dataNodes.length)].getNetworkFullPath();
|
||||
assertNull(cluster.chooseRandom(path, path));
|
||||
assertNotNull(cluster.chooseRandom(null, path));
|
||||
assertNotNull(cluster.chooseRandom("", path));
|
||||
List<String> pathList = new ArrayList<>();
|
||||
pathList.add(path);
|
||||
assertNull(cluster.chooseRandom(path, pathList));
|
||||
assertNotNull(cluster.chooseRandom(null, pathList));
|
||||
assertNotNull(cluster.chooseRandom("", pathList));
|
||||
|
||||
// test chooseRandom(String scope, Collection<Node> excludedNodes)
|
||||
assertNull(cluster.chooseRandom("", Arrays.asList(dataNodes)));
|
||||
|
@ -399,7 +401,9 @@ public class TestNetworkTopologyImpl {
|
|||
}
|
||||
|
||||
// "" excludedScope, no node will ever be chosen
|
||||
frequency = pickNodes(100, "", null, null, 0);
|
||||
List<String> pathList = new ArrayList();
|
||||
pathList.add("");
|
||||
frequency = pickNodes(100, pathList, null, null, 0);
|
||||
for (Node key : dataNodes) {
|
||||
assertTrue(frequency.get(key) == 0);
|
||||
}
|
||||
|
@ -411,8 +415,10 @@ public class TestNetworkTopologyImpl {
|
|||
assertTrue(frequency.get(key) == 0);
|
||||
}
|
||||
// out network topology excluded scope, every node should be chosen
|
||||
scope = "/city1";
|
||||
frequency = pickNodes(cluster.getNumOfLeafNode(null), scope, null, null, 0);
|
||||
pathList.clear();
|
||||
pathList.add("/city1");
|
||||
frequency = pickNodes(
|
||||
cluster.getNumOfLeafNode(null), pathList, null, null, 0);
|
||||
for (Node key : dataNodes) {
|
||||
assertTrue(frequency.get(key) != 0);
|
||||
}
|
||||
|
@ -582,19 +588,32 @@ public class TestNetworkTopologyImpl {
|
|||
}};
|
||||
int[] affinityNodeIndexs = {0, dataNodes.length - 1,
|
||||
random.nextInt(dataNodes.length), random.nextInt(dataNodes.length)};
|
||||
Node[][] excludedScopeIndexs = {{dataNodes[0]},
|
||||
{dataNodes[dataNodes.length - 1]},
|
||||
{dataNodes[random.nextInt(dataNodes.length)]},
|
||||
{dataNodes[random.nextInt(dataNodes.length)],
|
||||
dataNodes[random.nextInt(dataNodes.length)]
|
||||
},
|
||||
{dataNodes[random.nextInt(dataNodes.length)],
|
||||
dataNodes[random.nextInt(dataNodes.length)],
|
||||
dataNodes[random.nextInt(dataNodes.length)],
|
||||
}};
|
||||
int leafNum = cluster.getNumOfLeafNode(null);
|
||||
Map<Node, Integer> frequency;
|
||||
String scope;
|
||||
List<String> pathList = new ArrayList<>();
|
||||
for (int k : affinityNodeIndexs) {
|
||||
for (int i : excludedNodeIndexs) {
|
||||
String path = dataNodes[i].getNetworkFullPath();
|
||||
while (!path.equals(ROOT)) {
|
||||
for (Node[] excludedScopes : excludedScopeIndexs) {
|
||||
pathList.clear();
|
||||
pathList.addAll(Arrays.stream(excludedScopes)
|
||||
.map(node -> node.getNetworkFullPath())
|
||||
.collect(Collectors.toList()));
|
||||
while (!pathList.get(0).equals(ROOT)) {
|
||||
int ancestorGen = cluster.getMaxLevel() - 1;
|
||||
while (ancestorGen > 0) {
|
||||
for (Node[] list : excludedNodeLists) {
|
||||
List<Node> excludedList = Arrays.asList(list);
|
||||
frequency = pickNodes(leafNum, path, excludedList, dataNodes[k],
|
||||
ancestorGen);
|
||||
frequency = pickNodes(leafNum, pathList, excludedList,
|
||||
dataNodes[k], ancestorGen);
|
||||
Node affinityAncestor = dataNodes[k].getAncestor(ancestorGen);
|
||||
for (Node key : dataNodes) {
|
||||
if (affinityAncestor != null) {
|
||||
|
@ -605,28 +624,33 @@ public class TestNetworkTopologyImpl {
|
|||
} else if (excludedList != null &&
|
||||
excludedList.contains(key)) {
|
||||
continue;
|
||||
} else if (path != null &&
|
||||
key.getNetworkFullPath().startsWith(path)) {
|
||||
} else if (pathList != null &&
|
||||
pathList.stream().anyMatch(path ->
|
||||
key.getNetworkFullPath().startsWith(path))) {
|
||||
continue;
|
||||
} else {
|
||||
fail("Node is not picked when sequentially going " +
|
||||
"through ancestor node's leaf nodes. node:" +
|
||||
key.getNetworkFullPath() + ", ancestor node:" +
|
||||
affinityAncestor.getNetworkFullPath() +
|
||||
", excludedScope: " + path + ", " + "excludedList:" +
|
||||
(excludedList == null ? "" : excludedList.toString()));
|
||||
", excludedScope: " + pathList.toString() + ", " +
|
||||
"excludedList:" + (excludedList == null ? "" :
|
||||
excludedList.toString()));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
ancestorGen--;
|
||||
}
|
||||
path = path.substring(0, path.lastIndexOf(PATH_SEPARATOR_STR));
|
||||
pathList = pathList.stream().map(path ->
|
||||
path.substring(0, path.lastIndexOf(PATH_SEPARATOR_STR)))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// all nodes excluded, no node will be picked
|
||||
String scope;
|
||||
List<Node> excludedList = Arrays.asList(dataNodes);
|
||||
for (int k : affinityNodeIndexs) {
|
||||
for (int i : excludedNodeIndexs) {
|
||||
|
@ -880,9 +904,12 @@ public class TestNetworkTopologyImpl {
|
|||
frequency.put(dnd, 0);
|
||||
}
|
||||
|
||||
List<String> pathList = new ArrayList<>();
|
||||
pathList.add(excludedScope.substring(1));
|
||||
for (int j = 0; j < numNodes; j++) {
|
||||
Node node = cluster.chooseRandom("", excludedScope.substring(1),
|
||||
excludedNodes, affinityNode, ancestorGen);
|
||||
|
||||
Node node = cluster.chooseRandom("", pathList, excludedNodes,
|
||||
affinityNode, ancestorGen);
|
||||
if (node != null) {
|
||||
frequency.put(node, frequency.get(node) + 1);
|
||||
}
|
||||
|
@ -895,7 +922,7 @@ public class TestNetworkTopologyImpl {
|
|||
* This picks a large amount of nodes sequentially.
|
||||
*
|
||||
* @param numNodes the number of nodes
|
||||
* @param excludedScope the excluded scope, should not start with "~"
|
||||
* @param excludedScopes the excluded scopes, should not start with "~"
|
||||
* @param excludedNodes the excluded node list
|
||||
* @param affinityNode the chosen node should share the same ancestor at
|
||||
* generation "ancestorGen" with this node
|
||||
|
@ -903,8 +930,9 @@ public class TestNetworkTopologyImpl {
|
|||
* this generation with excludedNodes
|
||||
* @return the frequency that nodes were chosen
|
||||
*/
|
||||
private Map<Node, Integer> pickNodes(int numNodes, String excludedScope,
|
||||
Collection<Node> excludedNodes, Node affinityNode, int ancestorGen) {
|
||||
private Map<Node, Integer> pickNodes(int numNodes,
|
||||
List<String> excludedScopes, Collection<Node> excludedNodes,
|
||||
Node affinityNode, int ancestorGen) {
|
||||
Map<Node, Integer> frequency = new HashMap<>();
|
||||
for (Node dnd : dataNodes) {
|
||||
frequency.put(dnd, 0);
|
||||
|
@ -912,7 +940,7 @@ public class TestNetworkTopologyImpl {
|
|||
excludedNodes = excludedNodes == null ? null :
|
||||
excludedNodes.stream().distinct().collect(Collectors.toList());
|
||||
for (int j = 0; j < numNodes; j++) {
|
||||
Node node = cluster.getNode(j, null, excludedScope, excludedNodes,
|
||||
Node node = cluster.getNode(j, null, excludedScopes, excludedNodes,
|
||||
affinityNode, ancestorGen);
|
||||
if (node != null) {
|
||||
frequency.put(node, frequency.get(node) + 1);
|
||||
|
|
|
@ -37,7 +37,7 @@ public class SCMContainerPlacementMetrics implements MetricsSource {
|
|||
public static final String SOURCE_NAME =
|
||||
SCMContainerPlacementMetrics.class.getSimpleName();
|
||||
private static final MetricsInfo RECORD_INFO = Interns.info(SOURCE_NAME,
|
||||
"SCM Placement Metrics");
|
||||
"SCM Container Placement Metrics");
|
||||
private static MetricsRegistry registry;
|
||||
|
||||
// total datanode allocation request count
|
||||
|
@ -55,27 +55,23 @@ public class SCMContainerPlacementMetrics implements MetricsSource {
|
|||
public static SCMContainerPlacementMetrics create() {
|
||||
MetricsSystem ms = DefaultMetricsSystem.instance();
|
||||
registry = new MetricsRegistry(RECORD_INFO);
|
||||
return ms.register(SOURCE_NAME, "SCM Placement Metrics",
|
||||
return ms.register(SOURCE_NAME, "SCM Container Placement Metrics",
|
||||
new SCMContainerPlacementMetrics());
|
||||
}
|
||||
|
||||
public void incrDatanodeRequestCount(long count) {
|
||||
System.out.println("request + 1");
|
||||
this.datanodeRequestCount.incr(count);
|
||||
}
|
||||
|
||||
public void incrDatanodeChooseSuccessCount() {
|
||||
System.out.println("success + 1");
|
||||
this.datanodeChooseSuccessCount.incr(1);
|
||||
}
|
||||
|
||||
public void incrDatanodeChooseFallbackCount() {
|
||||
System.out.println("fallback + 1");
|
||||
this.datanodeChooseFallbackCount.incr(1);
|
||||
}
|
||||
|
||||
public void incrDatanodeChooseAttemptCount() {
|
||||
System.out.println("attempt + 1");
|
||||
this.datanodeChooseAttemptCount.incr(1);
|
||||
}
|
||||
|
||||
|
|
|
@ -242,15 +242,15 @@ public final class SCMContainerPlacementRackAware extends SCMCommonPolicy {
|
|||
long sizeRequired) throws SCMException {
|
||||
int ancestorGen = RACK_LEVEL;
|
||||
int maxRetry = MAX_RETRY;
|
||||
List<Node> excludedNodesForCapacity = null;
|
||||
List<String> excludedNodesForCapacity = null;
|
||||
boolean isFallbacked = false;
|
||||
while(true) {
|
||||
Node node = networkTopology.chooseRandom(NetConstants.ROOT, null,
|
||||
excludedNodes, affinityNode, ancestorGen);
|
||||
metrics.incrDatanodeChooseAttemptCount();
|
||||
Node node = networkTopology.chooseRandom(NetConstants.ROOT,
|
||||
excludedNodesForCapacity, excludedNodes, affinityNode, ancestorGen);
|
||||
if (node == null) {
|
||||
// cannot find the node which meets all constrains
|
||||
LOG.warn("Failed to find the datanode. excludedNodes:" +
|
||||
LOG.warn("Failed to find the datanode for container. excludedNodes:" +
|
||||
(excludedNodes == null ? "" : excludedNodes.toString()) +
|
||||
", affinityNode:" +
|
||||
(affinityNode == null ? "" : affinityNode.getNetworkFullPath()));
|
||||
|
@ -268,15 +268,12 @@ public final class SCMContainerPlacementRackAware extends SCMCommonPolicy {
|
|||
}
|
||||
}
|
||||
// there is no constrains to reduce or fallback is true
|
||||
throw new SCMException("No satisfied datanode to meet the " +
|
||||
throw new SCMException("No satisfied datanode to meet the" +
|
||||
" excludedNodes and affinityNode constrains.", null);
|
||||
}
|
||||
if (hasEnoughSpace((DatanodeDetails)node, sizeRequired)) {
|
||||
LOG.warn("Datanode {} is chosen. Required size is {}",
|
||||
LOG.debug("Datanode {} is chosen for container. Required size is {}",
|
||||
node.toString(), sizeRequired);
|
||||
if (excludedNodes != null && excludedNodesForCapacity != null) {
|
||||
excludedNodes.removeAll(excludedNodesForCapacity);
|
||||
}
|
||||
metrics.incrDatanodeChooseSuccessCount();
|
||||
if (isFallbacked) {
|
||||
metrics.incrDatanodeChooseFallbackCount();
|
||||
|
@ -294,12 +291,7 @@ public final class SCMContainerPlacementRackAware extends SCMCommonPolicy {
|
|||
if (excludedNodesForCapacity == null) {
|
||||
excludedNodesForCapacity = new ArrayList<>();
|
||||
}
|
||||
excludedNodesForCapacity.add(node);
|
||||
if (excludedNodes == null) {
|
||||
excludedNodes = excludedNodesForCapacity;
|
||||
} else {
|
||||
excludedNodes.add(node);
|
||||
}
|
||||
excludedNodesForCapacity.add(node.getNetworkFullPath());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,9 +33,13 @@ import org.apache.hadoop.hdds.scm.node.NodeManager;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import static org.apache.hadoop.hdds.scm.net.NetConstants.LEAF_SCHEMA;
|
||||
|
@ -43,16 +47,19 @@ import static org.apache.hadoop.hdds.scm.net.NetConstants.RACK_SCHEMA;
|
|||
import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT_SCHEMA;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.junit.Assume.assumeTrue;
|
||||
import static org.mockito.Matchers.anyObject;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* Test for the scm container rack aware placement.
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
public class TestSCMContainerPlacementRackAware {
|
||||
private NetworkTopology cluster;
|
||||
private Configuration conf;
|
||||
private NodeManager nodeManager;
|
||||
private Integer datanodeCount;
|
||||
private List<DatanodeDetails> datanodes = new ArrayList<>();
|
||||
// policy with fallback capability
|
||||
private SCMContainerPlacementRackAware policy;
|
||||
|
@ -61,6 +68,17 @@ public class TestSCMContainerPlacementRackAware {
|
|||
// node storage capacity
|
||||
private static final long STORAGE_CAPACITY = 100L;
|
||||
private SCMContainerPlacementMetrics metrics;
|
||||
private static final int NODE_PER_RACK = 5;
|
||||
|
||||
public TestSCMContainerPlacementRackAware(Integer count) {
|
||||
this.datanodeCount = count;
|
||||
}
|
||||
|
||||
@Parameterized.Parameters
|
||||
public static Collection<Object[]> setupDatanodes() {
|
||||
return Arrays.asList(new Object[][]{{3}, {4}, {5}, {6}, {7}, {8}, {9},
|
||||
{10}, {11}, {12}, {13}, {14}, {15}});
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
|
@ -74,10 +92,10 @@ public class TestSCMContainerPlacementRackAware {
|
|||
// build datanodes, and network topology
|
||||
String rack = "/rack";
|
||||
String hostname = "node";
|
||||
for (int i = 0; i < 15; i++) {
|
||||
for (int i = 0; i < datanodeCount; i++) {
|
||||
// Totally 3 racks, each has 5 datanodes
|
||||
DatanodeDetails node = TestUtils.createDatanodeDetails(
|
||||
hostname + i, rack + (i / 5));
|
||||
hostname + i, rack + (i / NODE_PER_RACK));
|
||||
datanodes.add(node);
|
||||
cluster.add(node);
|
||||
}
|
||||
|
@ -88,12 +106,22 @@ public class TestSCMContainerPlacementRackAware {
|
|||
.thenReturn(new ArrayList<>(datanodes));
|
||||
when(nodeManager.getNodeStat(anyObject()))
|
||||
.thenReturn(new SCMNodeMetric(STORAGE_CAPACITY, 0L, 100L));
|
||||
when(nodeManager.getNodeStat(datanodes.get(2)))
|
||||
.thenReturn(new SCMNodeMetric(STORAGE_CAPACITY, 90L, 10L));
|
||||
when(nodeManager.getNodeStat(datanodes.get(3)))
|
||||
.thenReturn(new SCMNodeMetric(STORAGE_CAPACITY, 80L, 20L));
|
||||
when(nodeManager.getNodeStat(datanodes.get(4)))
|
||||
.thenReturn(new SCMNodeMetric(STORAGE_CAPACITY, 70L, 30L));
|
||||
if (datanodeCount > 4) {
|
||||
when(nodeManager.getNodeStat(datanodes.get(2)))
|
||||
.thenReturn(new SCMNodeMetric(STORAGE_CAPACITY, 90L, 10L));
|
||||
when(nodeManager.getNodeStat(datanodes.get(3)))
|
||||
.thenReturn(new SCMNodeMetric(STORAGE_CAPACITY, 80L, 20L));
|
||||
when(nodeManager.getNodeStat(datanodes.get(4)))
|
||||
.thenReturn(new SCMNodeMetric(STORAGE_CAPACITY, 70L, 30L));
|
||||
} else if (datanodeCount > 3) {
|
||||
when(nodeManager.getNodeStat(datanodes.get(2)))
|
||||
.thenReturn(new SCMNodeMetric(STORAGE_CAPACITY, 90L, 10L));
|
||||
when(nodeManager.getNodeStat(datanodes.get(3)))
|
||||
.thenReturn(new SCMNodeMetric(STORAGE_CAPACITY, 80L, 20L));
|
||||
} else if (datanodeCount > 2) {
|
||||
when(nodeManager.getNodeStat(datanodes.get(2)))
|
||||
.thenReturn(new SCMNodeMetric(STORAGE_CAPACITY, 84L, 16L));
|
||||
}
|
||||
|
||||
// create placement policy instances
|
||||
metrics = SCMContainerPlacementMetrics.create();
|
||||
|
@ -118,12 +146,15 @@ public class TestSCMContainerPlacementRackAware {
|
|||
datanodeDetails = policy.chooseDatanodes(null, null, nodeNum, 15);
|
||||
Assert.assertEquals(nodeNum, datanodeDetails.size());
|
||||
Assert.assertTrue(cluster.isSameParent(datanodeDetails.get(0),
|
||||
datanodeDetails.get(1)));
|
||||
datanodeDetails.get(1)) || (datanodeCount % NODE_PER_RACK == 1));
|
||||
|
||||
// 3 replicas
|
||||
nodeNum = 3;
|
||||
datanodeDetails = policy.chooseDatanodes(null, null, nodeNum, 15);
|
||||
Assert.assertEquals(nodeNum, datanodeDetails.size());
|
||||
// requires at least 2 racks for following statement
|
||||
assumeTrue(datanodeCount > NODE_PER_RACK &&
|
||||
datanodeCount % NODE_PER_RACK > 1);
|
||||
Assert.assertTrue(cluster.isSameParent(datanodeDetails.get(0),
|
||||
datanodeDetails.get(1)));
|
||||
Assert.assertFalse(cluster.isSameParent(datanodeDetails.get(0),
|
||||
|
@ -135,6 +166,8 @@ public class TestSCMContainerPlacementRackAware {
|
|||
nodeNum = 4;
|
||||
datanodeDetails = policy.chooseDatanodes(null, null, nodeNum, 15);
|
||||
Assert.assertEquals(nodeNum, datanodeDetails.size());
|
||||
// requires at least 2 racks and enough datanodes for following statement
|
||||
assumeTrue(datanodeCount > NODE_PER_RACK + 1);
|
||||
Assert.assertTrue(cluster.isSameParent(datanodeDetails.get(0),
|
||||
datanodeDetails.get(1)));
|
||||
Assert.assertFalse(cluster.isSameParent(datanodeDetails.get(0),
|
||||
|
@ -147,6 +180,7 @@ public class TestSCMContainerPlacementRackAware {
|
|||
public void chooseNodeWithExcludedNodes() throws SCMException {
|
||||
// test choose new datanodes for under replicated pipeline
|
||||
// 3 replicas, two existing datanodes on same rack
|
||||
assumeTrue(datanodeCount > NODE_PER_RACK);
|
||||
int nodeNum = 1;
|
||||
List<DatanodeDetails> excludedNodes = new ArrayList<>();
|
||||
|
||||
|
@ -160,10 +194,10 @@ public class TestSCMContainerPlacementRackAware {
|
|||
Assert.assertFalse(cluster.isSameParent(datanodeDetails.get(0),
|
||||
excludedNodes.get(1)));
|
||||
|
||||
// 3 replicas, two existing datanodes on different rack
|
||||
// 3 replicas, one existing datanode
|
||||
nodeNum = 2;
|
||||
excludedNodes.clear();
|
||||
excludedNodes.add(datanodes.get(0));
|
||||
excludedNodes.add(datanodes.get(7));
|
||||
datanodeDetails = policy.chooseDatanodes(
|
||||
excludedNodes, null, nodeNum, 15);
|
||||
Assert.assertEquals(nodeNum, datanodeDetails.size());
|
||||
|
@ -171,10 +205,11 @@ public class TestSCMContainerPlacementRackAware {
|
|||
datanodeDetails.get(0), excludedNodes.get(0)) ||
|
||||
cluster.isSameParent(datanodeDetails.get(0), excludedNodes.get(1)));
|
||||
|
||||
// 3 replicas, one existing datanode
|
||||
nodeNum = 2;
|
||||
// 3 replicas, two existing datanodes on different rack
|
||||
nodeNum = 1;
|
||||
excludedNodes.clear();
|
||||
excludedNodes.add(datanodes.get(0));
|
||||
excludedNodes.add(datanodes.get(5));
|
||||
datanodeDetails = policy.chooseDatanodes(
|
||||
excludedNodes, null, nodeNum, 15);
|
||||
Assert.assertEquals(nodeNum, datanodeDetails.size());
|
||||
|
@ -188,6 +223,8 @@ public class TestSCMContainerPlacementRackAware {
|
|||
// 5 replicas. there are only 3 racks. policy with fallback should
|
||||
// allocate the 5th datanode though it will break the rack rule(first
|
||||
// 2 replicas on same rack, others on different racks).
|
||||
assumeTrue(datanodeCount > NODE_PER_RACK * 2 &&
|
||||
(datanodeCount % NODE_PER_RACK > 1));
|
||||
int nodeNum = 5;
|
||||
List<DatanodeDetails> datanodeDetails =
|
||||
policy.chooseDatanodes(null, null, nodeNum, 15);
|
||||
|
@ -218,6 +255,8 @@ public class TestSCMContainerPlacementRackAware {
|
|||
|
||||
@Test
|
||||
public void testNoFallback() throws SCMException {
|
||||
assumeTrue(datanodeCount > (NODE_PER_RACK * 2) &&
|
||||
(datanodeCount <= NODE_PER_RACK * 3));
|
||||
// 5 replicas. there are only 3 racks. policy prohibit fallback should fail.
|
||||
int nodeNum = 5;
|
||||
try {
|
||||
|
|
Loading…
Reference in New Issue