Browse Source

MAPREDUCE-4896. mapred queue -info spits out ugly exception when queue does not exist. (sandyr via tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1451452 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 12 năm trước cách đây
mục cha
commit
5d6eca08bd

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

@@ -198,6 +198,9 @@ Release 2.0.4-beta - UNRELEASED
     MAPREDUCE-4693. History server should include counters for failed tasks.
     (Xuan Gong via sseth)
 
+    MAPREDUCE-4896. mapred queue -info spits out ugly exception when queue does 
+    not exist. (sandyr via tucu)
+
 Release 2.0.3-alpha - 2013-02-06 
 
   INCOMPATIBLE CHANGES

+ 3 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java

@@ -106,8 +106,9 @@ public class ResourceMgrDelegate extends YarnClientImpl {
 
   public QueueInfo getQueue(String queueName) throws IOException,
   InterruptedException {
-    return TypeConverter.fromYarn(
-        super.getQueueInfo(queueName), this.conf);
+    org.apache.hadoop.yarn.api.records.QueueInfo queueInfo =
+        super.getQueueInfo(queueName);
+    return (queueInfo == null) ? null : TypeConverter.fromYarn(queueInfo, conf);
   }
 
   public QueueAclsInfo[] getQueueAclsForCurrentUser() throws IOException,

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -436,7 +436,6 @@ public class ClientRMService extends AbstractService implements
       response.setQueueInfo(queueInfo);
     } catch (IOException ioe) {
       LOG.info("Failed to getQueueInfo for " + request.getQueueName(), ioe);
-      throw RPCUtil.getRemoteException(ioe);
     }
     
     return response;

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

@@ -969,7 +969,7 @@ public class FairScheduler implements ResourceScheduler {
   public QueueInfo getQueueInfo(String queueName, boolean includeChildQueues,
       boolean recursive) throws IOException {
     if (!queueMgr.exists(queueName)) {
-      return null;
+      throw new IOException("queue " + queueName + " does not exist");
     }
     return queueMgr.getQueue(queueName).getQueueInfo(includeChildQueues,
         recursive);

+ 8 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java

@@ -19,7 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -176,6 +176,10 @@ public class TestClientRMService {
     List<ApplicationReport> applications = queueInfo.getQueueInfo()
         .getApplications();
     Assert.assertEquals(2, applications.size());
+    request.setQueueName("nonexistentqueue");
+    request.setIncludeApplications(true);
+    // should not throw exception on nonexistent queue
+    queueInfo = rmService.getQueueInfo(request);
   }
 
   private static final UserGroupInformation owner =
@@ -334,8 +338,10 @@ public class TestClientRMService {
     when(rmContext.getDispatcher()).thenReturn(dispatcher);
     QueueInfo queInfo = recordFactory.newRecordInstance(QueueInfo.class);
     queInfo.setQueueName("testqueue");
-    when(yarnScheduler.getQueueInfo(anyString(), anyBoolean(), anyBoolean()))
+    when(yarnScheduler.getQueueInfo(eq("testqueue"), anyBoolean(), anyBoolean()))
         .thenReturn(queInfo);
+    when(yarnScheduler.getQueueInfo(eq("nonexistentqueue"), anyBoolean(), anyBoolean()))
+        .thenThrow(new IOException("queue does not exist"));
     ConcurrentHashMap<ApplicationId, RMApp> apps = getRMApps(rmContext,
         yarnScheduler);
     when(rmContext.getRMApps()).thenReturn(apps);