Bladeren bron

YARN-2694. Ensure only single node label specified in ResourceRequest. Contributed by Wangda Tan
(cherry picked from commit c1957fef29b07fea70938e971b30532a1e131fd0)

Jian He 10 jaren geleden
bovenliggende
commit
3ddafaa7c8
17 gewijzigde bestanden met toevoegingen van 355 en 151 verwijderingen
  1. 3 0
      hadoop-yarn-project/CHANGES.txt
  2. 12 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
  3. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
  4. 36 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
  5. 13 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
  6. 47 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
  7. 11 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
  8. 40 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
  9. 61 43
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
  10. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  11. 21 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
  12. 11 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
  13. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
  14. 23 27
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
  15. 46 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
  16. 15 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
  17. 8 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -467,6 +467,9 @@ Release 2.7.0 - UNRELEASED
     YARN-1537. Fix race condition in
     TestLocalResourcesTrackerImpl.testLocalResourceCache. (xgong via acmurthy)
 
+    YARN-2694. Ensure only single node label specified in ResourceRequest.
+    (Wangda Tan via jianhe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

+ 12 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java

@@ -253,24 +253,27 @@ public abstract class ResourceRequest implements Comparable<ResourceRequest> {
   /**
    * Get node-label-expression for this Resource Request. If this is set, all
    * containers allocated to satisfy this resource-request will be only on those
-   * nodes that satisfy this node-label-expression
+   * nodes that satisfy this node-label-expression.
+   *  
+   * Please note that node label expression now can only take effect when the
+   * resource request has resourceName = ANY
    * 
    * @return node-label-expression
    */
   @Public
   @Evolving
-  public abstract String getNodeLabelExpression(); 
+  public abstract String getNodeLabelExpression();
   
   /**
-   * Set node label expression of this resource request. Now only
-   * support AND(&&), in the future will provide support for OR(||), NOT(!).
+   * Set node label expression of this resource request. Now only support
+   * specifying a single node label. In the future we will support more complex
+   * node label expression specification like AND(&&), OR(||), etc.
    * 
-   * Examples: 
-   * - GPU && LARGE_MEM, ask for node has label GPU and LARGE_MEM together
-   * - "" (empty) means ask for node doesn't have label on it, this is default
-   *   behavior
+   * Any please note that node label expression now can only take effect when
+   * the resource request has resourceName = ANY
    * 
-   * @param nodelabelExpression node-label-expression of this ResourceRequest
+   * @param nodelabelExpression
+   *          node-label-expression of this ResourceRequest
    */
   @Public
   @Evolving

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java

@@ -169,7 +169,8 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
      *          If true, containers for this request may be assigned on hosts
      *          and racks other than the ones explicitly requested.
      * @param nodeLabelsExpression
-     *          Set node labels to allocate resource
+     *          Set node labels to allocate resource, now we only support
+     *          asking for only a single node label
      */
     public ContainerRequest(Resource capability, String[] nodes,
         String[] racks, Priority priority, boolean relaxLocality,

+ 36 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java

@@ -421,6 +421,8 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     checkLocalityRelaxationConflict(req.getPriority(), dedupedRacks, true);
     checkLocalityRelaxationConflict(req.getPriority(), inferredRacks,
         req.getRelaxLocality());
+    // check if the node label expression specified is valid
+    checkNodeLabelExpression(req);
 
     if (req.getNodes() != null) {
       HashSet<String> dedupedNodes = new HashSet<String>(req.getNodes());
@@ -586,6 +588,37 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
       }
   }
   
+  /**
+   * Valid if a node label expression specified on container request is valid or
+   * not
+   * 
+   * @param containerRequest
+   */
+  private void checkNodeLabelExpression(T containerRequest) {
+    String exp = containerRequest.getNodeLabelExpression();
+    
+    if (null == exp || exp.isEmpty()) {
+      return;
+    }
+
+    // Don't support specifying >= 2 node labels in a node label expression now
+    if (exp.contains("&&") || exp.contains("||")) {
+      throw new InvalidContainerRequestException(
+          "Cannot specify more than two node labels"
+              + " in a single node label expression");
+    }
+    
+    // Don't allow specify node label against ANY request
+    if ((containerRequest.getRacks() != null && 
+        (!containerRequest.getRacks().isEmpty()))
+        || 
+        (containerRequest.getNodes() != null && 
+        (!containerRequest.getNodes().isEmpty()))) {
+      throw new InvalidContainerRequestException(
+          "Cannot specify node label with rack and node");
+    }
+  }
+  
   private void addResourceRequestToAsk(ResourceRequest remoteRequest) {
     // This code looks weird but is needed because of the following scenario.
     // A ResourceRequest is removed from the remoteRequestTable. A 0 container 
@@ -640,7 +673,9 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
       resourceRequestInfo.containerRequests.add(req);
     }
     
-    resourceRequestInfo.remoteRequest.setNodeLabelExpression(labelExpression);
+    if (ResourceRequest.ANY.equals(resourceName)) {
+      resourceRequestInfo.remoteRequest.setNodeLabelExpression(labelExpression);
+    }
 
     // Note this down for next interaction with ResourceManager
     addResourceRequestToAsk(resourceRequestInfo.remoteRequest);

+ 13 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLa
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 
 @Private
@@ -102,7 +103,9 @@ public class RMAdminCLI extends HAAdmin {
           .put("-replaceLabelsOnNode",
               new UsageInfo(
                   "[node1[:port]=label1,label2 node2[:port]=label1,label2]",
-                  "replace labels on nodes"))
+                  "replace labels on nodes"
+                      + " (please note that we do not support specifying multiple"
+                      + " labels on a single host for now.)"))
           .put("-directlyAccessNodeLabelStore",
               new UsageInfo("", "Directly access node label store, "
                   + "with this option, all node label related operations"
@@ -389,8 +392,7 @@ public class RMAdminCLI extends HAAdmin {
     return 0;
   }
   
-  private Map<NodeId, Set<String>> buildNodeLabelsMapFromStr(String args)
-      throws IOException {
+  private Map<NodeId, Set<String>> buildNodeLabelsMapFromStr(String args) {
     Map<NodeId, Set<String>> map = new HashMap<NodeId, Set<String>>();
 
     for (String nodeToLabels : args.split("[ \n]")) {
@@ -411,10 +413,9 @@ public class RMAdminCLI extends HAAdmin {
       if (index == 0) {
         splits = splits[1].split(",");
       }
-
-      if (nodeIdStr.trim().isEmpty()) {
-        throw new IOException("node name cannot be empty");
-      }
+      
+      Preconditions.checkArgument(!nodeIdStr.trim().isEmpty(),
+          "node name cannot be empty");
 
       NodeId nodeId = ConverterUtils.toNodeIdWithDefaultPort(nodeIdStr);
       map.put(nodeId, new HashSet<String>());
@@ -424,6 +425,11 @@ public class RMAdminCLI extends HAAdmin {
           map.get(nodeId).add(splits[i].trim());
         }
       }
+      
+      int nLabels = map.get(nodeId).size();
+      Preconditions.checkArgument(nLabels <= 1, "%d labels specified on host=%s"
+          + ", please note that we do not support specifying multiple"
+          + " labels on a single host for now.", nLabels, nodeIdStr);
     }
 
     if (map.isEmpty()) {

+ 47 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java

@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.client.api.impl;
 
-import com.google.common.base.Supplier;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -40,7 +38,6 @@ import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
@@ -75,6 +72,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
+import org.apache.hadoop.yarn.client.api.InvalidContainerRequestException;
 import org.apache.hadoop.yarn.client.api.NMTokenCache;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -89,6 +87,7 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -96,6 +95,8 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.mortbay.log.Log;
 
+import com.google.common.base.Supplier;
+
 public class TestAMRMClient {
   static Configuration conf = null;
   static MiniYARNCluster yarnCluster = null;
@@ -148,7 +149,6 @@ public class TestAMRMClient {
     racks = new String[]{ rack };
   }
   
-  @SuppressWarnings("deprecation")
   @Before
   public void startApp() throws Exception {
     // submit new app
@@ -678,21 +678,57 @@ public class TestAMRMClient {
     AMRMClientImpl<ContainerRequest> client =
         new AMRMClientImpl<ContainerRequest>();
 
-    // add x, y to ANY
+    // add exp=x to ANY
     client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
-        1), null, null, Priority.UNDEFINED, true, "x && y"));
+        1), null, null, Priority.UNDEFINED, true, "x"));
     Assert.assertEquals(1, client.ask.size());
-    Assert.assertEquals("x && y", client.ask.iterator().next()
+    Assert.assertEquals("x", client.ask.iterator().next()
         .getNodeLabelExpression());
 
-    // add x, y and a, b to ANY, only a, b should be kept
+    // add exp=x then add exp=a to ANY in same priority, only exp=a should kept
     client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
-        1), null, null, Priority.UNDEFINED, true, "x && y"));
+        1), null, null, Priority.UNDEFINED, true, "x"));
     client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
