Browse Source

YARN-10006. IOException used in place of YARNException in CapaitySheduler

Contributed by Adam Antal. Reviewed by Szilard Nemeth.
prabhujoseph 5 years ago
parent
commit
e575df5ca6

+ 11 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java

@@ -323,7 +323,7 @@ public class CapacityScheduler extends
 
   @VisibleForTesting
   void initScheduler(Configuration configuration) throws
-      IOException {
+      IOException, YarnException {
     writeLock.lock();
     try {
       String confProviderStr = configuration.get(
@@ -768,16 +768,19 @@ public class CapacityScheduler extends
 
   @Lock(CapacityScheduler.class)
   private void initializeQueues(CapacitySchedulerConfiguration conf)
-    throws IOException {
-
-    this.queueManager.initializeQueues(conf);
+    throws YarnException {
+    try {
+      this.queueManager.initializeQueues(conf);
 
-    updatePlacementRules();
+      updatePlacementRules();
 
-    this.workflowPriorityMappingsMgr.initialize(this);
+      this.workflowPriorityMappingsMgr.initialize(this);
 
-    // Notify Preemption Manager
-    preemptionManager.refreshQueues(null, this.getRootQueue());
+      // Notify Preemption Manager
+      preemptionManager.refreshQueues(null, this.getRootQueue());
+    } catch (Exception e) {
+      throw new YarnException("Failed to initialize queues", e);
+    }
   }
 
   @Lock(CapacityScheduler.class)

+ 4 - 2
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

@@ -53,6 +53,7 @@ import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CyclicBarrier;
 
 import com.google.common.collect.Sets;
+import org.apache.hadoop.service.ServiceStateException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -5409,9 +5410,10 @@ public class TestCapacityScheduler extends CapacitySchedulerTestBase {
     try {
       setUpCSQueue(maxLifetime, defaultLifetime);
       Assert.fail("Expected to fails since maxLifetime < defaultLifetime.");
-    } catch (YarnRuntimeException ye) {
+    } catch (ServiceStateException sse) {
+      Throwable rootCause = sse.getCause().getCause();
       Assert.assertTrue(
-          ye.getMessage().contains("can't exceed maximum lifetime"));
+          rootCause.getMessage().contains("can't exceed maximum lifetime"));
     }
 
     maxLifetime = -1;

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java

@@ -27,6 +27,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.service.ServiceOperations;
+import org.apache.hadoop.service.ServiceStateException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
@@ -35,7 +36,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabels
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy.PriorityUtilizationQueueOrderingPolicy;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy.QueueOrderingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -941,7 +941,7 @@ public class TestQueueParsing {
    * Test init a queue configuration, children's capacity for a given label
    * doesn't equals to 100%. This expect IllegalArgumentException thrown.
    */
-  @Test(expected = IllegalArgumentException.class)
+  @Test(expected = ServiceStateException.class)
   public void testQueueParsingFailWhenSumOfChildrenNonLabeledCapacityNot100Percent()
       throws IOException {
     nodeLabelManager.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet
@@ -971,7 +971,7 @@ public class TestQueueParsing {
    * Test init a queue configuration, children's capacity for a given label
    * doesn't equals to 100%. This expect IllegalArgumentException thrown.
    */
-  @Test(expected = IllegalArgumentException.class)
+  @Test(expected = ServiceStateException.class)
   public void testQueueParsingFailWhenSumOfChildrenLabeledCapacityNot100Percent()
       throws IOException {
     nodeLabelManager.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet
@@ -1002,7 +1002,7 @@ public class TestQueueParsing {
    * Test init a queue configuration, children's capacity for a given label
    * doesn't equals to 100%. This expect IllegalArgumentException thrown.
    */
-  @Test(expected = IllegalArgumentException.class)
+  @Test(expected = ServiceStateException.class)
   public void testQueueParsingWithSumOfChildLabelCapacityNot100PercentWithWildCard()
       throws IOException {
     nodeLabelManager.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet
@@ -1066,7 +1066,7 @@ public class TestQueueParsing {
     capacityScheduler.reinitialize(csConf, rmContext);
   }
 
-  @Test(timeout = 60000, expected = java.lang.IllegalArgumentException.class)
+  @Test(timeout = 60000, expected = ServiceStateException.class)
   public void testRMStartWrongNodeCapacity() throws Exception {
     YarnConfiguration config = new YarnConfiguration();
     nodeLabelManager = new NullRMNodeLabelsManager();