Repository: hadoop
Updated Branches:
  refs/heads/branch-2 df29f7784 -> aef6e455b


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aef6e455/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
index 60b728e..e8c8bca 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
@@ -42,7 +42,7 @@ public class AppAttemptInfo {
   protected String nodeId;
   protected String logsLink;
   protected String blacklistedNodes;
-  protected String rmBlacklistedNodesForAMLaunches;
+  private String nodesBlacklistedBySystem;
   protected String appAttemptId;
 
   public AppAttemptInfo() {
@@ -69,9 +69,9 @@ public class AppAttemptInfo {
             + masterContainer.getNodeHttpAddress(),
             ConverterUtils.toString(masterContainer.getId()), user);
 
-        rmBlacklistedNodesForAMLaunches = StringUtils.join(
-            attempt.getAMBlacklist().getBlacklistUpdates().getAdditions(),
-            ", ");
+        nodesBlacklistedBySystem =
+            StringUtils.join(attempt.getAMBlacklistManager()
+              .getBlacklistUpdates().getBlacklistAdditions(), ", ");
         if (rm.getResourceScheduler() instanceof AbstractYarnScheduler) {
           AbstractYarnScheduler ayScheduler =
               (AbstractYarnScheduler) rm.getResourceScheduler();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aef6e455/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.java
index 4cbe7a8..3d95ca1 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.java
@@ -87,9 +87,6 @@ public class ApplicationSubmissionContextInfo {
   @XmlElement(name = "reservation-id")
   String reservationId;
 
-  @XmlElement(name = "am-black-listing-requests")
-  AMBlackListingRequestInfo amBlackListingRequestInfo;
-
   public ApplicationSubmissionContextInfo() {
     applicationId = "";
     applicationName = "";
@@ -106,7 +103,6 @@ public class ApplicationSubmissionContextInfo {
     logAggregationContextInfo = null;
     attemptFailuresValidityInterval = -1;
     reservationId = "";
-    amBlackListingRequestInfo = null;
   }
 
   public String getApplicationId() {
@@ -173,10 +169,6 @@ public class ApplicationSubmissionContextInfo {
     return attemptFailuresValidityInterval;
   }
 
-  public AMBlackListingRequestInfo getAMBlackListingRequestInfo() {
-    return amBlackListingRequestInfo;
-  }
-
   public String getReservationId() {
     return reservationId;
   }
@@ -252,9 +244,4 @@ public class ApplicationSubmissionContextInfo {
   public void setReservationId(String reservationId) {
     this.reservationId = reservationId;
   }
-
-  public void setAMBlackListingRequestInfo(
-      AMBlackListingRequestInfo amBlackListingRequestInfo) {
-    this.amBlackListingRequestInfo = amBlackListingRequestInfo;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aef6e455/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java
new file mode 100644
index 0000000..ef6d43b
--- /dev/null
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java
@@ -0,0 +1,251 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager;
+
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.DrainDispatcher;
+import org.apache.hadoop.yarn.event.EventDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import 
org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager.TestAMRestart;
+import 
org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import 
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import 
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import 
org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Validate system behavior when the am-scheduling logic 'blacklists' a node 
for
+ * an application because of AM failures.
+ */
+public class TestNodeBlacklistingOnAMFailures {
+
+  @Test(timeout = 100000)
+  public void testNodeBlacklistingOnAMFailure() throws Exception {
+
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.setBoolean(YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_ENABLED,
+        true);
+
+    DrainDispatcher dispatcher = new DrainDispatcher();
+    MockRM rm = startRM(conf, dispatcher);
+    CapacityScheduler scheduler = (CapacityScheduler) 
rm.getResourceScheduler();
+
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 8000, rm.getResourceTrackerService());
+    nm1.registerNode();
+
+    MockNM nm2 =
+        new MockNM("127.0.0.2:2345", 8000, rm.getResourceTrackerService());
+    nm2.registerNode();
+
+    RMApp app = rm.submitApp(200);
+
+    MockAM am1 = MockRM.launchAndRegisterAM(app, rm, nm1);
+    ContainerId amContainerId =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    RMContainer rmContainer = scheduler.getRMContainer(amContainerId);
+    NodeId nodeWhereAMRan = rmContainer.getAllocatedNode();
+
+    MockNM currentNode, otherNode;
+    if (nodeWhereAMRan.equals(nm1.getNodeId())) {
+      currentNode = nm1;
+      otherNode = nm2;
+    } else {
+      currentNode = nm2;
+      otherNode = nm1;
+    }
+
+    // Set the exist status to INVALID so that we can verify that the system
+    // automatically blacklisting the node
+    makeAMContainerExit(rm, amContainerId, currentNode,
+        ContainerExitStatus.INVALID);
+
+    // restart the am
+    RMAppAttempt attempt = MockRM.waitForAttemptScheduled(app, rm);
+    System.out.println("New AppAttempt launched " + attempt.getAppAttemptId());
+
+    // Try the current node a few times
+    for (int i = 0; i <= 2; i++) {
+      currentNode.nodeHeartbeat(true);
+      dispatcher.await();
+
+      Assert.assertEquals(
+          "AppAttemptState should still be SCHEDULED if currentNode is "
+              + "blacklisted correctly", RMAppAttemptState.SCHEDULED,
+          attempt.getAppAttemptState());
+    }
+
+    // Now try the other node
+    otherNode.nodeHeartbeat(true);
+    dispatcher.await();
+
+    // Now the AM container should be allocated
+    MockRM.waitForState(attempt, RMAppAttemptState.ALLOCATED, 20000);
+
+    MockAM am2 = rm.sendAMLaunched(attempt.getAppAttemptId());
+    rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
+    amContainerId =
+        ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
+    rmContainer = scheduler.getRMContainer(amContainerId);
+    nodeWhereAMRan = rmContainer.getAllocatedNode();
+
+    // The other node should now receive the assignment
+    Assert.assertEquals(
+        "After blacklisting, AM should have run on the other node",
+        otherNode.getNodeId(), nodeWhereAMRan);
+
+    am2.registerAppAttempt();
+    rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
+
+    List<Container> allocatedContainers =
+        TestAMRestart.allocateContainers(currentNode, am2, 1);
+    Assert.assertEquals(
+        "Even though AM is blacklisted from the node, application can "
+            + "still allocate non-AM containers there",
+        currentNode.getNodeId(), allocatedContainers.get(0).getNodeId());
+  }
+
+  @Test(timeout = 100000)
+  public void testNoBlacklistingForNonSystemErrors() throws Exception {
+
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_ENABLED,
+        true);
+    // disable the float so it is possible to blacklist the entire cluster
+    conf.setFloat(
+        YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD,
+        1.5f);
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 100);
+
+    DrainDispatcher dispatcher = new DrainDispatcher();
+    MockRM rm = startRM(conf, dispatcher);
+
+    MockNM node =
+        new MockNM("127.0.0.1:1234", 8000, rm.getResourceTrackerService());
+    node.registerNode();
+
+    RMApp app = rm.submitApp(200);
+    ApplicationId appId = app.getApplicationId();
+
+    int numAppAttempts = 1;
+
+    // Now the AM container should be allocated
+    RMAppAttempt attempt = MockRM.waitForAttemptScheduled(app, rm);
+    node.nodeHeartbeat(true);
+    dispatcher.await();
+    MockRM.waitForState(attempt, RMAppAttemptState.ALLOCATED, 20000);
+    rm.sendAMLaunched(attempt.getAppAttemptId());
+    rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(appId, numAppAttempts);
+    ContainerId amContainerId = ContainerId.newContainerId(appAttemptId, 1);
+
+    for (int containerExitStatus : new int[] {
+        ContainerExitStatus.PREEMPTED,
+        ContainerExitStatus.KILLED_BY_RESOURCEMANAGER,
+        // ContainerExitStatus.KILLED_BY_APPMASTER,
+        ContainerExitStatus.KILLED_AFTER_APP_COMPLETION,
+        ContainerExitStatus.ABORTED, ContainerExitStatus.DISKS_FAILED,
+        ContainerExitStatus.KILLED_EXCEEDED_VMEM,
+        ContainerExitStatus.KILLED_EXCEEDED_PMEM }) {
+
+      // Set the exist status to be containerExitStatus so that we can verify
+      // that the system automatically blacklisting the node
+      makeAMContainerExit(rm, amContainerId, node, containerExitStatus);
+
+      // restart the am
+      attempt = MockRM.waitForAttemptScheduled(app, rm);
+      System.out
+          .println("New AppAttempt launched " + attempt.getAppAttemptId());
+
+      node.nodeHeartbeat(true);
+      dispatcher.await();
+
+      MockRM.waitForState(attempt, RMAppAttemptState.ALLOCATED, 20000);
+      rm.sendAMLaunched(attempt.getAppAttemptId());
+      rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
+
+      numAppAttempts++;
+      appAttemptId = ApplicationAttemptId.newInstance(appId, numAppAttempts);
+      amContainerId = ContainerId.newContainerId(appAttemptId, 1);
+      rm.waitForState(node, amContainerId, RMContainerState.ACQUIRED);
+    }
+  }
+
+  private void makeAMContainerExit(MockRM rm, ContainerId amContainer,
+      MockNM node, int exitStatus) throws Exception, InterruptedException {
+    ContainerStatus containerStatus =
+        BuilderUtils.newContainerStatus(amContainer, ContainerState.COMPLETE,
+            "", exitStatus, Resources.createResource(200));
+    node.containerStatus(containerStatus);
+    ApplicationAttemptId amAttemptID = amContainer.getApplicationAttemptId();
+    rm.waitForState(amAttemptID, RMAppAttemptState.FAILED);
+    rm.waitForState(amAttemptID.getApplicationId(), RMAppState.ACCEPTED);
+  }
+
+  private MockRM startRM(YarnConfiguration conf,
+      final DrainDispatcher dispatcher) {
+
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+
+    MockRM rm1 = new MockRM(conf, memStore) {
+      @Override
+      protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
+        return new EventDispatcher<SchedulerEvent>(this.scheduler,
+            this.scheduler.getClass().getName()) {
+          @Override
+          public void handle(SchedulerEvent event) {
+            super.handle(event);
+          }
+        };
+      }
+
+      @Override
+      protected Dispatcher createDispatcher() {
+        return dispatcher;
+      }
+    };
+
+    rm1.start();
+    return rm1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aef6e455/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index 66780a0..76a17c3 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -35,13 +35,8 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
-import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.Dispatcher;
-import org.apache.hadoop.yarn.event.DrainDispatcher;
-import org.apache.hadoop.yarn.event.EventDispatcher;
-import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
@@ -54,18 +49,14 @@ import 
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import 
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import 
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import 
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import 
org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.Records;
-import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -238,7 +229,7 @@ public class TestAMRestart {
     rm1.stop();
   }
 
-  private List<Container> allocateContainers(MockNM nm1, MockAM am1,
+  public static List<Container> allocateContainers(MockNM nm1, MockAM am1,
       int NUM_CONTAINERS) throws Exception {
     // allocate NUM_CONTAINERS containers
     am1.allocate("127.0.0.1", 1024, NUM_CONTAINERS,
@@ -277,7 +268,9 @@ public class TestAMRestart {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 3);
     // To prevent test from blacklisting nm1 for AM, we sit threshold to half
     // of 2 nodes which is 1
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD, 0.5f);
+    conf.setFloat(
+        YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD,
+        0.5f);
 
     MockRM rm1 = new MockRM(conf);
     rm1.start();
@@ -379,168 +372,6 @@ public class TestAMRestart {
     rm1.stop();
   }
 
-  @Test(timeout = 100000)
-  public void testAMBlacklistPreventsRestartOnSameNode() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
-        ResourceScheduler.class);
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    testAMBlacklistPreventRestartOnSameNode(false, conf);
-  }
-
-  @Test(timeout = 100000)
-  public void testAMBlacklistPreventsRestartOnSameNodeForMinicluster()
-      throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME,
-        true);
-    testAMBlacklistPreventRestartOnSameNode(false, conf);
-  }
-
-  @Test(timeout = 100000)
-  public void testAMBlacklistPreemption() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    // disable the float so it is possible to blacklist the entire cluster
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD, 1.5f);
-    // since the exit status is PREEMPTED, it should not lead to the node being
-    // blacklisted
-    testAMBlacklistPreventRestartOnSameNode(true, conf);
-  }
-
-  /**
-   * Tests AM blacklisting. In the multi-node mode (i.e. singleNode = false),
-   * it tests the blacklisting behavior so that the AM container gets allocated
-   * on the node that is not blacklisted. In the single-node mode, it tests the
-   * PREEMPTED status to see if the AM container can continue to be scheduled.
-   */
-  private void testAMBlacklistPreventRestartOnSameNode(boolean singleNode,
-      YarnConfiguration conf) throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    final DrainDispatcher dispatcher = new DrainDispatcher();
-    MockRM rm1 = new MockRM(conf, memStore) {
-      @Override
-      protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
-        return new EventDispatcher<SchedulerEvent>(this.scheduler,
-            this.scheduler.getClass().getName()) {
-          @Override
-          public void handle(SchedulerEvent event) {
-            super.handle(event);
-          }
-        };
-      }
-
-      @Override
-      protected Dispatcher createDispatcher() {
-        return dispatcher;
-      }
-    };
-
-    rm1.start();
-
-    MockNM nm1 =
-        new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
-    nm1.registerNode();
-
-    MockNM nm2 = null;
-    if (!singleNode) {
-      nm2 =
-          new MockNM("127.0.0.2:2345", 8000, rm1.getResourceTrackerService());
-      nm2.registerNode();
-    }
-
-    RMApp app1 = rm1.submitApp(200);
-
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-    CapacityScheduler scheduler =
-        (CapacityScheduler) rm1.getResourceScheduler();
-    ContainerId amContainer =
-        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
-    // Preempt the first attempt;
-    RMContainer rmContainer = scheduler.getRMContainer(amContainer);
-    NodeId nodeWhereAMRan = rmContainer.getAllocatedNode();
-
-    MockNM currentNode, otherNode;
-    if (singleNode) {
-      Assert.assertEquals(nm1.getNodeId(), nodeWhereAMRan);
-      currentNode = nm1;
-      otherNode = null; // not applicable
-    } else {
-      if (nodeWhereAMRan == nm1.getNodeId()) {
-        currentNode = nm1;
-        otherNode = nm2;
-      } else {
-        currentNode = nm2;
-        otherNode = nm1;
-      }
-    }
-
-    // set the exist status to test
-    // any status other than SUCCESS and PREEMPTED should cause the node to be
-    // blacklisted
-    int exitStatus = singleNode ?
-            ContainerExitStatus.PREEMPTED :
-            ContainerExitStatus.INVALID;
-    ContainerStatus containerStatus =
-        BuilderUtils.newContainerStatus(amContainer, ContainerState.COMPLETE,
-            "", exitStatus, Resources.createResource(200));
-    currentNode.containerStatus(containerStatus);
-    rm1.waitForState(am1.getApplicationAttemptId(), RMAppAttemptState.FAILED);
-    rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
-
-    // restart the am
-    RMAppAttempt attempt = MockRM.waitForAttemptScheduled(app1, rm1);
-    System.out.println("Launch AM " + attempt.getAppAttemptId());
-
-
-
-    currentNode.nodeHeartbeat(true);
-    dispatcher.await();
-
-    if (!singleNode) {
-      Assert.assertEquals(
-          "AppAttemptState should still be SCHEDULED if currentNode is " +
-          "blacklisted correctly",
-          RMAppAttemptState.SCHEDULED,
-          attempt.getAppAttemptState());
-
-      otherNode.nodeHeartbeat(true);
-      dispatcher.await();
-    }
-
-    MockAM am2 = rm1.sendAMLaunched(attempt.getAppAttemptId());
-    rm1.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
-    amContainer =
-        ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
-    rmContainer = scheduler.getRMContainer(amContainer);
-    nodeWhereAMRan = rmContainer.getAllocatedNode();
-    if (singleNode) {
-      // with preemption, the node should not be blacklisted and should get the
-      // assignment (with a single node)
-      Assert.assertEquals(
-          "AM should still have been able to run on the same node",
-          currentNode.getNodeId(), nodeWhereAMRan);
-    } else {
-      // with a failed status, the other node should receive the assignment
-      Assert.assertEquals(
-          "After blacklisting AM should have run on the other node",
-          otherNode.getNodeId(), nodeWhereAMRan);
-
-      am2.registerAppAttempt();
-      rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
-
-      List<Container> allocatedContainers =
-          allocateContainers(currentNode, am2, 1);
-      Assert.assertEquals(
-          "Even though AM is blacklisted from the node, application can " +
-          "still allocate containers there",
-          currentNode.getNodeId(), allocatedContainers.get(0).getNodeId());
-    }
-  }
-
-
   // AM container preempted, nm disk failure
   // should not be counted towards AM max retry count.
   @Test(timeout = 100000)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aef6e455/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java