-        1), null, null, Priority.UNDEFINED, true, "a && b"));
+        1), null, null, Priority.UNDEFINED, true, "a"));
     Assert.assertEquals(1, client.ask.size());
-    Assert.assertEquals("a && b", client.ask.iterator().next()
+    Assert.assertEquals("a", client.ask.iterator().next()
         .getNodeLabelExpression());
+    
+    // add exp=x to ANY, rack and node, only resource request has ANY resource
+    // name will be assigned the label expression
+    // add exp=x then add exp=a to ANY in same priority, only exp=a should kept
+    client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
+        1), null, null, Priority.UNDEFINED, true,
+        "y"));
+    Assert.assertEquals(1, client.ask.size());
+    for (ResourceRequest req : client.ask) {
+      if (ResourceRequest.ANY.equals(req.getResourceName())) {
+        Assert.assertEquals("y", req.getNodeLabelExpression());
+      } else {
+        Assert.assertNull(req.getNodeLabelExpression());
+      }
+    }
+  }
+  
+  private void verifyAddRequestFailed(AMRMClient<ContainerRequest> client,
+      ContainerRequest request) {
+    try {
+      client.addContainerRequest(request);
+    } catch (InvalidContainerRequestException e) {
+      return;
+    }
+    Assert.fail();
+  }
+  
+  @Test(timeout=30000)
+  public void testAskWithInvalidNodeLabels() {
+    AMRMClientImpl<ContainerRequest> client =
+        new AMRMClientImpl<ContainerRequest>();
+
+    // specified exp with more than one node labels
+    verifyAddRequestFailed(client,
+        new ContainerRequest(Resource.newInstance(1024, 1), null, null,
+            Priority.UNDEFINED, true, "x && y"));
   }
     
   private void testAllocation(final AMRMClientImpl<ContainerRequest> amClient)  

+ 11 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java

