瀏覽代碼

HDDS-1879. Support multiple excluded scopes when choosing datanodes in NetworkTopology (#1194)

Sammi Chen 5 年之前
父節點
當前提交
4222b62f2b

+ 3 - 2
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/InnerNode.java

@@ -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);
 }

+ 35 - 21
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/InnerNodeImpl.java

@@ -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;
+    }
+
+    for (String scope: excludedScopes) {
+      Node excludedScopeNode = getNode(scope);
+      nodeCounts.put(scope, excludedScopeNode == null ? 0 :
+          excludedScopeNode.getNumOfLeaves());
     }
-    Node excludedScopeNode = getNode(excludedScope);
-    return excludedScopeNode == null ? 0 : excludedScopeNode.getNumOfLeaves();
+    return nodeCounts;
   }
 }

+ 23 - 18
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetUtils.java

@@ -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;
     }

+ 6 - 29
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopology.java

@@ -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

+ 68 - 68
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java

@@ -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);
+        }
+      });
     }
   }
 

+ 51 - 23
hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/net/TestNetworkTopologyImpl.java

@@ -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);

+ 2 - 6
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementMetrics.java

@@ -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);
   }
 

+ 7 - 15
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackAware.java

@@ -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());
       }
     }
   }

+ 52 - 13
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java

@@ -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 {