index 96b373f..3102c6f 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java
@@ -19,12 +19,13 @@
 package org.apache.hadoop.yarn.server.resourcemanager.blacklist;
 
 
-import org.junit.Assert;
-import org.junit.Test;
-
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+import org.junit.Assert;
+import org.junit.Test;
+
 public class TestBlacklistManager {
 
   @Test
@@ -37,12 +38,12 @@ public class TestBlacklistManager {
     String anyNode2 = "bar";
     manager.addNode(anyNode);
     manager.addNode(anyNode2);
-    BlacklistUpdates blacklist = manager
+    ResourceBlacklistRequest blacklist = manager
         .getBlacklistUpdates();
 
-    List<String> blacklistAdditions = blacklist.getAdditions();
+    List<String> blacklistAdditions = blacklist.getBlacklistAdditions();
     Collections.sort(blacklistAdditions);
-    List<String> blacklistRemovals = blacklist.getRemovals();
+    List<String> blacklistRemovals = blacklist.getBlacklistRemovals();
     String[] expectedBlacklistAdditions = new String[]{anyNode2, anyNode};
     Assert.assertArrayEquals(
         "Blacklist additions was not as expected",
@@ -61,12 +62,12 @@ public class TestBlacklistManager {
     String anyNode = "foo";
     String anyNode2 = "bar";
     manager.addNode(anyNode);
-    BlacklistUpdates blacklist = manager
+    ResourceBlacklistRequest blacklist = manager
         .getBlacklistUpdates();
 
-    List<String> blacklistAdditions = blacklist.getAdditions();
+    List<String> blacklistAdditions = blacklist.getBlacklistAdditions();
     Collections.sort(blacklistAdditions);
-    List<String> blacklistRemovals = blacklist.getRemovals();
+    List<String> blacklistRemovals = blacklist.getBlacklistRemovals();
     String[] expectedBlacklistAdditions = new String[]{anyNode};
     Assert.assertArrayEquals(
         "Blacklist additions was not as expected",
@@ -81,9 +82,9 @@ public class TestBlacklistManager {
 
     blacklist = manager
         .getBlacklistUpdates();
-    blacklistAdditions = blacklist.getAdditions();
+    blacklistAdditions = blacklist.getBlacklistAdditions();
     Collections.sort(blacklistAdditions);
-    blacklistRemovals = blacklist.getRemovals();
+    blacklistRemovals = blacklist.getBlacklistRemovals();
     Collections.sort(blacklistRemovals);
     String[] expectedBlacklistRemovals = new String[] {anyNode2, anyNode};
     Assert.assertTrue(
@@ -101,11 +102,11 @@ public class TestBlacklistManager {
     BlacklistManager disabled = new DisabledBlacklistManager();
     String anyNode = "foo";
     disabled.addNode(anyNode);
-    BlacklistUpdates blacklist = disabled
+    ResourceBlacklistRequest blacklist = disabled
         .getBlacklistUpdates();
 
-    List<String> blacklistAdditions = blacklist.getAdditions();
-    List<String> blacklistRemovals = blacklist.getRemovals();
+    List<String> blacklistAdditions = blacklist.getBlacklistAdditions();
+    List<String> blacklistRemovals = blacklist.getBlacklistRemovals();
     Assert.assertTrue(
         "Blacklist additions should be empty but was " +
             blacklistAdditions,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aef6e455/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
index 293c0b6..bb5c25c 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.yarn.MockApps;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -1029,63 +1028,6 @@ public class TestRMAppTransitions {
             + "/"));
   }
 
-  @Test
-  public void testAMBlackListConfigFromApp() {
-    // Scenario 1: Application enables AM blacklisting
-    float disableThreshold = 0.9f;
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, false);
-    ApplicationSubmissionContext submissionContext =
-        new ApplicationSubmissionContextPBImpl();
-    submissionContext.setAMBlackListRequest(AMBlackListingRequest.newInstance(
-        true, disableThreshold));
-    RMAppImpl application = (RMAppImpl) createNewTestApp(submissionContext);
-
-    Assert.assertTrue(application.isAmBlacklistingEnabled());
-    Assert.assertEquals(disableThreshold,
-        application.getAmBlacklistingDisableThreshold(), 1e-8);
-
-    // Scenario 2: Application disables AM blacklisting
-    float globalThreshold = 0.9f;
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-        globalThreshold);
-    ApplicationSubmissionContext submissionContext2 =
-        new ApplicationSubmissionContextPBImpl();
-    submissionContext2.setAMBlackListRequest(AMBlackListingRequest.newInstance(
-        false, disableThreshold));
-    RMAppImpl application2 = (RMAppImpl) createNewTestApp(submissionContext2);
-
-    // Am blacklisting will be disabled eventhough its enabled in RM.
-    Assert.assertFalse(application2.isAmBlacklistingEnabled());
-
-    // Scenario 3: Application updates invalid AM threshold
-    float invalidDisableThreshold = -0.5f;
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-        globalThreshold);
-    ApplicationSubmissionContext submissionContext3 =
-        new ApplicationSubmissionContextPBImpl();
-    submissionContext3.setAMBlackListRequest(AMBlackListingRequest.newInstance(
-        true, invalidDisableThreshold));
-    RMAppImpl application3 = (RMAppImpl) createNewTestApp(submissionContext3);
-
-    Assert.assertTrue(application3.isAmBlacklistingEnabled());
-    Assert.assertEquals(globalThreshold,
-        application3.getAmBlacklistingDisableThreshold(), 1e-8);
-
-    // Scenario 4: Empty AMBlackListingRequest in Submission Context
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-        globalThreshold);
-    ApplicationSubmissionContext submissionContext4 =
-        new ApplicationSubmissionContextPBImpl();
-    RMAppImpl application4 = (RMAppImpl) createNewTestApp(submissionContext4);
-
-    Assert.assertTrue(application4.isAmBlacklistingEnabled());
-    Assert.assertEquals(globalThreshold,
-        application4.getAmBlacklistingDisableThreshold(), 1e-8);
-  }
-
   private void verifyApplicationFinished(RMAppState state) {
     ArgumentCaptor<RMAppState> finalState =
         ArgumentCaptor.forClass(RMAppState.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aef6e455/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
index 4141a53..a1c6294 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
@@ -41,32 +41,32 @@ public class TestAppSchedulingInfo {
     AppSchedulingInfo  appSchedulingInfo = new AppSchedulingInfo(
         appAttemptId, "test", queue, null, 0, new ResourceUsage());
 
-    appSchedulingInfo.updateBlacklist(new ArrayList<String>(),
+    appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         new ArrayList<String>());
     Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
 
     ArrayList<String> blacklistAdditions = new ArrayList<String>();
     blacklistAdditions.add("node1");
     blacklistAdditions.add("node2");
-    appSchedulingInfo.updateBlacklist(blacklistAdditions,
+    appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions,
         new ArrayList<String>());
     Assert.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged());
 
     blacklistAdditions.clear();
     blacklistAdditions.add("node1");
-    appSchedulingInfo.updateBlacklist(blacklistAdditions,
+    appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions,
         new ArrayList<String>());
     Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
 
     ArrayList<String> blacklistRemovals = new ArrayList<String>();
     blacklistRemovals.add("node1");
-    appSchedulingInfo.updateBlacklist(new ArrayList<String>(),
+    appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         blacklistRemovals);
-    appSchedulingInfo.updateBlacklist(new ArrayList<String>(),
+    appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         blacklistRemovals);
     Assert.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged());
 
-    appSchedulingInfo.updateBlacklist(new ArrayList<String>(),
+    appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         blacklistRemovals);
     Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aef6e455/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index 72d2f85..7c34292 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -684,14 +684,16 @@ public class TestCapacityScheduler {
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null, null, null);
-    
Assert.assertTrue(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
+    Assert.assertTrue(cs.getApplicationAttempt(appAttemptId)
+        .isPlaceBlacklisted(host));
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host), null, null);
-    
Assert.assertFalse(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
+    Assert.assertFalse(cs.getApplicationAttempt(appAttemptId)
+        .isPlaceBlacklisted(host));
     rm.stop();
   }
-  
+
   @Test
   public void testAllocateReorder() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aef6e455/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
index 1901fa9..af1dc62 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
@@ -314,24 +314,24 @@ public class TestFSAppAttempt extends 
FairSchedulerTestBase {
     FSAppAttempt spyApp = spy(app);
     doReturn(false)
         .when(spyApp).isWaitingForAMContainer();
-    assertTrue(spyApp.isBlacklisted(n1.getNodeName()));
-    assertFalse(spyApp.isBlacklisted(n2.getNodeName()));
+    assertTrue(spyApp.isPlaceBlacklisted(n1.getNodeName()));
+    assertFalse(spyApp.isPlaceBlacklisted(n2.getNodeName()));
     assertEquals(n2.getUnallocatedResource(), spyApp.getHeadroom());
 
     blacklistAdditions.clear();
     blacklistAdditions.add(n2.getNodeName());
     blacklistRemovals.add(n1.getNodeName());
     app.updateBlacklist(blacklistAdditions, blacklistRemovals);
-    assertFalse(spyApp.isBlacklisted(n1.getNodeName()));
-    assertTrue(spyApp.isBlacklisted(n2.getNodeName()));
+    assertFalse(spyApp.isPlaceBlacklisted(n1.getNodeName()));
+    assertTrue(spyApp.isPlaceBlacklisted(n2.getNodeName()));
     assertEquals(n1.getUnallocatedResource(), spyApp.getHeadroom());
 
     blacklistAdditions.clear();
     blacklistRemovals.clear();
     blacklistRemovals.add(n2.getNodeName());
     app.updateBlacklist(blacklistAdditions, blacklistRemovals);
-    assertFalse(spyApp.isBlacklisted(n1.getNodeName()));
-    assertFalse(spyApp.isBlacklisted(n2.getNodeName()));
+    assertFalse(spyApp.isPlaceBlacklisted(n1.getNodeName()));
+    assertFalse(spyApp.isPlaceBlacklisted(n2.getNodeName()));
     assertEquals(clusterResource, spyApp.getHeadroom());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aef6e455/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index ff08349..905436c 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -3995,11 +3995,12 @@ public class TestFairScheduler extends 
FairSchedulerTestBase {
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null, null, null);
-    assertTrue(app.isBlacklisted(host));
+    assertTrue(app.isPlaceBlacklisted(host));
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host), null, null);
-    assertFalse(scheduler.getSchedulerApp(appAttemptId).isBlacklisted(host));
+    assertFalse(scheduler.getSchedulerApp(appAttemptId)
+        .isPlaceBlacklisted(host));
 
     List<ResourceRequest> update = Arrays.asList(
         createResourceRequest(GB, node.getHostName(), 1, 0, true));
@@ -4008,7 +4009,7 @@ public class TestFairScheduler extends 
FairSchedulerTestBase {
     scheduler.allocate(appAttemptId, update,
         Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null, null, null);
-    assertTrue(app.isBlacklisted(host));
+    assertTrue(app.isPlaceBlacklisted(host));
     scheduler.update();
     scheduler.handle(updateEvent);
     assertEquals("Incorrect number of containers allocated", 0, app
@@ -4018,7 +4019,7 @@ public class TestFairScheduler extends 
FairSchedulerTestBase {
     scheduler.allocate(appAttemptId, update,
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host), null, null);
-    assertFalse(app.isBlacklisted(host));
+    assertFalse(app.isPlaceBlacklisted(host));
     createSchedulingRequest(GB, "root.default", "user", 1);
     scheduler.update();
     scheduler.handle(updateEvent);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aef6e455/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
index 38b32e9..c7ef8fa 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
@@ -22,7 +22,14 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assume.assumeTrue;
 
-import java.io.*;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.io.StringWriter;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -43,15 +50,12 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
 
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import 
org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
@@ -72,7 +76,13 @@ import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.*;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import 
org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import 
org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CredentialsInfo;
+import 
org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LocalResourceInfo;
+import 
org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LogAggregationContextInfo;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
@@ -101,6 +111,7 @@ import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
 import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
 import com.sun.jersey.api.client.filter.LoggingFilter;
 import com.sun.jersey.api.json.JSONJAXBContext;
 import com.sun.jersey.api.json.JSONMarshaller;
@@ -827,17 +838,6 @@ public class TestRMWebServicesAppsModification extends 
JerseyTestBase {
         System.currentTimeMillis(), 1).toString();
     appInfo.setReservationId(reservationId);
 
-    // Set AMBlackListingRequestInfo
-    boolean isAMBlackListingEnabled = true;
-    float disableFailureThreshold = 0.01f;
-    AMBlackListingRequestInfo amBlackListingRequestInfo
-        = new AMBlackListingRequestInfo();
-    amBlackListingRequestInfo.setAMBlackListingEnabled(
-        isAMBlackListingEnabled);
-    amBlackListingRequestInfo.setBlackListingDisableFailureThreshold(
-        disableFailureThreshold);
-    appInfo.setAMBlackListingRequestInfo(amBlackListingRequestInfo);
-
     ClientResponse response =
         this.constructWebResource(urlPath).accept(acceptMedia)
           .entity(appInfo, contentMedia).post(ClientResponse.class);
@@ -912,13 +912,6 @@ public class TestRMWebServicesAppsModification extends 
JerseyTestBase {
     // Check ReservationId
     assertEquals(reservationId, app.getReservationId().toString());
 
-    // Check AMBlackListingRequestInfo
-    AMBlackListingRequest amBlackListingRequest = asc.getAMBlackListRequest();
-    assertEquals(isAMBlackListingEnabled,
-        amBlackListingRequest.isAMBlackListingEnabled());
-    assertTrue(disableFailureThreshold == amBlackListingRequest
-        .getBlackListingDisableFailureThreshold());
-
     response =
         this.constructWebResource("apps", appId).accept(acceptMedia)
           .get(ClientResponse.class);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-commits-h...@hadoop.apache.org

Reply via email to