@@ -512,7 +512,7 @@ public class TestRMAdminCLI {
         .addToCluserNodeLabels(ImmutableSet.of("x", "y", "Y"));
     String[] args =
         { "-replaceLabelsOnNode",
-            "node1:8000,x,y node2:8000=y node3,x,Y node4=Y",
+            "node1:8000,x node2:8000=y node3,x node4=Y",
             "-directlyAccessNodeLabelStore" };
     assertEquals(0, rmAdminCLI.run(args));
     assertTrue(dummyNodeLabelsManager.getNodeLabels().containsKey(
@@ -540,6 +540,16 @@ public class TestRMAdminCLI {
     args = new String[] { "-replaceLabelsOnNode", ", " };
     assertTrue(0 != rmAdminCLI.run(args));
   }
+  
+  @Test
+  public void testReplaceMultipleLabelsOnSingleNode() throws Exception {
+    // Successfully replace labels
+    dummyNodeLabelsManager.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    String[] args =
+        { "-replaceLabelsOnNode", "node1,x,y",
+            "-directlyAccessNodeLabelStore" };
+    assertTrue(0 != rmAdminCLI.run(args));
+  }
 
   private void testError(String[] args, String template,
       ByteArrayOutputStream data, int resultCode) throws Exception {

+ 40 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java

@@ -45,11 +45,12 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.nodelabels.event.StoreNewClusterNodeLabels;
 import org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent;
 import org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEventType;
 import org.apache.hadoop.yarn.nodelabels.event.RemoveClusterNodeLabels;
+import org.apache.hadoop.yarn.nodelabels.event.StoreNewClusterNodeLabels;
 import org.apache.hadoop.yarn.nodelabels.event.UpdateNodeToLabelsMappingsEvent;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -309,14 +310,35 @@ public class CommonNodeLabelsManager extends AbstractService {
     // check all labels being added existed
     Set<String> knownLabels = labelCollections.keySet();
     for (Entry<NodeId, Set<String>> entry : addedLabelsToNode.entrySet()) {
-      if (!knownLabels.containsAll(entry.getValue())) {
+      NodeId nodeId = entry.getKey();
+      Set<String> labels = entry.getValue();
+      
+      if (!knownLabels.containsAll(labels)) {
         String msg =
             "Not all labels being added contained by known "
                 + "label collections, please check" + ", added labels=["
-                + StringUtils.join(entry.getValue(), ",") + "]";
+                + StringUtils.join(labels, ",") + "]";
         LOG.error(msg);
         throw new IOException(msg);
       }
+      
+      // In YARN-2694, we temporarily disable user add more than 1 labels on a
+      // same host
+      if (!labels.isEmpty()) {
+        Set<String> newLabels = new HashSet<String>(getLabelsByNode(nodeId));
+        newLabels.addAll(labels);
+        // we don't allow number of labels on a node > 1 after added labels
+        if (newLabels.size() > 1) {
+          String msg =
+              String.format(
+                      "%d labels specified on host=%s after add labels to node"
+                          + ", please note that we do not support specifying multiple"
+                          + " labels on a single host for now.",
+                      newLabels.size(), nodeId.getHost());
+          LOG.error(msg);
+          throw new IOException(msg);
+        }
+      }
     }
   }
   
@@ -620,11 +642,24 @@ public class CommonNodeLabelsManager extends AbstractService {
     // check all labels being added existed
     Set<String> knownLabels = labelCollections.keySet();
     for (Entry<NodeId, Set<String>> entry : replaceLabelsToNode.entrySet()) {
-      if (!knownLabels.containsAll(entry.getValue())) {
+      NodeId nodeId = entry.getKey();
+      Set<String> labels = entry.getValue();
+      
+      // As in YARN-2694, we disable user add more than 1 labels on a same host
+      if (labels.size() > 1) {
+        String msg = String.format("%d labels specified on host=%s"
+            + ", please note that we do not support specifying multiple"
+            + " labels on a single host for now.", labels.size(),
+            nodeId.getHost());
+        LOG.error(msg);
+        throw new IOException(msg);
+      }
+      
+      if (!knownLabels.containsAll(labels)) {
         String msg =
             "Not all labels being replaced contained by known "
                 + "label collections, please check" + ", new labels=["
-                + StringUtils.join(entry.getValue(), ",") + "]";
+                + StringUtils.join(labels, ",") + "]";
         LOG.error(msg);
         throw new IOException(msg);
       }

+ 61 - 43
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java

@@ -227,19 +227,17 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
         ImmutableMap.of(toNodeId("n1"), CommonNodeLabelsManager.EMPTY_STRING_SET));
 
     // add label on node
-    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p1"),
-        toNodeId("n2"), toSet("p2")));
+    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p1")));
     assertMapEquals(
         mgr.getNodeLabels(),
         ImmutableMap.of(toNodeId("n1"), toSet("p1"), toNodeId("n2"),
-            toSet("p2", "p3"), toNodeId("n3"), toSet("p3")));
+            toSet("p3"), toNodeId("n3"), toSet("p3")));
     assertMapEquals(mgr.lastNodeToLabels,
-        ImmutableMap.of(toNodeId("n1"), toSet("p1"), toNodeId("n2"),
-            toSet("p2", "p3")));
+        ImmutableMap.of(toNodeId("n1"), toSet("p1")));
 
     // remove labels on node
     mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("n1"), toSet("p1"),
-        toNodeId("n2"), toSet("p2", "p3"), toNodeId("n3"), toSet("p3")));
+        toNodeId("n2"), toSet("p3"), toNodeId("n3"), toSet("p3")));
     Assert.assertEquals(0, mgr.getNodeLabels().size());
     assertMapEquals(mgr.lastNodeToLabels, ImmutableMap.of(toNodeId("n1"),
         CommonNodeLabelsManager.EMPTY_STRING_SET, toNodeId("n2"),
@@ -276,10 +274,10 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
   @Test(timeout = 5000) 
   public void testTrimLabelsWhenModifyLabelsOnNodes() throws IOException {
     mgr.addToCluserNodeLabels(toSet(" p1", "p2"));
-    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p1 ", "p2")));
+    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p1 ")));
     assertMapEquals(
         mgr.getNodeLabels(),
-        ImmutableMap.of(toNodeId("n1"), toSet("p1", "p2")));
+        ImmutableMap.of(toNodeId("n1"), toSet("p1")));
     mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet(" p2")));
     assertMapEquals(
         mgr.getNodeLabels(),
@@ -292,16 +290,16 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
   public void testReplaceLabelsOnHostsShouldUpdateNodesBelongTo()
       throws IOException {
     mgr.addToCluserNodeLabels(toSet("p1", "p2", "p3"));
-    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p1", "p2")));
+    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p1")));
     assertMapEquals(
         mgr.getNodeLabels(),
-        ImmutableMap.of(toNodeId("n1"), toSet("p1", "p2")));
+        ImmutableMap.of(toNodeId("n1"), toSet("p1")));
     
     // Replace labels on n1:1 to P2
     mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p2"),
         toNodeId("n1:2"), toSet("p2")));
     assertMapEquals(mgr.getNodeLabels(), ImmutableMap.of(toNodeId("n1"),
-        toSet("p1", "p2"), toNodeId("n1:1"), toSet("p2"), toNodeId("n1:2"),
+        toSet("p1"), toNodeId("n1:1"), toSet("p2"), toNodeId("n1:2"),
         toSet("p2")));
     
     // Replace labels on n1 to P1, both n1:1/n1 will be P1 now
