Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 8F118200B77 for ; Sat, 20 Aug 2016 02:04:11 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 8D5E8160AAB; Sat, 20 Aug 2016 00:04:11 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id D5285160AC1 for ; Sat, 20 Aug 2016 02:04:09 +0200 (CEST) Received: (qmail 53262 invoked by uid 500); 20 Aug 2016 00:04:07 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 52797 invoked by uid 99); 20 Aug 2016 00:04:07 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 20 Aug 2016 00:04:07 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 08CAEE0252; Sat, 20 Aug 2016 00:04:07 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: wangda@apache.org To: common-commits@hadoop.apache.org Date: Sat, 20 Aug 2016 00:04:12 -0000 Message-Id: <8c8fd98f0fc84a12bcc851fadd5160eb@git.apache.org> In-Reply-To: <67f9457907b549c6a9c2e02b8a6e107c@git.apache.org> References: <67f9457907b549c6a9c2e02b8a6e107c@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [7/9] hadoop git commit: YARN-4837. User facing aspects of 'AM blacklisting' feature need fixing. (vinodkv via wangda) archived-at: Sat, 20 Aug 2016 00:04:11 -0000 http://git-wip-us.apache.org/repos/asf/hadoop/blob/9bf420b3/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..0d61b36 --- /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,252 @@ +/** + * 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.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 + rm.waitForState(attempt.getAppAttemptId(), 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 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(); + rm.waitForState(attempt.getAppAttemptId(), 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(); + + rm.waitForState(attempt.getAppAttemptId(), 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 createSchedulerEventDispatcher() { + return new SchedulerEventDispatcher(this.scheduler) { + @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/9bf420b3/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 77d61a1..d5eed98 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,12 +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.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; @@ -53,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.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.SystemClock; -import org.apache.hadoop.yarn.util.resource.Resources; import org.junit.Assert; import org.junit.Test; @@ -237,7 +229,7 @@ public class TestAMRestart { rm1.stop(); } - private List allocateContainers(MockNM nm1, MockAM am1, + public static List allocateContainers(MockNM nm1, MockAM am1, int NUM_CONTAINERS) throws Exception { // allocate NUM_CONTAINERS containers am1.allocate("127.0.0.1", 1024, NUM_CONTAINERS, @@ -276,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(); @@ -378,165 +372,6 @@ public class TestAMRestart { rm1.stop(); } - @Test(timeout = 100000) - public void testAMBlacklistPreventsRestartOnSameNode() throws Exception { - YarnConfiguration conf = new YarnConfiguration(); - 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 createSchedulerEventDispatcher() { - return new SchedulerEventDispatcher(this.scheduler) { - @Override - public void handle(SchedulerEvent event) { - scheduler.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); - am1.waitForState(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 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/9bf420b3/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 blacklistAdditions = blacklist.getAdditions(); + List blacklistAdditions = blacklist.getBlacklistAdditions(); Collections.sort(blacklistAdditions); - List blacklistRemovals = blacklist.getRemovals(); + List 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 blacklistAdditions = blacklist.getAdditions(); + List blacklistAdditions = blacklist.getBlacklistAdditions(); Collections.sort(blacklistAdditions); - List blacklistRemovals = blacklist.getRemovals(); + List 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 blacklistAdditions = blacklist.getAdditions(); - List blacklistRemovals = blacklist.getRemovals(); + List blacklistAdditions = blacklist.getBlacklistAdditions(); + List blacklistRemovals = blacklist.getBlacklistRemovals(); Assert.assertTrue( "Blacklist additions should be empty but was " + blacklistAdditions, http://git-wip-us.apache.org/repos/asf/hadoop/blob/9bf420b3/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 f2762a7..1762838 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 @@ -44,7 +44,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; @@ -1063,63 +1062,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 finalState = ArgumentCaptor.forClass(RMAppState.class); http://git-wip-us.apache.org/repos/asf/hadoop/blob/9bf420b3/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(), + appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList(), new ArrayList()); Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged()); ArrayList blacklistAdditions = new ArrayList(); blacklistAdditions.add("node1"); blacklistAdditions.add("node2"); - appSchedulingInfo.updateBlacklist(blacklistAdditions, + appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions, new ArrayList()); Assert.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged()); blacklistAdditions.clear(); blacklistAdditions.add("node1"); - appSchedulingInfo.updateBlacklist(blacklistAdditions, + appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions, new ArrayList()); Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged()); ArrayList blacklistRemovals = new ArrayList(); blacklistRemovals.add("node1"); - appSchedulingInfo.updateBlacklist(new ArrayList(), + appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList(), blacklistRemovals); - appSchedulingInfo.updateBlacklist(new ArrayList(), + appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList(), blacklistRemovals); Assert.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged()); - appSchedulingInfo.updateBlacklist(new ArrayList(), + appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList(), blacklistRemovals); Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged()); } http://git-wip-us.apache.org/repos/asf/hadoop/blob/9bf420b3/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 a3a4ca0..f1d36d5 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 @@ -737,14 +737,16 @@ public class TestCapacityScheduler { cs.allocate(appAttemptId, Collections.emptyList(), Collections.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.emptyList(), Collections.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/9bf420b3/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 b30fafd..079ed93 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 @@ -313,24 +313,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.getAvailableResource(), 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.getAvailableResource(), 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/9bf420b3/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 06bbfe9..367a9c2 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 @@ -4907,11 +4907,12 @@ public class TestFairScheduler extends FairSchedulerTestBase { scheduler.allocate(appAttemptId, Collections.emptyList(), Collections.emptyList(), Collections.singletonList(host), null, null, null); - assertTrue(app.isBlacklisted(host)); + assertTrue(app.isPlaceBlacklisted(host)); scheduler.allocate(appAttemptId, Collections.emptyList(), Collections.emptyList(), null, Collections.singletonList(host), null, null); - assertFalse(scheduler.getSchedulerApp(appAttemptId).isBlacklisted(host)); + assertFalse(scheduler.getSchedulerApp(appAttemptId) + .isPlaceBlacklisted(host)); List update = Arrays.asList( createResourceRequest(GB, node.getHostName(), 1, 0, true)); @@ -4920,7 +4921,7 @@ public class TestFairScheduler extends FairSchedulerTestBase { scheduler.allocate(appAttemptId, update, Collections.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 @@ -4930,7 +4931,7 @@ public class TestFairScheduler extends FairSchedulerTestBase { scheduler.allocate(appAttemptId, update, Collections.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/9bf420b3/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 73e9fc7..455f8bc 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 @@ -18,21 +18,38 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.google.inject.Singleton; -import com.google.inject.servlet.GuiceServletContextListener; -import com.google.inject.servlet.ServletModule; -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; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import com.sun.jersey.test.framework.WebAppDescriptor; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assume.assumeTrue; + +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; +import java.util.Collection; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.ws.rs.core.HttpHeaders; +import javax.ws.rs.core.MediaType; +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; + import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; @@ -82,36 +99,21 @@ import org.w3c.dom.NodeList; import org.xml.sax.InputSource; import org.xml.sax.SAXException; -import javax.servlet.FilterConfig; -import javax.servlet.ServletException; -import javax.ws.rs.core.HttpHeaders; -import javax.ws.rs.core.MediaType; -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; -import javax.xml.parsers.ParserConfigurationException; -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; -import java.util.Collection; -import java.util.Enumeration; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assume.assumeTrue; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.Singleton; +import com.google.inject.servlet.GuiceServletContextListener; +import com.google.inject.servlet.ServletModule; +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; +import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; +import com.sun.jersey.test.framework.WebAppDescriptor; @RunWith(Parameterized.class) public class TestRMWebServicesAppsModification extends JerseyTestBase { --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org For additional commands, e-mail: common-commits-help@hadoop.apache.org