@@ -312,17 +310,6 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
     
     // Set labels on n1:1 to P2 again to verify if add/remove works
     mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p2")));
-    // Add p3 to n1, should makes n1:1 to be p2/p3, and n1:2 to be p1/p3
-    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p3")));
-    assertMapEquals(mgr.getNodeLabels(), ImmutableMap.of(toNodeId("n1"),
-        toSet("p1", "p3"), toNodeId("n1:1"), toSet("p2", "p3"),
-        toNodeId("n1:2"), toSet("p1", "p3")));
-    
-    // Remove P3 from n1, should makes n1:1 to be p2, and n1:2 to be p1
-    mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("n1"), toSet("p3")));
-    assertMapEquals(mgr.getNodeLabels(), ImmutableMap.of(toNodeId("n1"),
-        toSet("p1"), toNodeId("n1:1"), toSet("p2"), toNodeId("n1:2"),
-        toSet("p1")));
   }
 
   private void assertNodeLabelsDisabledErrorMessage(IOException e) {
@@ -404,13 +391,12 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
   public void testLabelsToNodes()
       throws IOException {
     mgr.addToCluserNodeLabels(toSet("p1", "p2", "p3"));
-    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p1", "p2")));
+    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p1")));
     Map<String, Set<NodeId>> labelsToNodes = mgr.getLabelsToNodes();
     assertLabelsToNodesEquals(
         labelsToNodes,
         ImmutableMap.of(
-        "p1", toSet(toNodeId("n1")),
-        "p2",toSet(toNodeId("n1"))));
+        "p1", toSet(toNodeId("n1"))));
     assertLabelsToNodesEquals(
         labelsToNodes, transposeNodeToLabels(mgr.getNodeLabels()));
 
@@ -422,7 +408,7 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
         labelsToNodes,
         ImmutableMap.of(
         "p1", toSet(toNodeId("n1")),
-        "p2", toSet(toNodeId("n1"),toNodeId("n1:1"),toNodeId("n1:2"))));
+        "p2", toSet(toNodeId("n1:1"),toNodeId("n1:2"))));
     assertLabelsToNodesEquals(
         labelsToNodes, transposeNodeToLabels(mgr.getNodeLabels()));
 
@@ -439,19 +425,19 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
     // Set labels on n1:1 to P2 again to verify if add/remove works
     mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p2")));
     // Add p3 to n1, should makes n1:1 to be p2/p3, and n1:2 to be p1/p3
-    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p3")));
+    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n2"), toSet("p3")));
     labelsToNodes = mgr.getLabelsToNodes();
     assertLabelsToNodesEquals(
         labelsToNodes,
         ImmutableMap.of(
         "p1", toSet(toNodeId("n1"),toNodeId("n1:2")),
         "p2", toSet(toNodeId("n1:1")),
-        "p3", toSet(toNodeId("n1"),toNodeId("n1:1"),toNodeId("n1:2"))));
+        "p3", toSet(toNodeId("n2"))));
     assertLabelsToNodesEquals(
         labelsToNodes, transposeNodeToLabels(mgr.getNodeLabels()));
 
     // Remove P3 from n1, should makes n1:1 to be p2, and n1:2 to be p1
-    mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("n1"), toSet("p3")));
+    mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("n2"), toSet("p3")));
     labelsToNodes = mgr.getLabelsToNodes();
     assertLabelsToNodesEquals(
         labelsToNodes,
@@ -468,14 +454,14 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
     mgr.addToCluserNodeLabels(toSet("p1", "p2", "p3"));
     mgr.addLabelsToNode(
         ImmutableMap.of(
-        toNodeId("n1:1"), toSet("p1", "p2"),
-        toNodeId("n1:2"), toSet("p1", "p2")));
+        toNodeId("n1:1"), toSet("p1"),
+        toNodeId("n1:2"), toSet("p2")));
     Set<String> setlabels =
         new HashSet<String>(Arrays.asList(new String[]{"p1"}));
     assertLabelsToNodesEquals(mgr.getLabelsToNodes(setlabels),
-        ImmutableMap.of("p1", toSet(toNodeId("n1:1"),toNodeId("n1:2"))));
+        ImmutableMap.of("p1", toSet(toNodeId("n1:1"))));
 
-    // Replace labels on n1:1 to P2
+    // Replace labels on n1:1 to P3
     mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p3")));
     assertTrue(mgr.getLabelsToNodes(setlabels).isEmpty());
     setlabels = new HashSet<String>(Arrays.asList(new String[]{"p2", "p3"}));
@@ -484,11 +470,11 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
         ImmutableMap.of(
         "p3", toSet(toNodeId("n1"), toNodeId("n1:1"),toNodeId("n1:2"))));
 
-    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1:3"), toSet("p1", "p2")));
+    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n2"), toSet("p2")));
     assertLabelsToNodesEquals(
         mgr.getLabelsToNodes(setlabels),
         ImmutableMap.of(
-        "p2", toSet(toNodeId("n1:3")),
+        "p2", toSet(toNodeId("n2")),
         "p3", toSet(toNodeId("n1"), toNodeId("n1:1"),toNodeId("n1:2"))));
 
     mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("n1"), toSet("p3")));
@@ -497,25 +483,57 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
     assertLabelsToNodesEquals(
         mgr.getLabelsToNodes(setlabels),
         ImmutableMap.of(
-        "p1", toSet(toNodeId("n1:3")),
-        "p2", toSet(toNodeId("n1:3"))));
+        "p2", toSet(toNodeId("n2"))));
 
-    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n2:2"), toSet("p1", "p2")));
+    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n3"), toSet("p1")));
     assertLabelsToNodesEquals(
         mgr.getLabelsToNodes(setlabels),
         ImmutableMap.of(
-        "p1", toSet(toNodeId("n1:3"), toNodeId("n2:2")),
-        "p2", toSet(toNodeId("n1:3"), toNodeId("n2:2"))));
+        "p1", toSet(toNodeId("n3")),
+        "p2", toSet(toNodeId("n2"))));
 
     mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n2:2"), toSet("p3")));
     assertLabelsToNodesEquals(
         mgr.getLabelsToNodes(setlabels),
         ImmutableMap.of(
-        "p1", toSet(toNodeId("n1:3")),
-        "p2", toSet(toNodeId("n1:3")),
+        "p1", toSet(toNodeId("n3")),
+        "p2", toSet(toNodeId("n2")),
         "p3", toSet(toNodeId("n2:2"))));
     setlabels = new HashSet<String>(Arrays.asList(new String[]{"p1"}));
     assertLabelsToNodesEquals(mgr.getLabelsToNodes(setlabels),
-        ImmutableMap.of("p1", toSet(toNodeId("n1:3"))));
+        ImmutableMap.of("p1", toSet(toNodeId("n3"))));
+  }
+
+  @Test(timeout = 5000)
+  public void testNoMoreThanOneLabelExistedInOneHost() throws IOException {
+    boolean failed = false;
+    // As in YARN-2694, we temporarily disable no more than one label existed in
+    // one host
+    mgr.addToCluserNodeLabels(toSet("p1", "p2", "p3"));
+    try {
+      mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p1", "p2")));
+    } catch (IOException e) {
+      failed = true;
+    }
+    Assert.assertTrue("Should failed when set > 1 labels on a host", failed);
+
+    try {
+      mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p1", "p2")));
+    } catch (IOException e) {
+      failed = true;
+    }
+    Assert.assertTrue("Should failed when add > 1 labels on a host", failed);
+
+    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p1")));
+    // add a same label to a node, #labels in this node is still 1, shouldn't
+    // fail
+    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p1")));
+    try {
+      mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p2")));
+    } catch (IOException e) {
+      failed = true;
+    }
+    Assert.assertTrue("Should failed when #labels > 1 on a host after add",
+        failed);
   }
 }

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java

@@ -488,10 +488,11 @@ public class ApplicationMasterService extends AbstractService implements
       RMApp app =
           this.rmContext.getRMApps().get(applicationId);
       
-      // set label expression for Resource Requests
+      // set label expression for Resource Requests if resourceName=ANY 
       ApplicationSubmissionContext asc = app.getApplicationSubmissionContext();
       for (ResourceRequest req : ask) {
-        if (null == req.getNodeLabelExpression()) {
+        if (null == req.getNodeLabelExpression()
+            && ResourceRequest.ANY.equals(req.getResourceName())) {
           req.setNodeLabelExpression(asc.getNodeLabelExpression());
         }
       }

+ 21 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java

@@ -231,11 +231,31 @@ public class SchedulerUtils {
     
     // if queue has default label expression, and RR doesn't have, use the
     // default label expression of queue
-    if (labelExp == null && queueInfo != null) {
+    if (labelExp == null && queueInfo != null
+        && ResourceRequest.ANY.equals(resReq.getResourceName())) {
       labelExp = queueInfo.getDefaultNodeLabelExpression();
       resReq.setNodeLabelExpression(labelExp);
     }
     
+    // we don't allow specify label expression other than resourceName=ANY now
+    if (!ResourceRequest.ANY.equals(resReq.getResourceName())
+        && labelExp != null && !labelExp.trim().isEmpty()) {
+      throw new InvalidResourceRequestException(
+          "Invailid resource request, queue=" + queueInfo.getQueueName()
+              + " specified node label expression in a "
+              + "resource request has resource name = "
+              + resReq.getResourceName());
+    }
+    
+    // we don't allow specify label expression with more than one node labels now
+    if (labelExp != null && labelExp.contains("&&")) {
+      throw new InvalidResourceRequestException(
+          "Invailid resource request, queue=" + queueInfo.getQueueName()
+              + " specified more than one node label "
+              + "in a node label expression, node label expression = "
+              + labelExp);
+    }
+    
     if (labelExp != null && !labelExp.trim().isEmpty() && queueInfo != null) {
       if (!checkQueueLabelExpression(queueInfo.getAccessibleNodeLabels(),
           labelExp)) {

+ 11 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java

@@ -165,12 +165,17 @@ public class MockAM {
       int containers, String labelExpression) throws Exception {
     List<ResourceRequest> reqs = new ArrayList<ResourceRequest>();
     for (String host : hosts) {
-      ResourceRequest hostReq = createResourceReq(host, memory, priority,
-          containers, labelExpression);
-      reqs.add(hostReq);
-      ResourceRequest rackReq = createResourceReq("/default-rack", memory,
-          priority, containers, labelExpression);
-      reqs.add(rackReq);
+      // only add host/rack request when asked host isn't ANY
+      if (!host.equals(ResourceRequest.ANY)) {
+        ResourceRequest hostReq =
+            createResourceReq(host, memory, priority, containers,
+                labelExpression);
+        reqs.add(hostReq);
+        ResourceRequest rackReq =
+            createResourceReq("/default-rack", memory, priority, containers,
+                labelExpression);
+        reqs.add(rackReq);
+      }
     }
 
     ResourceRequest offRackReq = createResourceReq(ResourceRequest.ANY, memory,

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java

@@ -2104,7 +2104,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     nodeLabelManager.removeFromClusterNodeLabels(toSet("z"));
     
     // Replace nodelabel h1->x,y
-    nodeLabelManager.replaceLabelsOnNode(ImmutableMap.of(n1, toSet("x", "y")));
+    nodeLabelManager.replaceLabelsOnNode(ImmutableMap.of(n1, toSet("y")));
 
     // Wait for updating store.It is expected NodeStore update should happen
     // very fast since it has separate dispatcher. So waiting for max 5 seconds,
@@ -2122,7 +2122,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
     Map<NodeId, Set<String>> nodeLabels = nodeLabelManager.getNodeLabels();
     Assert.assertEquals(1, nodeLabelManager.getNodeLabels().size());
-    Assert.assertTrue(nodeLabels.get(n1).equals(toSet("x", "y")));
+    Assert.assertTrue(nodeLabels.get(n1).equals(toSet("y")));
 
     MockRM rm2 = new MockRM(conf, memStore) {
       @Override
@@ -2141,7 +2141,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
     nodeLabels = nodeLabelManager.getNodeLabels();
     Assert.assertEquals(1, nodeLabelManager.getNodeLabels().size());
-    Assert.assertTrue(nodeLabels.get(n1).equals(toSet("x", "y")));
+    Assert.assertTrue(nodeLabels.get(n1).equals(toSet("y")));
     rm1.stop();
     rm2.stop();
   }

+ 23 - 27
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java

@@ -211,16 +211,14 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
     
     /*
      * Node->Labels:
-     *   host1 : red, blue
-     *   host2 : blue, yellow
+     *   host1 : red
+     *   host2 : blue
      *   host3 : yellow
      *   host4 :
      */
     mgr.addToCluserNodeLabels(toSet("red", "blue", "yellow"));
-    mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("host1"),
-        toSet("red", "blue")));
-    mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("host2"),
-        toSet("blue", "yellow")));
+    mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("host1"), toSet("red")));
+    mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("host2"), toSet("blue")));
     mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("host3"), toSet("yellow")));
     
     // active two NM to n1, one large and one small
@@ -248,31 +246,29 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
     // check resource
     Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
         mgr.getQueueResource("Q1", q1Label, clusterResource));
-    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 4),
-        mgr.getQueueResource("Q2", q2Label, clusterResource));
     Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
+        mgr.getQueueResource("Q2", q2Label, clusterResource));
+    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2),
         mgr.getQueueResource("Q3", q3Label, clusterResource));
     Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 1),
         mgr.getQueueResource("Q4", q4Label, clusterResource));
     Assert.assertEquals(clusterResource,
         mgr.getQueueResource("Q5", q5Label, clusterResource));
     
-    mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("host1"), toSet("red"),
-        toNodeId("host2"), toSet("blue", "yellow")));
-    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("host3"), toSet("red")));
+    mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("host2"), toSet("blue")));
     /*
      * Check resource after changes some labels
      * Node->Labels:
-     *   host1 : blue (was: red, blue)
-     *   host2 : (was: blue, yellow)
-     *   host3 : red, yellow (was: yellow)
+     *   host1 : red
+     *   host2 : (was: blue)
+     *   host3 : yellow
      *   host4 :
      */
     
     // check resource
-    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 4),
+    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
         mgr.getQueueResource("Q1", q1Label, clusterResource));
-    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 4),
+    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
         mgr.getQueueResource("Q2", q2Label, clusterResource));
     Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
         mgr.getQueueResource("Q3", q3Label, clusterResource));
@@ -284,9 +280,9 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
     /*
      * Check resource after deactive/active some nodes 
      * Node->Labels:
-     *   (deactived) host1 : blue
+     *   (deactived) host1 : red
      *   host2 :
-     *   (deactived and then actived) host3 : red, yellow
+     *   (deactived and then actived) host3 : yellow
      *   host4 :
      */
     mgr.deactivateNode(NodeId.newInstance("host1", 1));
@@ -294,7 +290,7 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
     mgr.activateNode(NodeId.newInstance("host3", 1), SMALL_RESOURCE);
     
     // check resource
-    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
+    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2),
         mgr.getQueueResource("Q1", q1Label, clusterResource));
     Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
         mgr.getQueueResource("Q2", q2Label, clusterResource));
@@ -331,9 +327,9 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
     // check resource
     Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2),
         mgr.getQueueResource("Q1", q1Label, clusterResource));
-    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
+    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2),
         mgr.getQueueResource("Q2", q2Label, clusterResource));
-    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
+    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2),
         mgr.getQueueResource("Q3", q3Label, clusterResource));
     Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2),
         mgr.getQueueResource("Q4", q4Label, clusterResource));
@@ -344,7 +340,7 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
      * Active NMs in nodes already have NM
      * Node->Labels:
      *   host2 :
-     *   host3 : red, yellow (3 NMs)
+     *   host3 : yellow (3 NMs)
      *   host4 : (2 NMs)
      */
     mgr.activateNode(NodeId.newInstance("host3", 2), SMALL_RESOURCE);
@@ -354,9 +350,9 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
     // check resource
     Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
         mgr.getQueueResource("Q1", q1Label, clusterResource));
-    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 6),
+    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
         mgr.getQueueResource("Q2", q2Label, clusterResource));
-    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 6),
+    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
         mgr.getQueueResource("Q3", q3Label, clusterResource));
     Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
         mgr.getQueueResource("Q4", q4Label, clusterResource));
@@ -367,7 +363,7 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
      * Deactive NMs in nodes already have NMs
      * Node->Labels:
      *   host2 :
-     *   host3 : red, yellow (2 NMs)
+     *   host3 : yellow (2 NMs)
      *   host4 : (0 NMs)
      */
     mgr.deactivateNode(NodeId.newInstance("host3", 3));
@@ -377,9 +373,9 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
     // check resource
     Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 1),
         mgr.getQueueResource("Q1", q1Label, clusterResource));
-    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
+    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 1),
         mgr.getQueueResource("Q2", q2Label, clusterResource));
-    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3),
+    Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 1),
         mgr.getQueueResource("Q3", q3Label, clusterResource));
     Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 1),
         mgr.getQueueResource("Q4", q4Label, clusterResource));

+ 46 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java

@@ -62,6 +62,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.nodelabels.NodeLabelsStore;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMWithAMS;
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager;
@@ -213,11 +215,7 @@ public class TestSchedulerUtils {
       resReq.setNodeLabelExpression("x");
       SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
           scheduler);
-      
-      resReq.setNodeLabelExpression("x && y");
-      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
-          scheduler);
-      
+
       resReq.setNodeLabelExpression("y");
       SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
           scheduler);
@@ -252,6 +250,8 @@ public class TestSchedulerUtils {
     } catch (InvalidResourceRequestException e) {
     }
     
+    // we don't allow specify more than two node labels in a single expression
+    // now
     try {
       // set queue accessible node labesl to [x, y]
       queueAccessibleNodeLabels.clear();
@@ -262,7 +262,7 @@ public class TestSchedulerUtils {
           YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
           mock(Priority.class), ResourceRequest.ANY, resource, 1);
-      resReq.setNodeLabelExpression("x && y && z");
+      resReq.setNodeLabelExpression("x && y");
       SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
           scheduler);
       fail("Should fail");
@@ -327,7 +327,7 @@ public class TestSchedulerUtils {
       SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
           scheduler);
       
-      resReq.setNodeLabelExpression("x && y && z");
+      resReq.setNodeLabelExpression("y");
       SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
           scheduler);
       
@@ -336,7 +336,45 @@ public class TestSchedulerUtils {
           scheduler);
     } catch (InvalidResourceRequestException e) {
       e.printStackTrace();
-      fail("Should be valid when request labels is empty");
+      fail("Should be valid when queue can access any labels");
+    }
+    
+    // we don't allow resource name other than ANY and specify label
+    try {
+      // set queue accessible node labesl to [x, y]
+      queueAccessibleNodeLabels.clear();
+      queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
+      
+      Resource resource = Resources.createResource(
+          0,
+          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+      ResourceRequest resReq = BuilderUtils.newResourceRequest(
+          mock(Priority.class), "rack", resource, 1);
+      resReq.setNodeLabelExpression("x");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      fail("Should fail");
+    } catch (InvalidResourceRequestException e) {
+    }
+    
+    // we don't allow resource name other than ANY and specify label even if
+    // queue has accessible label = *
+    try {
+      // set queue accessible node labesl to *
+      queueAccessibleNodeLabels.clear();
+      queueAccessibleNodeLabels.addAll(Arrays
+          .asList(CommonNodeLabelsManager.ANY));
+      
+      Resource resource = Resources.createResource(
+          0,
+          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+      ResourceRequest resReq = BuilderUtils.newResourceRequest(
+          mock(Priority.class), "rack", resource, 1);
+      resReq.setNodeLabelExpression("x");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      fail("Should fail");
+    } catch (InvalidResourceRequestException e) {
     }
   }
 

+ 15 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java

@@ -87,6 +87,7 @@ public class TestContainerAllocation {
 
   @Test(timeout = 3000000)
   public void testExcessReservationThanNodeManagerCapacity() throws Exception {
+    @SuppressWarnings("resource")
     MockRM rm = new MockRM(conf);
     rm.start();
 
@@ -393,6 +394,7 @@ public class TestContainerAllocation {
     }
   }
   
+  @SuppressWarnings("unchecked")
   private <E> Set<E> toSet(E... elements) {
     Set<E> set = Sets.newHashSet(elements);
     return set;
@@ -449,7 +451,7 @@ public class TestContainerAllocation {
     return conf;
   }
   
-  @Test(timeout = 300000)
+  @Test (timeout = 300000)
   public void testContainerAllocationWithSingleUserLimits() throws Exception {
     final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
     mgr.init(conf);
@@ -470,7 +472,7 @@ public class TestContainerAllocation {
     rm1.getRMContext().setNodeLabelManager(mgr);
     rm1.start();
     MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
-    MockNM nm2 = rm1.registerNode("h2:1234", 8000); // label = y
+    rm1.registerNode("h2:1234", 8000); // label = y
     MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
 
     // launch an app to queue a1 (label = x), and check all container will
@@ -518,9 +520,9 @@ public class TestContainerAllocation {
      *                           
      * Node structure:
      * h1 : x
-     * h2 : x, y
+     * h2 : y
      * h3 : y
-     * h4 : y, z
+     * h4 : z
      * h5 : NO
      * 
      * Total resource:
@@ -540,9 +542,9 @@ public class TestContainerAllocation {
     // set node -> label
     mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y", "z"));
     mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0),
-        toSet("x"), NodeId.newInstance("h2", 0), toSet("x", "y"),
+        toSet("x"), NodeId.newInstance("h2", 0), toSet("y"),
         NodeId.newInstance("h3", 0), toSet("y"), NodeId.newInstance("h4", 0),
-        toSet("y", "z"), NodeId.newInstance("h5", 0),
+        toSet("z"), NodeId.newInstance("h5", 0),
         RMNodeLabelsManager.EMPTY_STRING_SET));
 
     // inject node label manager
@@ -568,12 +570,10 @@ public class TestContainerAllocation {
     RMApp app1 = rm1.submitApp(1024, "app", "user", null, "a1");
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
 
-    // request a container (label = x && y). can only allocate on nm2 
-    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x && y");
+    // request a container (label = y). can be allocated on nm2 
+    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
     containerId =
-        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
-    Assert.assertFalse(rm1.waitForState(nm1, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2L);
     Assert.assertTrue(rm1.waitForState(nm2, containerId,
         RMContainerState.ALLOCATED, 10 * 1000));
     checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
@@ -609,12 +609,10 @@ public class TestContainerAllocation {
     checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
         "h3");
     
-    // try to allocate container (request label = y && z) on nm3 (label = y) and
-    // nm4 (label = y,z). Will sucessfully allocate on nm4 only.
-    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y && z");
-    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 3);
-    Assert.assertFalse(rm1.waitForState(nm3, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
+    // try to allocate container (request label = z) on nm4 (label = y,z). 
+    // Will successfully allocate on nm4 only.
+    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "z");
+    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 3L);
     Assert.assertTrue(rm1.waitForState(nm4, containerId,
         RMContainerState.ALLOCATED, 10 * 1000));
     checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,

+ 8 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java

@@ -155,7 +155,7 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("replace-labels")
             .queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\": [\"a\", \"b\"]}",
+            .entity("{\"nodeLabels\": [\"a\"]}",
               MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     LOG.info("posted node nodelabel");
@@ -168,8 +168,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     json = response.getEntity(JSONObject.class);
-    jarr = json.getJSONArray("nodeLabels");
-    assertEquals(2, jarr.length());
+    assertEquals("a", json.getString("nodeLabels"));
+
     
     // Replace
     response =
@@ -178,9 +178,10 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("replace-labels")
             .queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\":\"a\"}", MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\":\"b\"}", MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     LOG.info("posted node nodelabel");
+
     // Verify
     response =
         r.path("ws").path("v1").path("cluster")
@@ -189,13 +190,12 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     json = response.getEntity(JSONObject.class);
-    assertEquals("a", json.getString("nodeLabels"));
+    assertEquals("b", json.getString("nodeLabels"));
             
     // Replace labels using node-to-labels
     NodeToLabelsInfo ntli = new NodeToLabelsInfo();
     NodeLabelsInfo nli = new NodeLabelsInfo();
     nli.getNodeLabels().add("a");
-    nli.getNodeLabels().add("b");
     ntli.getNodeToLabels().put("nid:0", nli);
     response =
         r.path("ws").path("v1").path("cluster")
@@ -214,9 +214,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     ntli = response.getEntity(NodeToLabelsInfo.class);
     nli = ntli.getNodeToLabels().get("nid:0");
-    assertEquals(2, nli.getNodeLabels().size());
+    assertEquals(1, nli.getNodeLabels().size());
     assertTrue(nli.getNodeLabels().contains("a"));
-    assertTrue(nli.getNodeLabels().contains("b"));
     
     // Remove all
     response =
@@ -267,7 +266,7 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("replace-labels")
             .queryParam("user.name", notUserName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\": [\"a\", \"b\"]}",
+            .entity("{\"nodeLabels\": [\"b\"]}",
               MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     // Verify