Преглед изворни кода

YARN-635. Renamed YarnRemoteException to YarnException. Contributed by Siddharth Seth.
MAPREDUCE-5301. Updated MR code to work with YARN-635 changes of renaming YarnRemoteException to YarnException. Contributed by Siddharth Seth.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1489288 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli пре 12 година
родитељ
комит
5f20c37919
100 измењених фајлова са 540 додато и 530 уклоњено
  1. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
  3. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
  4. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java
  5. 11 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
  6. 8 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  7. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/commit/CommitterEventHandler.java
  8. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  9. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  10. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
  11. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
  12. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
  13. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
  14. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
  15. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java
  16. 8 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java
  17. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
  18. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
  19. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
  20. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
  21. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
  22. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
  23. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java
  24. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
  25. 8 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
  26. 9 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java
  27. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
  28. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/TestRPCFactories.java
  29. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/TestRecordFactory.java
  30. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CachedHistoryStorage.java
  31. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
  32. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
  33. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
  34. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
  35. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java
  36. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/MockHistoryContext.java
  37. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java
  38. 10 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
  39. 11 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
  40. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
  41. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
  42. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
  43. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java
  44. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java
  45. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
  46. 3 0
      hadoop-yarn-project/CHANGES.txt
  47. 8 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/YarnRuntimeException.java
  48. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/AMRMProtocol.java
  49. 25 25
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java
  50. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManager.java
  51. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/RMAdminProtocol.java
  52. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnException.java
  53. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factory/providers/RecordFactoryProvider.java
  54. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  55. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
  56. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java
  57. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClient.java
  58. 8 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientAsync.java
  59. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java
  60. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClient.java
  61. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientAsync.java
  62. 13 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java
  63. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/RMAdmin.java
  64. 27 27
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClient.java
  65. 14 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java
  66. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
  67. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
  68. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java
  69. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java
  70. 10 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java
  71. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java
  72. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java
  73. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/AMRMProtocolPBClientImpl.java
  74. 13 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java
  75. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java
  76. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java
  77. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/AMRMProtocolPBServiceImpl.java
  78. 13 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java
  79. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagerPBServiceImpl.java
  80. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java
  81. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
  82. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RecordFactoryPBImpl.java
  83. 8 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java
  84. 11 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcServerFactoryPBImpl.java
  85. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RpcFactoryProvider.java
  86. 12 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/RPCUtil.java
  87. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java
  88. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
  89. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/AdminACLsManager.java
  90. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java
  91. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/CompositeService.java
  92. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java
  93. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java
  94. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebAppException.java
  95. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
  96. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
  97. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPCFactories.java
  98. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRecordFactory.java
  99. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRpcFactoryProvider.java
  100. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestYarnUncaughtExceptionHandler.java

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

@@ -362,6 +362,9 @@ Release 2.1.0-beta - UNRELEASED
     MAPREDUCE-5297. Updated MR App since BuilderUtils is no longer public
     after YARN-748. (Jian He via vinodkv)
 
+    MAPREDUCE-5301. Updated MR code to work with YARN-635 changes of renaming
+    YarnRemoteException to YarnException. (Siddharth Seth via vinodkv)
+
   BREAKDOWN OF HADOOP-8562 SUBTASKS
 
     MAPREDUCE-4739. Some MapReduce tests fail to find winutils.

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java

@@ -50,7 +50,7 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.service.AbstractService;
@@ -129,7 +129,7 @@ public class LocalContainerLauncher extends AbstractService implements
     try {
       eventQueue.put(event);
     } catch (InterruptedException e) {
-      throw new YarnException(e);  // FIXME? YarnException is "for runtime exceptions only"
+      throw new YarnRuntimeException(e);  // FIXME? YarnRuntimeException is "for runtime exceptions only"
     }
   }
 

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java

@@ -52,7 +52,7 @@ import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.util.StringInterner;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.service.CompositeService;
 
 /**
@@ -134,7 +134,7 @@ public class TaskAttemptListenerImpl extends CompositeService
       server.start();
       this.address = NetUtils.getConnectAddress(server);
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java

@@ -30,7 +30,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.service.CompositeService;
@@ -76,7 +76,7 @@ public class JobHistoryCopyService extends CompositeService implements HistoryEv
       //TODO should we parse on a background thread???
       parse();
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
     super.start();
   }

+ 11 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java

@@ -52,7 +52,7 @@ import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.service.AbstractService;
 
@@ -131,7 +131,7 @@ public class JobHistoryEventHandler extends AbstractService
           JobHistoryUtils.getHistoryIntermediateDoneDirForUser(conf);
     } catch (IOException e) {
       LOG.error("Failed while getting the configured log directories", e);
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
 
     //Check for the existence of the history staging dir. Maybe create it. 
@@ -144,7 +144,7 @@ public class JobHistoryEventHandler extends AbstractService
     } catch (IOException e) {
       LOG.error("Failed while checking for/creating  history staging path: ["
           + stagingDirPath + "]", e);
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
 
     //Check for the existence of intermediate done dir.
@@ -178,13 +178,13 @@ public class JobHistoryEventHandler extends AbstractService
                 + ". Either set to true or pre-create this directory with" +
                 " appropriate permissions";
         LOG.error(message);
-        throw new YarnException(message);
+        throw new YarnRuntimeException(message);
       }
       }
     } catch (IOException e) {
       LOG.error("Failed checking for the existance of history intermediate " +
       		"done directory: [" + doneDirPath + "]");
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
 
     //Check/create user directory under intermediate done dir.
@@ -196,7 +196,7 @@ public class JobHistoryEventHandler extends AbstractService
     } catch (IOException e) {
       LOG.error("Error creating user intermediate history done directory: [ "
           + doneDirPrefixPath + "]", e);
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
 
     // Maximum number of unflushed completion-events that can stay in the queue
@@ -457,7 +457,7 @@ public class JobHistoryEventHandler extends AbstractService
 
       eventQueue.put(event);
     } catch (InterruptedException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 
@@ -479,7 +479,7 @@ public class JobHistoryEventHandler extends AbstractService
         } catch (IOException ioe) {
           LOG.error("Error JobHistoryEventHandler in handleEvent: " + event,
               ioe);
-          throw new YarnException(ioe);
+          throw new YarnRuntimeException(ioe);
         }
       }
 
@@ -501,7 +501,7 @@ public class JobHistoryEventHandler extends AbstractService
       } catch (IOException e) {
         LOG.error("Error writing History Event: " + event.getHistoryEvent(),
             e);
-        throw new YarnException(e);
+        throw new YarnRuntimeException(e);
       }
 
       if (event.getHistoryEvent().getEventType() == EventType.JOB_SUBMITTED) {
@@ -523,7 +523,7 @@ public class JobHistoryEventHandler extends AbstractService
           mi.getJobIndexInfo().setJobStatus(JobState.SUCCEEDED.toString());
           closeEventWriter(event.getJobID());
         } catch (IOException e) {
-          throw new YarnException(e);
+          throw new YarnRuntimeException(e);
         }
       }
 
@@ -539,7 +539,7 @@ public class JobHistoryEventHandler extends AbstractService
           mi.getJobIndexInfo().setJobStatus(jucEvent.getStatus());
           closeEventWriter(event.getJobID());
         } catch (IOException e) {
-          throw new YarnException(e);
+          throw new YarnRuntimeException(e);
         }
       }
     }

+ 8 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -113,7 +113,7 @@ import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.ClusterInfo;
 import org.apache.hadoop.yarn.SystemClock;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
@@ -301,7 +301,7 @@ public class MRAppMaster extends CompositeService {
         }
       }
     } catch (IOException e) {
-      throw new YarnException("Error while initializing", e);
+      throw new YarnRuntimeException("Error while initializing", e);
     }
     
     if (errorHappenedShutDown) {
@@ -442,7 +442,7 @@ public class MRAppMaster extends CompositeService {
             .getOutputFormatClass(), conf);
         committer = outputFormat.getOutputCommitter(taskContext);
       } catch (Exception e) {
-        throw new YarnException(e);
+        throw new YarnRuntimeException(e);
       }
     } else {
       committer = ReflectionUtils.newInstance(conf.getClass(
@@ -623,7 +623,7 @@ public class MRAppMaster extends CompositeService {
           + jobTokenFile);
       currentUser.addCredentials(fsTokens); // For use by AppMaster itself.
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 
@@ -662,19 +662,19 @@ public class MRAppMaster extends CompositeService {
     } catch (InstantiationException ex) {
       LOG.error("Can't make a speculator -- check "
           + MRJobConfig.MR_AM_JOB_SPECULATOR, ex);
-      throw new YarnException(ex);
+      throw new YarnRuntimeException(ex);
     } catch (IllegalAccessException ex) {
       LOG.error("Can't make a speculator -- check "
           + MRJobConfig.MR_AM_JOB_SPECULATOR, ex);
-      throw new YarnException(ex);
+      throw new YarnRuntimeException(ex);
     } catch (InvocationTargetException ex) {
       LOG.error("Can't make a speculator -- check "
           + MRJobConfig.MR_AM_JOB_SPECULATOR, ex);
-      throw new YarnException(ex);
+      throw new YarnRuntimeException(ex);
     } catch (NoSuchMethodException ex) {
       LOG.error("Can't make a speculator -- check "
           + MRJobConfig.MR_AM_JOB_SPECULATOR, ex);
-      throw new YarnException(ex);
+      throw new YarnRuntimeException(ex);
     }
   }
 

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/commit/CommitterEventHandler.java

@@ -48,7 +48,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.service.AbstractService;
 
@@ -103,7 +103,7 @@ public class CommitterEventHandler extends AbstractService
       endCommitSuccessFile = MRApps.getEndJobCommitSuccessFile(conf, user, jobId);
       endCommitFailureFile = MRApps.getEndJobCommitFailureFile(conf, user, jobId);
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 
@@ -143,7 +143,7 @@ public class CommitterEventHandler extends AbstractService
     try {
       eventQueue.put(event);
     } catch (InterruptedException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 
@@ -218,7 +218,7 @@ public class CommitterEventHandler extends AbstractService
         handleTaskAbort((CommitterTaskAbortEvent) event);
         break;
       default:
-        throw new YarnException("Unexpected committer event "
+        throw new YarnRuntimeException("Unexpected committer event "
             + event.toString());
       }
     }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -113,7 +113,7 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.Clock;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -1466,7 +1466,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
             job.conf, 
             job.remoteJobSubmitDir);
       } catch (IOException e) {
-        throw new YarnException(e);
+        throw new YarnRuntimeException(e);
       }
       return allTaskSplitMetaInfo;
     }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -111,7 +111,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.Clock;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -733,7 +733,7 @@ public abstract class TaskAttemptImpl implements
             initialAppClasspath);
       }
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
 
     // Shell
@@ -1207,7 +1207,7 @@ public abstract class TaskAttemptImpl implements
     case SUCCEEDED:
       return TaskAttemptState.SUCCEEDED;
     default:
-      throw new YarnException("Attempt to convert invalid "
+      throw new YarnRuntimeException("Attempt to convert invalid "
           + "stateMachineTaskAttemptState to externalTaskAttemptState: "
           + smState);
     }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java

@@ -48,7 +48,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.ContainerManager;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
@@ -423,7 +423,7 @@ public class ContainerLauncherImpl extends AbstractService implements
     try {
       eventQueue.put(event);
     } catch (InterruptedException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java

@@ -35,7 +35,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssigned
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -109,7 +109,7 @@ public class LocalContainerAllocator extends RMCommunicator
         LOG.error("Could not contact RM after " + retryInterval + " milliseconds.");
         eventHandler.handle(new JobEvent(this.getJob().getID(),
                                          JobEventType.INTERNAL_ERROR));
-        throw new YarnException("Could not contact RM after " +
+        throw new YarnRuntimeException("Could not contact RM after " +
                                 retryInterval + " milliseconds.");
       }
       // Throw this up to the caller, which may decide to ignore it and
@@ -122,7 +122,7 @@ public class LocalContainerAllocator extends RMCommunicator
       // this application must clean itself up.
       eventHandler.handle(new JobEvent(this.getJob().getID(),
                                        JobEventType.JOB_AM_REBOOT));
-      throw new YarnException("Resource Manager doesn't recognize AttemptId: " +
+      throw new YarnRuntimeException("Resource Manager doesn't recognize AttemptId: " +
                                this.getContext().getApplicationID());
     }
   }

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java

@@ -39,7 +39,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
@@ -163,7 +163,7 @@ public abstract class RMCommunicator extends AbstractService
       LOG.info("maxContainerCapability: " + maxContainerCapability.getMemory());
     } catch (Exception are) {
       LOG.error("Exception while registering", are);
-      throw new YarnException(are);
+      throw new YarnRuntimeException(are);
     }
   }
 
@@ -237,7 +237,7 @@ public abstract class RMCommunicator extends AbstractService
             Thread.sleep(rmPollInterval);
             try {
               heartbeat();
-            } catch (YarnException e) {
+            } catch (YarnRuntimeException e) {
               LOG.error("Error communicating with RM: " + e.getMessage() , e);
               return;
             } catch (Exception e) {
@@ -273,7 +273,7 @@ public abstract class RMCommunicator extends AbstractService
     try {
       currentUser = UserGroupInformation.getCurrentUser();
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
 
     // CurrentUser should already have AMToken loaded.

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java

@@ -58,7 +58,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
 import org.apache.hadoop.util.StringInterner;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -274,7 +274,7 @@ public class RMContainerAllocator extends RMContainerRequestor
     try {
       eventQueue.put(event);
     } catch (InterruptedException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 
@@ -563,7 +563,7 @@ public class RMContainerAllocator extends RMContainerRequestor
         LOG.error("Could not contact RM after " + retryInterval + " milliseconds.");
         eventHandler.handle(new JobEvent(this.getJob().getID(),
                                          JobEventType.INTERNAL_ERROR));
-        throw new YarnException("Could not contact RM after " +
+        throw new YarnRuntimeException("Could not contact RM after " +
                                 retryInterval + " milliseconds.");
       }
       // Throw this up to the caller, which may decide to ignore it and
@@ -575,7 +575,7 @@ public class RMContainerAllocator extends RMContainerRequestor
       // this application must clean itself up.
       eventHandler.handle(new JobEvent(this.getJob().getID(),
                                        JobEventType.JOB_AM_REBOOT));
-      throw new YarnException("Resource Manager doesn't recognize AttemptId: " +
+      throw new YarnRuntimeException("Resource Manager doesn't recognize AttemptId: " +
                                this.getContext().getApplicationID());
     }
     int newHeadRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0;

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java

@@ -36,14 +36,14 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 
@@ -137,7 +137,7 @@ public abstract class RMContainerRequestor extends RMCommunicator {
             MRJobConfig.DEFAULT_MR_AM_IGNORE_BLACKLISTING_BLACKLISTED_NODE_PERCENT);
     LOG.info("maxTaskFailuresPerNode is " + maxTaskFailuresPerNode);
     if (blacklistDisablePercent < -1 || blacklistDisablePercent > 100) {
-      throw new YarnException("Invalid blacklistDisablePercent: "
+      throw new YarnRuntimeException("Invalid blacklistDisablePercent: "
           + blacklistDisablePercent
           + ". Should be an integer between 0 and 100 or -1 to disabled");
     }
@@ -152,7 +152,7 @@ public abstract class RMContainerRequestor extends RMCommunicator {
     AllocateResponse allocateResponse;
     try {
       allocateResponse = scheduler.allocate(allocateRequest);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
     lastResponseID = allocateResponse.getResponseId();

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java

@@ -48,7 +48,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
 import org.apache.hadoop.yarn.Clock;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.service.AbstractService;
 
@@ -129,16 +129,16 @@ public class DefaultSpeculator extends AbstractService implements
       estimator.contextualize(conf, context);
     } catch (InstantiationException ex) {
       LOG.error("Can't make a speculation runtime extimator", ex);
-      throw new YarnException(ex);
+      throw new YarnRuntimeException(ex);
     } catch (IllegalAccessException ex) {
       LOG.error("Can't make a speculation runtime extimator", ex);
-      throw new YarnException(ex);
+      throw new YarnRuntimeException(ex);
     } catch (InvocationTargetException ex) {
       LOG.error("Can't make a speculation runtime extimator", ex);
-      throw new YarnException(ex);
+      throw new YarnRuntimeException(ex);
     } catch (NoSuchMethodException ex) {
       LOG.error("Can't make a speculation runtime extimator", ex);
-      throw new YarnException(ex);
+      throw new YarnRuntimeException(ex);
     }
     
   return estimator;

+ 8 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java

@@ -58,7 +58,7 @@ import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TasksInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 
@@ -102,8 +102,8 @@ public class AMWebServices {
     Job job;
     try {
       jobId = MRApps.toJobID(jid);
-    } catch (YarnException e) {
-      // TODO: after MAPREDUCE-2793 YarnException is probably not expected here
+    } catch (YarnRuntimeException e) {
+      // TODO: after MAPREDUCE-2793 YarnRuntimeException is probably not expected here
       // anymore but keeping it for now just in case other stuff starts failing.
       // Also, the webservice should ideally return BadRequest (HTTP:400) when
       // the id is malformed instead of NotFound (HTTP:404). The webserver on
@@ -132,8 +132,8 @@ public class AMWebServices {
     Task task;
     try {
       taskID = MRApps.toTaskID(tid);
-    } catch (YarnException e) {
-      // TODO: after MAPREDUCE-2793 YarnException is probably not expected here
+    } catch (YarnRuntimeException e) {
+      // TODO: after MAPREDUCE-2793 YarnRuntimeException is probably not expected here
       // anymore but keeping it for now just in case other stuff starts failing.
       // Also, the webservice should ideally return BadRequest (HTTP:400) when
       // the id is malformed instead of NotFound (HTTP:404). The webserver on
@@ -165,8 +165,8 @@ public class AMWebServices {
     TaskAttempt ta;
     try {
       attemptId = MRApps.toTaskAttemptID(attId);
-    } catch (YarnException e) {
-      // TODO: after MAPREDUCE-2793 YarnException is probably not expected here
+    } catch (YarnRuntimeException e) {
+      // TODO: after MAPREDUCE-2793 YarnRuntimeException is probably not expected here
       // anymore but keeping it for now just in case other stuff starts failing.
       // Also, the webservice should ideally return BadRequest (HTTP:400) when
       // the id is malformed instead of NotFound (HTTP:404). The webserver on
@@ -304,7 +304,7 @@ public class AMWebServices {
       if (type != null && !type.isEmpty()) {
         try {
           ttype = MRApps.taskType(type);
-        } catch (YarnException e) {
+        } catch (YarnRuntimeException e) {
           throw new BadRequestException("tasktype must be either m or r");
         }
       }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java

@@ -44,7 +44,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -254,7 +254,7 @@ public class TestJobHistoryEventHandler {
       return testWorkDir.getAbsolutePath();
     } catch (Exception e) {
       LOG.warn("Could not cleanup", e);
-      throw new YarnException("could not cleanup test dir", e);
+      throw new YarnRuntimeException("could not cleanup test dir", e);
     }
   }
 

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java

@@ -87,7 +87,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.ClusterInfo;
 import org.apache.hadoop.yarn.SystemClock;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -197,7 +197,7 @@ public class MRApp extends MRAppMaster {
         FileContext.getLocalFSFileContext().delete(testAbsPath, true);
       } catch (Exception e) {
         LOG.warn("COULD NOT CLEANUP: " + testAbsPath, e);
-        throw new YarnException("could not cleanup test dir", e);
+        throw new YarnRuntimeException("could not cleanup test dir", e);
       }
     }
 
@@ -215,7 +215,7 @@ public class MRApp extends MRAppMaster {
       FileSystem fs = getFileSystem(conf);
       fs.mkdirs(stagingDir);
     } catch (Exception e) {
-      throw new YarnException("Error creating staging dir", e);
+      throw new YarnRuntimeException("Error creating staging dir", e);
     }
     
     super.init(conf);
@@ -404,7 +404,7 @@ public class MRApp extends MRAppMaster {
     try {
       currentUser = UserGroupInformation.getCurrentUser();
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
     Job newJob = new TestJob(getJobId(), getAttemptID(), conf, 
     		getDispatcher().getEventHandler(),

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssigned
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
@@ -126,7 +126,7 @@ public class MRAppBenchmark {
         try {
           eventQueue.put(event);
         } catch (InterruptedException e) {
-          throw new YarnException(e);
+          throw new YarnRuntimeException(e);
         }
       }
       @Override

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java

@@ -58,7 +58,7 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -450,7 +450,7 @@ class MRAppMasterTest extends MRAppMaster {
     try {
       this.currentUser = UserGroupInformation.getCurrentUser();
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
   

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java

@@ -74,7 +74,7 @@ import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.ClusterInfo;
 import org.apache.hadoop.yarn.SystemClock;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -1429,7 +1429,7 @@ public class TestRMContainerAllocator {
         super.heartbeat();
       } catch (Exception e) {
         LOG.error("error in heartbeat ", e);
-        throw new YarnException(e);
+        throw new YarnRuntimeException(e);
       }
 
       List<TaskAttemptContainerAssignedEvent> result

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java

@@ -49,7 +49,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -292,7 +292,7 @@ import org.junit.Test;
       try {
         currentUser = UserGroupInformation.getCurrentUser();
       } catch (IOException e) {
-        throw new YarnException(e);
+        throw new YarnRuntimeException(e);
       }
       Job newJob = new TestJob(getJobId(), getAttemptID(), conf,
           getDispatcher().getEventHandler(),

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java

@@ -60,7 +60,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.JobCommitCompletedEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobCommitFailedEvent;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Event;
@@ -295,7 +295,7 @@ public class TestCommitterEventHandler {
     when(mockContext.getEventHandler()).thenReturn(waitForItHandler);
     when(mockContext.getClock()).thenReturn(mockClock);
     
-    doThrow(new YarnException("Intentional Failure")).when(mockCommitter)
+    doThrow(new YarnRuntimeException("Intentional Failure")).when(mockCommitter)
       .commitJob(any(JobContext.class));
     
     handler.init(conf);

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java

@@ -61,7 +61,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -460,14 +460,14 @@ public class TestContainerLauncherImpl {
   }
   
   @SuppressWarnings("serial")
-  private static class ContainerException extends YarnRemoteException {
+  private static class ContainerException extends YarnException {
 
     public ContainerException(String message) {
       super(message);
     }
 
     @Override
-    public YarnRemoteException getCause() {
+    public YarnException getCause() {
       return null;
     }
     

+ 8 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java

@@ -30,14 +30,14 @@ import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.yarn.ClusterInfo;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.junit.Assert;
 import org.junit.Test;
@@ -55,13 +55,13 @@ public class TestLocalContainerAllocator {
     try {
       lca.heartbeat();
       Assert.fail("heartbeat was supposed to throw");
-    } catch (YarnRemoteException e) {
-      // YarnRemoteException is expected
+    } catch (YarnException e) {
+      // YarnException is expected
     } finally {
       lca.stop();
     }
 
-    // verify YarnException is thrown when the retry interval has expired
+    // verify YarnRuntimeException is thrown when the retry interval has expired
     conf.setLong(MRJobConfig.MR_AM_TO_RM_WAIT_INTERVAL_MS, 0);
     lca = new StubbedLocalContainerAllocator();
     lca.init(conf);
@@ -69,8 +69,8 @@ public class TestLocalContainerAllocator {
     try {
       lca.heartbeat();
       Assert.fail("heartbeat was supposed to throw");
-    } catch (YarnException e) {
-      // YarnException is expected
+    } catch (YarnRuntimeException e) {
+      // YarnRuntimeException is expected
     } finally {
       lca.stop();
     }
@@ -99,7 +99,7 @@ public class TestLocalContainerAllocator {
       try {
         when(scheduler.allocate(isA(AllocateRequest.class)))
           .thenThrow(RPCUtil.getRemoteException(new IOException("forcefail")));
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
       } catch (IOException e) {
       }
       return scheduler;

+ 9 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java

@@ -41,7 +41,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
@@ -98,7 +98,7 @@ public class TypeConverter {
     case REDUCE:
       return org.apache.hadoop.mapreduce.TaskType.REDUCE;
     default:
-      throw new YarnException("Unrecognized task type: " + taskType);
+      throw new YarnRuntimeException("Unrecognized task type: " + taskType);
     }
   }
 
@@ -110,7 +110,7 @@ public class TypeConverter {
     case REDUCE:
       return TaskType.REDUCE;
     default:
-      throw new YarnException("Unrecognized task type: " + taskType);
+      throw new YarnRuntimeException("Unrecognized task type: " + taskType);
     }
   }
 
@@ -145,7 +145,7 @@ public class TypeConverter {
     case UNASSIGNED:
       return TaskAttemptState.STARTING;
     default:
-      throw new YarnException("Unrecognized State: " + state);
+      throw new YarnRuntimeException("Unrecognized State: " + state);
     }
   }
 
@@ -164,7 +164,7 @@ public class TypeConverter {
     case CLEANUP:
       return Phase.CLEANUP;
     }
-    throw new YarnException("Unrecognized Phase: " + phase);
+    throw new YarnRuntimeException("Unrecognized Phase: " + phase);
   }
 
   public static TaskCompletionEvent[] fromYarn(
@@ -202,7 +202,7 @@ public class TypeConverter {
     case TIPFAILED:
       return TaskCompletionEvent.Status.TIPFAILED;
     }
-    throw new YarnException("Unrecognized status: " + newStatus);
+    throw new YarnRuntimeException("Unrecognized status: " + newStatus);
   }
 
   public static org.apache.hadoop.mapred.TaskAttemptID fromYarn(
@@ -328,7 +328,7 @@ public class TypeConverter {
     case ERROR:
       return org.apache.hadoop.mapred.JobStatus.FAILED;
     }
-    throw new YarnException("Unrecognized job state: " + state);
+    throw new YarnRuntimeException("Unrecognized job state: " + state);
   }
 
   public static org.apache.hadoop.mapred.TIPStatus fromYarn(
@@ -346,7 +346,7 @@ public class TypeConverter {
     case FAILED:
       return org.apache.hadoop.mapred.TIPStatus.FAILED;
     }
-    throw new YarnException("Unrecognized task state: " + state);
+    throw new YarnRuntimeException("Unrecognized task state: " + state);
   }
 
   public static TaskReport fromYarn(org.apache.hadoop.mapreduce.v2.api.records.TaskReport report) {
@@ -408,7 +408,7 @@ public class TypeConverter {
     case KILLED:
       return State.KILLED;
     }
-    throw new YarnException("Unrecognized application state: " + yarnApplicationState);
+    throw new YarnRuntimeException("Unrecognized application state: " + yarnApplicationState);
   }
 
   private static final String TT_NAME_PREFIX = "tracker_";

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java

@@ -49,7 +49,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.yarn.ContainerLogAppender;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.LocalResource;
@@ -97,7 +97,7 @@ public class MRApps extends Apps {
       case MAP:           return "m";
       case REDUCE:        return "r";
     }
-    throw new YarnException("Unknown task type: "+ type.toString());
+    throw new YarnRuntimeException("Unknown task type: "+ type.toString());
   }
 
   public static enum TaskAttemptStateUI {
@@ -126,7 +126,7 @@ public class MRApps extends Apps {
     // JDK 7 supports switch on strings
     if (symbol.equals("m")) return TaskType.MAP;
     if (symbol.equals("r")) return TaskType.REDUCE;
-    throw new YarnException("Unknown task symbol: "+ symbol);
+    throw new YarnRuntimeException("Unknown task symbol: "+ symbol);
   }
 
   public static TaskAttemptStateUI taskAttemptState(String attemptStateStr) {

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/TestRPCFactories.java

@@ -56,7 +56,7 @@ import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskResponse;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.impl.pb.RpcClientFactoryPBImpl;
 import org.apache.hadoop.yarn.factories.impl.pb.RpcServerFactoryPBImpl;
 import org.junit.Test;
@@ -84,7 +84,7 @@ public class TestRPCFactories {
         RpcServerFactoryPBImpl.get().getServer(
           MRClientProtocol.class, instance, addr, conf, null, 1);
       server.start();
-    } catch (YarnException e) {
+    } catch (YarnRuntimeException e) {
       e.printStackTrace();
       Assert.fail("Failed to crete server");
     } finally {
@@ -110,12 +110,12 @@ public class TestRPCFactories {
       MRClientProtocol client = null;
       try {
         client = (MRClientProtocol) RpcClientFactoryPBImpl.get().getClient(MRClientProtocol.class, 1, NetUtils.getConnectAddress(server), conf);
-      } catch (YarnException e) {
+      } catch (YarnRuntimeException e) {
         e.printStackTrace();
         Assert.fail("Failed to crete client");
       }
       
-    } catch (YarnException e) {
+    } catch (YarnRuntimeException e) {
       e.printStackTrace();
       Assert.fail("Failed to crete server");
     } finally {

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/TestRecordFactory.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.mapreduce.v2;
 
 import junit.framework.Assert;
 
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.impl.pb.RecordFactoryPBImpl;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetCountersRequest;
@@ -38,7 +38,7 @@ public class TestRecordFactory {
     try {
       CounterGroup response = pbRecordFactory.newRecordInstance(CounterGroup.class);
       Assert.assertEquals(CounterGroupPBImpl.class, response.getClass());
-    } catch (YarnException e) {
+    } catch (YarnRuntimeException e) {
       e.printStackTrace();
       Assert.fail("Failed to crete record");
     }
@@ -46,7 +46,7 @@ public class TestRecordFactory {
     try {
       GetCountersRequest response = pbRecordFactory.newRecordInstance(GetCountersRequest.class);
       Assert.assertEquals(GetCountersRequestPBImpl.class, response.getClass());
-    } catch (YarnException e) {
+    } catch (YarnRuntimeException e) {
       e.printStackTrace();
       Assert.fail("Failed to crete record");
     }

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CachedHistoryStorage.java

@@ -37,7 +37,7 @@ import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo;
 import org.apache.hadoop.mapreduce.v2.hs.HistoryFileManager.HistoryFileInfo;
 import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.service.AbstractService;
 
 /**
@@ -60,7 +60,7 @@ public class CachedHistoryStorage extends AbstractService implements
 
   @SuppressWarnings("serial")
   @Override
-  public void init(Configuration conf) throws YarnException {
+  public void init(Configuration conf) throws YarnRuntimeException {
     LOG.info("CachedHistoryStorage Init");
 
     loadedJobCacheSize = conf.getInt(
@@ -94,7 +94,7 @@ public class CachedHistoryStorage extends AbstractService implements
       loadedJobCache.put(job.getID(), job);
       return job;
     } catch (IOException e) {
-      throw new YarnException(
+      throw new YarnRuntimeException(
           "Could not find/load job: " + fileInfo.getJobId(), e);
     }
   }
@@ -120,7 +120,7 @@ public class CachedHistoryStorage extends AbstractService implements
       }
       return result;
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 
@@ -137,7 +137,7 @@ public class CachedHistoryStorage extends AbstractService implements
       }
     } catch (IOException e) {
       LOG.warn("Error trying to scan for all FileInfos", e);
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
     return result;
   }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java

@@ -60,7 +60,7 @@ import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.Records;
 
 
@@ -333,12 +333,12 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
                 historyFileAbsolute);
         this.jobInfo = parser.parse();
       } catch (IOException e) {
-        throw new YarnException("Could not load history file "
+        throw new YarnRuntimeException("Could not load history file "
             + historyFileAbsolute, e);
       }
       IOException parseException = parser.getParseException(); 
       if (parseException != null) {
-        throw new YarnException(
+        throw new YarnRuntimeException(
             "Could not parse history file " + historyFileAbsolute, 
             parseException);
       }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java

@@ -61,7 +61,7 @@ import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.service.AbstractService;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -477,7 +477,7 @@ public class HistoryFileManager extends AbstractService {
       mkdir(doneDirFc, doneDirPrefixPath, new FsPermission(
           JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION));
     } catch (IOException e) {
-      throw new YarnException("Error creating done directory: ["
+      throw new YarnRuntimeException("Error creating done directory: ["
           + doneDirPrefixPath + "]", e);
     }
 
@@ -493,7 +493,7 @@ public class HistoryFileManager extends AbstractService {
           JobHistoryUtils.HISTORY_INTERMEDIATE_DONE_DIR_PERMISSIONS.toShort()));
     } catch (IOException e) {
       LOG.info("error creating done directory on dfs " + e);
-      throw new YarnException("Error creating intermediate done directory: ["
+      throw new YarnRuntimeException("Error creating intermediate done directory: ["
           + intermediateDoneDirPath + "]", e);
     }
 

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java

@@ -40,7 +40,7 @@ import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.ClusterInfo;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -71,7 +71,7 @@ public class JobHistory extends AbstractService implements HistoryContext {
   private HistoryFileManager hsManager = null;
 
   @Override
-  public void init(Configuration conf) throws YarnException {
+  public void init(Configuration conf) throws YarnRuntimeException {
     LOG.info("JobHistory Init");
     this.conf = conf;
     this.appID = ApplicationId.newInstance(0, 0);
@@ -87,7 +87,7 @@ public class JobHistory extends AbstractService implements HistoryContext {
     try {
       hsManager.initExisting();
     } catch (IOException e) {
-      throw new YarnException("Failed to intialize existing directories", e);
+      throw new YarnRuntimeException("Failed to intialize existing directories", e);
     }
 
     storage = ReflectionUtils.newInstance(conf.getClass(

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -73,7 +73,7 @@ public class JobHistoryServer extends CompositeService {
     try {
       doSecureLogin(conf);
     } catch(IOException ie) {
-      throw new YarnException("History Server Failed to login", ie);
+      throw new YarnRuntimeException("History Server Failed to login", ie);
     }
     jobHistoryService = new JobHistory();
     historyContext = (HistoryContext)jobHistoryService;

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java

@@ -55,7 +55,7 @@ import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.HistoryInfo;
 import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
 import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.apache.hadoop.yarn.webapp.WebApp;
@@ -255,7 +255,7 @@ public class HsWebServices {
       if (type != null && !type.isEmpty()) {
         try {
           ttype = MRApps.taskType(type);
-        } catch (YarnException e) {
+        } catch (YarnRuntimeException e) {
           throw new BadRequestException("tasktype must be either m or r");
         }
       }

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/MockHistoryContext.java

@@ -28,7 +28,7 @@ import org.apache.hadoop.mapreduce.v2.app.MockAppContext;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.hs.MockHistoryJobs.JobsPair;
 import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 
 public class MockHistoryContext extends MockAppContext implements HistoryContext {
@@ -42,7 +42,7 @@ public class MockHistoryContext extends MockAppContext implements HistoryContext
     try {
       jobs = MockHistoryJobs.newHistoryJobs(numJobs, numTasks, numAttempts);
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
     partialJobs = jobs.partial;
     fullJobs = jobs.full;
@@ -56,7 +56,7 @@ public class MockHistoryContext extends MockAppContext implements HistoryContext
       jobs = MockHistoryJobs.newHistoryJobs(getApplicationID(), numJobs, numTasks,
           numAttempts);
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
     partialJobs = jobs.partial;
     fullJobs = jobs.full;
@@ -76,7 +76,7 @@ public class MockHistoryContext extends MockAppContext implements HistoryContext
       jobs = MockHistoryJobs.newHistoryJobs(getApplicationID(), numJobs, numTasks,
           numAttempts, hasFailedTasks);
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
     partialJobs = jobs.partial;
     fullJobs = jobs.full;

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

@@ -33,7 +33,7 @@ import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 
 public class ClientCache {
@@ -60,7 +60,7 @@ public class ClientCache {
         hsProxy = instantiateHistoryProxy();
       } catch (IOException e) {
         LOG.warn("Could not connect to History server.", e);
-        throw new YarnException("Could not connect to History server.", e);
+        throw new YarnRuntimeException("Could not connect to History server.", e);
       }
     }
     ClientServiceDelegate client = cache.get(jobId);

+ 10 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java

@@ -65,12 +65,12 @@ import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -143,7 +143,7 @@ public class ClientServiceDelegate {
     ApplicationReport application = null;
     try {
       application = rm.getApplicationReport(appId);
-    } catch (YarnRemoteException e2) {
+    } catch (YarnException e2) {
       throw new IOException(e2);
     }
     if (application != null) {
@@ -212,11 +212,11 @@ public class ClientServiceDelegate {
           Thread.sleep(2000);
         } catch (InterruptedException e1) {
           LOG.warn("getProxy() call interruped", e1);
-          throw new YarnException(e1);
+          throw new YarnRuntimeException(e1);
         }
         try {
           application = rm.getApplicationReport(appId);
-        } catch (YarnRemoteException e1) {
+        } catch (YarnException e1) {
           throw new IOException(e1);
         }
         if (application == null) {
@@ -226,8 +226,8 @@ public class ClientServiceDelegate {
         }
       } catch (InterruptedException e) {
         LOG.warn("getProxy() call interruped", e);
-        throw new YarnException(e);
-      } catch (YarnRemoteException e) {
+        throw new YarnRuntimeException(e);
+      } catch (YarnException e) {
         throw new IOException(e);
       }
     }
@@ -296,9 +296,9 @@ public class ClientServiceDelegate {
     try {
       methodOb = MRClientProtocol.class.getMethod(method, argClass);
     } catch (SecurityException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (NoSuchMethodException e) {
-      throw new YarnException("Method name mismatch", e);
+      throw new YarnRuntimeException("Method name mismatch", e);
     }
     int maxRetries = this.conf.getInt(
         MRJobConfig.MR_CLIENT_MAX_RETRIES,
@@ -308,7 +308,7 @@ public class ClientServiceDelegate {
       try {
         return methodOb.invoke(getProxy(), args);
       } catch (InvocationTargetException e) {
-        // Will not throw out YarnRemoteException anymore
+        // Will not throw out YarnException anymore
         LOG.debug("Failed to contact AM/History for job " + jobId + 
             " retrying..", e.getTargetException());
         // Force reconnection by setting the proxy to null.

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

@@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.client.YarnClientImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.ProtoUtils;
 
 public class ResourceMgrDelegate extends YarnClientImpl {
@@ -68,7 +68,7 @@ public class ResourceMgrDelegate extends YarnClientImpl {
       InterruptedException {
     try {
       return TypeConverter.fromYarnNodes(super.getNodeReports());
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }
@@ -76,7 +76,7 @@ public class ResourceMgrDelegate extends YarnClientImpl {
   public JobStatus[] getAllJobs() throws IOException, InterruptedException {
     try {
       return TypeConverter.fromYarnApps(super.getApplicationList(), this.conf);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }
@@ -98,7 +98,7 @@ public class ResourceMgrDelegate extends YarnClientImpl {
               metrics.getNumNodeManagers() * 2, 1,
               metrics.getNumNodeManagers(), 0, 0);
       return oldMetrics;
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }
@@ -113,7 +113,7 @@ public class ResourceMgrDelegate extends YarnClientImpl {
     try {
       return ProtoUtils.convertFromProtoFormat(
         super.getRMDelegationToken(renewer), rmAddress);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }
@@ -127,7 +127,7 @@ public class ResourceMgrDelegate extends YarnClientImpl {
       this.application = super.getNewApplication();
       this.applicationId = this.application.getApplicationId();
       return TypeConverter.fromYarn(applicationId);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }
@@ -139,7 +139,7 @@ public class ResourceMgrDelegate extends YarnClientImpl {
           super.getQueueInfo(queueName);
       return (queueInfo == null) ? null : TypeConverter.fromYarn(queueInfo,
           conf);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }
@@ -149,7 +149,7 @@ public class ResourceMgrDelegate extends YarnClientImpl {
     try {
       return TypeConverter.fromYarnQueueUserAclsInfo(super
         .getQueueAclsInfo());
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }
@@ -157,7 +157,7 @@ public class ResourceMgrDelegate extends YarnClientImpl {
   public QueueInfo[] getQueues() throws IOException, InterruptedException {
     try {
       return TypeConverter.fromYarnQueueInfo(super.getAllQueues(), this.conf);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }
@@ -166,7 +166,7 @@ public class ResourceMgrDelegate extends YarnClientImpl {
     try {
       return TypeConverter.fromYarnQueueInfo(super.getRootQueueInfos(),
           this.conf);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }
@@ -176,7 +176,7 @@ public class ResourceMgrDelegate extends YarnClientImpl {
     try {
       return TypeConverter.fromYarnQueueInfo(super.getChildQueueInfos(parent),
         this.conf);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java

@@ -64,7 +64,7 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
@@ -79,7 +79,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.client.RMTokenSelector;
@@ -287,7 +287,7 @@ public class YARNRunner implements ClientProtocol {
     try {
       ts.writeTokenStorageFile(applicationTokensFile, conf);
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
 
     // Construct necessary information to start the MR AM
@@ -311,7 +311,7 @@ public class YARNRunner implements ClientProtocol {
             diagnostics);
       }
       return clientCache.getClient(jobId).getJobStatus(jobId);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }
@@ -561,7 +561,7 @@ public class YARNRunner implements ClientProtocol {
     if (status.getState() != JobStatus.State.RUNNING) {
       try {
         resMgrDelegate.killApplication(TypeConverter.toYarn(arg0).getAppId());
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         throw new IOException(e);
       }
       return;
@@ -589,7 +589,7 @@ public class YARNRunner implements ClientProtocol {
     if (status.getState() != JobStatus.State.KILLED) {
       try {
         resMgrDelegate.killApplication(TypeConverter.toYarn(arg0).getAppId());
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         throw new IOException(e);
       }
     }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java

@@ -68,7 +68,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
@@ -404,7 +404,7 @@ public class TestClientRedirect {
         address.getAddress();
         hostNameResolved = InetAddress.getLocalHost();
       } catch (UnknownHostException e) {
-        throw new YarnException(e);
+        throw new YarnRuntimeException(e);
       }
 
       server =

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java

@@ -56,7 +56,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -215,7 +215,7 @@ public class TestClientServiceDelegate {
           getRunningApplicationReport(null, 0)).thenReturn(
           getRunningApplicationReport(null, 0)).thenReturn(
           getRunningApplicationReport("am2", 90));
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
 
@@ -285,7 +285,7 @@ public class TestClientServiceDelegate {
             getRunningApplicationReport("am1", 78)).thenReturn(
               getRunningApplicationReport("am1", 78)).thenReturn(
           getFinishedApplicationReport());
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
 
@@ -366,7 +366,7 @@ public class TestClientServiceDelegate {
       verify(rmDelegate, times(3)).getApplicationReport(
           any(ApplicationId.class));
       Assert.assertNotNull(jobStatus);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }
@@ -396,7 +396,7 @@ public class TestClientServiceDelegate {
       }
       verify(rmDelegate, times(noOfRetries)).getApplicationReport(
           any(ApplicationId.class));
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
   }  
@@ -447,7 +447,7 @@ public class TestClientServiceDelegate {
     ResourceMgrDelegate rm = mock(ResourceMgrDelegate.class);
     try {
       when(rm.getApplicationReport(jobId.getAppId())).thenReturn(null);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
     return rm;

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java

@@ -45,7 +45,7 @@ import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.junit.Test;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -181,12 +181,12 @@ public class TestNetworkedJob {
       
       try {
         client.getSetupTaskReports(jobId);
-      } catch (YarnException e) {
+      } catch (YarnRuntimeException e) {
         assertEquals(e.getMessage(), "Unrecognized task type: JOB_SETUP");
       }
       try {
         client.getCleanupTaskReports(jobId);
-      } catch (YarnException e) {
+      } catch (YarnRuntimeException e) {
         assertEquals(e.getMessage(), "Unrecognized task type: JOB_CLEANUP");
       }
       assertEquals(client.getReduceTaskReports(jobId).length, 0);

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java

@@ -38,7 +38,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
@@ -60,7 +60,7 @@ public class TestResourceMgrDelegate {
     try {
       Mockito.when(applicationsManager.getQueueInfo(Mockito.any(
         GetQueueInfoRequest.class))).thenReturn(response);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
 
@@ -78,7 +78,7 @@ public class TestResourceMgrDelegate {
     try {
       Mockito.verify(applicationsManager).getQueueInfo(
         argument.capture());
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new IOException(e);
     }
 

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java

@@ -35,7 +35,7 @@ import org.apache.hadoop.mapreduce.v2.hs.JobHistoryServer;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.util.JarFinder;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
@@ -100,7 +100,7 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
       Path doneDirPath = fc.makeQualified(new Path(doneDir));
       fc.mkdir(doneDirPath, null, true);
     } catch (IOException e) {
-      throw new YarnException("Could not create staging directory. ", e);
+      throw new YarnRuntimeException("Could not create staging directory. ", e);
     }
     conf.set(MRConfig.MASTER_ADDRESS, "test"); // The default is local because of
                                              // which shuffle doesn't happen
@@ -158,7 +158,7 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
         }
         super.start();
       } catch (Throwable t) {
-        throw new YarnException(t);
+        throw new YarnRuntimeException(t);
       }
       //need to do this because historyServer.init creates a new Configuration
       getConfig().set(JHAdminConfig.MR_HISTORY_ADDRESS,

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

@@ -70,6 +70,9 @@ Release 2.1.0-beta - UNRELEASED
     YARN-748. Moved BuilderUtils from yarn-common to yarn-server-common for
     eventual retirement. (Jian He via vinodkv)
 
+    YARN-635. Renamed YarnRemoteException to YarnException. (Siddharth Seth via
+    vinodkv)
+
   NEW FEATURES
 
     YARN-482. FS: Extend SchedulingMode to intermediate queues. 

+ 8 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/YarnException.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/YarnRuntimeException.java

@@ -24,10 +24,14 @@ package org.apache.hadoop.yarn;
  * service, must include a String only constructor for the exception to be 
  * unwrapped on the client.
  */
-public class YarnException extends RuntimeException {
-  public YarnException(Throwable cause) { super(cause); }
-  public YarnException(String message) { super(message); }
-  public YarnException(String message, Throwable cause) {
+public class YarnRuntimeException extends RuntimeException {
+  /**
+   * 
+   */
+  private static final long serialVersionUID = -7153142425412203936L;
+  public YarnRuntimeException(Throwable cause) { super(cause); }
+  public YarnRuntimeException(String message) { super(message); }
+  public YarnRuntimeException(String message, Throwable cause) {
     super(message, cause);
   }
 }

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/AMRMProtocol.java

@@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterReque
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 
 /**
  * <p>The protocol between a live instance of <code>ApplicationMaster</code> 
@@ -58,12 +58,12 @@ public interface AMRMProtocol {
    *  
    * @param request registration request
    * @return registration respose
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public RegisterApplicationMasterResponse registerApplicationMaster(
       RegisterApplicationMasterRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   /**
    * <p>The interface used by an <code>ApplicationMaster</code> to notify the 
@@ -78,12 +78,12 @@ public interface AMRMProtocol {
    * 
    * @param request completion request
    * @return completion response
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public FinishApplicationMasterResponse finishApplicationMaster(
       FinishApplicationMasterRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   /**
    * <p>The main interface between an <code>ApplicationMaster</code> 
@@ -108,9 +108,9 @@ public interface AMRMProtocol {
    * 
    * @param request allocation request
    * @return allocation response
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public AllocateResponse allocate(AllocateRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
 }

+ 25 - 25
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java

@@ -54,7 +54,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 
 /**
  * <p>The protocol between clients and the <code>ResourceManager</code>
@@ -79,13 +79,13 @@ public interface ClientRMProtocol {
    * @param request request to get a new <code>ApplicationId</code>
    * @return response containing the new <code>ApplicationId</code> to be used
    * to submit an application
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    * @see #submitApplication(SubmitApplicationRequest)
    */
   public GetNewApplicationResponse getNewApplication(
       GetNewApplicationRequest request)
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   /**
    * <p>The interface used by clients to submit a new application to the
@@ -108,13 +108,13 @@ public interface ClientRMProtocol {
    * 
    * @param request request to submit a new application
    * @return (empty) response on accepting the submission
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    * @see #getNewApplication(GetNewApplicationRequest)
    */
   public SubmitApplicationResponse submitApplication(
       SubmitApplicationRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   /**
    * <p>The interface used by clients to request the 
@@ -132,13 +132,13 @@ public interface ClientRMProtocol {
    * @param request request to abort a submited application
    * @return <code>ResourceManager</code> returns an empty response
    *         on success and throws an exception on rejecting the request
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    * @see #getQueueUserAcls(GetQueueUserAclsInfoRequest) 
    */
   public KillApplicationResponse forceKillApplication(
       KillApplicationRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
 
   /**
    * <p>The interface used by clients to get a report of an Application from
@@ -168,12 +168,12 @@ public interface ClientRMProtocol {
    *
    * @param request request for an application report
    * @return application report 
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public GetApplicationReportResponse getApplicationReport(
       GetApplicationReportRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   /**
    * <p>The interface used by clients to get metrics about the cluster from
@@ -186,12 +186,12 @@ public interface ClientRMProtocol {
    * 
    * @param request request for cluster metrics
    * @return cluster metrics
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public GetClusterMetricsResponse getClusterMetrics(
       GetClusterMetricsRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   /**
    * <p>The interface used by clients to get a report of all Applications
@@ -208,12 +208,12 @@ public interface ClientRMProtocol {
    *
    * @param request request for report on all running applications
    * @return report on all running applications
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public GetAllApplicationsResponse getAllApplications(
       GetAllApplicationsRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   /**
    * <p>The interface used by clients to get a report of all nodes
@@ -225,12 +225,12 @@ public interface ClientRMProtocol {
    * 
    * @param request request for report on all nodes
    * @return report on all nodes
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public GetClusterNodesResponse getClusterNodes(
       GetClusterNodesRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   /**
    * <p>The interface used by clients to get information about <em>queues</em>
@@ -244,12 +244,12 @@ public interface ClientRMProtocol {
    * 
    * @param request request to get queue information
    * @return queue information
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public GetQueueInfoResponse getQueueInfo(
       GetQueueInfoRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   /**
    * <p>The interface used by clients to get information about <em>queue 
@@ -261,12 +261,12 @@ public interface ClientRMProtocol {
    * 
    * @param request request to get queue acls for <em>current user</em>
    * @return queue acls for <em>current user</em>
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public GetQueueUserAclsInfoResponse getQueueUserAcls(
       GetQueueUserAclsInfoRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   /**
    * <p>The interface used by clients to get delegation token, enabling the 
@@ -277,24 +277,24 @@ public interface ClientRMProtocol {
    *  service.
    * @param request request to get a delegation token for the client.
    * @return delegation token that can be used to talk to this service
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public GetDelegationTokenResponse getDelegationToken(
       GetDelegationTokenRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   /**
    * Renew an existing delegation token.
    * 
    * @param request the delegation token to be renewed.
    * @return the new expiry time for the delegation token.
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   @Private
   public RenewDelegationTokenResponse renewDelegationToken(
-      RenewDelegationTokenRequest request) throws YarnRemoteException,
+      RenewDelegationTokenRequest request) throws YarnException,
       IOException;
 
   /**
@@ -302,11 +302,11 @@ public interface ClientRMProtocol {
    * 
    * @param request the delegation token to be cancelled.
    * @return an empty response.
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   @Private
   public CancelDelegationTokenResponse cancelDelegationToken(
-      CancelDelegationTokenRequest request) throws YarnRemoteException,
+      CancelDelegationTokenRequest request) throws YarnException,
       IOException;
 }

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManager.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 
 /**
  * <p>The protocol between an <code>ApplicationMaster</code> and a 
@@ -69,13 +69,13 @@ public interface ContainerManager {
    * @param request request to start a container
    * @return empty response to indicate acceptance of the request 
    *         or an exception
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   @Public
   @Stable
   StartContainerResponse startContainer(StartContainerRequest request)
-      throws YarnRemoteException, IOException;
+      throws YarnException, IOException;
 
   /**
    * <p>The <code>ApplicationMaster</code> requests a <code>NodeManager</code>
@@ -96,13 +96,13 @@ public interface ContainerManager {
    * @param request request to stop a container
    * @return empty response to indicate acceptance of the request 
    *         or an exception
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   @Public
   @Stable
   StopContainerResponse stopContainer(StopContainerRequest request)
-      throws YarnRemoteException, IOException;
+      throws YarnException, IOException;
 
   /**
    * <p>The api used by the <code>ApplicationMaster</code> to request for 
@@ -121,12 +121,12 @@ public interface ContainerManager {
    *                with the specified <code>ContainerId</code>
    * @return response containing the <code>ContainerStatus</code> of the
    * container
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   @Public
   @Stable
   GetContainerStatusResponse getContainerStatus(
-      GetContainerStatusRequest request) throws YarnRemoteException,
+      GetContainerStatusRequest request) throws YarnException,
       IOException;
 }

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/RMAdminProtocol.java

@@ -21,7 +21,7 @@ package org.apache.hadoop.yarn.api;
 import java.io.IOException;
 
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesRequest;
@@ -37,25 +37,25 @@ import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsRes
 
 public interface RMAdminProtocol extends GetUserMappingsProtocol {
   public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request) 
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   public RefreshSuperUserGroupsConfigurationResponse 
   refreshSuperUserGroupsConfiguration(
       RefreshSuperUserGroupsConfigurationRequest request)
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
 
   public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
       RefreshUserToGroupsMappingsRequest request)
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   public RefreshAdminAclsResponse refreshAdminAcls(
       RefreshAdminAclsRequest request)
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
   
   public RefreshServiceAclsResponse refreshServiceAcls(
       RefreshServiceAclsRequest request)
-  throws YarnRemoteException, IOException;
+  throws YarnException, IOException;
 }

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnRemoteException.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnException.java

@@ -18,23 +18,23 @@
 
 package org.apache.hadoop.yarn.exceptions;
 
-public class YarnRemoteException extends Exception {
+public class YarnException extends Exception {
 
   private static final long serialVersionUID = 1L;
 
-  public YarnRemoteException() {
+  public YarnException() {
     super();
   }
 
-  public YarnRemoteException(String message) {
+  public YarnException(String message) {
     super(message);
   }
 
-  public YarnRemoteException(Throwable cause) {
+  public YarnException(Throwable cause) {
     super(cause);
   }
 
-  public YarnRemoteException(String message, Throwable cause) {
+  public YarnException(String message, Throwable cause) {
     super(message, cause);
   }
 

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factory/providers/RecordFactoryProvider.java

@@ -22,7 +22,7 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 
@@ -55,13 +55,13 @@ public class RecordFactoryProvider {
       method.setAccessible(true);
       return method.invoke(null, null);
     } catch (ClassNotFoundException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (NoSuchMethodException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (InvocationTargetException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (IllegalAccessException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 }

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java

@@ -72,7 +72,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.client.AMRMClientAsync;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -434,10 +434,10 @@ public class ApplicationMaster {
   /**
    * Main run function for the application master
    *
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
-  public boolean run() throws YarnRemoteException, IOException {
+  public boolean run() throws YarnException, IOException {
     LOG.info("Starting ApplicationMaster");
 
     AMRMClientAsync.CallbackHandler allocListener = new RMCallbackHandler();
@@ -537,7 +537,7 @@ public class ApplicationMaster {
     }
     try {
       resourceManager.unregisterApplicationMaster(appStatus, appMessage, null);
-    } catch (YarnRemoteException ex) {
+    } catch (YarnException ex) {
       LOG.error("Failed to unregister application", ex);
     } catch (IOException e) {
       LOG.error("Failed to unregister application", e);
@@ -777,7 +777,7 @@ public class ApplicationMaster {
       startReq.setContainerToken(container.getContainerToken());
       try {
         cm.startContainer(startReq);
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         LOG.info("Start container failed for :" + ", containerId="
             + container.getId());
         e.printStackTrace();
@@ -802,7 +802,7 @@ public class ApplicationMaster {
       // LOG.info("Container Status"
       // + ", id=" + container.getId()
       // + ", status=" +statusResp.getStatus());
-      // } catch (YarnRemoteException e) {
+      // } catch (YarnException e) {
       // e.printStackTrace();
       // }
     }

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java

@@ -62,7 +62,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.client.YarnClientImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
 
@@ -312,9 +312,9 @@ public class Client extends YarnClientImpl {
    * Main run function for the client
    * @return true if application completed successfully
    * @throws IOException
-   * @throws YarnRemoteException
+   * @throws YarnException
    */
-  public boolean run() throws IOException, YarnRemoteException {
+  public boolean run() throws IOException, YarnException {
 
     LOG.info("Running Client");
     start();
@@ -591,11 +591,11 @@ public class Client extends YarnClientImpl {
    * Kill application if time expires. 
    * @param appId Application Id of application to be monitored
    * @return true if application completed successfully
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   private boolean monitorApplication(ApplicationId appId)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
 
     while (true) {
 
@@ -656,11 +656,11 @@ public class Client extends YarnClientImpl {
   /**
    * Kill a submitted application by sending a call to the ASM
    * @param appId Application Id to be killed. 
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   private void forceKillApplication(ApplicationId appId)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     // TODO clarify whether multiple jobs with the same app id can be submitted and be running at 
     // the same time. 
     // If yes, can we kill a particular attempt only?

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java

@@ -50,7 +50,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.YarnClientImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
@@ -268,7 +268,7 @@ public class UnmanagedAMLauncher {
     amProc.destroy();
   }
   
-  public boolean run() throws IOException, YarnRemoteException {
+  public boolean run() throws IOException, YarnException {
     LOG.info("Starting Client");
     
     // Connect to ResourceManager
@@ -353,11 +353,11 @@ public class UnmanagedAMLauncher {
    * @param appId
    *          Application Id of application to be monitored
    * @return true if application completed successfully
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   private ApplicationReport monitorApplication(ApplicationId appId,
-      Set<YarnApplicationState> finalState) throws YarnRemoteException,
+      Set<YarnApplicationState> finalState) throws YarnException,
       IOException {
 
     long foundAMCompletedTime = 0;

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClient.java

@@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.service.Service;
 
 import com.google.common.collect.ImmutableList;
@@ -113,14 +113,14 @@ public interface AMRMClient<T extends AMRMClient.ContainerRequest> extends Servi
    * @param appHostPort Port master is listening on
    * @param appTrackingUrl URL at which the master info can be seen
    * @return <code>RegisterApplicationMasterResponse</code>
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public RegisterApplicationMasterResponse 
                registerApplicationMaster(String appHostName,
                                          int appHostPort,
                                          String appTrackingUrl) 
-               throws YarnRemoteException, IOException;
+               throws YarnException, IOException;
   
   /**
    * Request additional containers and receive new container allocations.
@@ -134,24 +134,24 @@ public interface AMRMClient<T extends AMRMClient.ContainerRequest> extends Servi
    * App should not make concurrent allocate requests. May cause request loss.
    * @param progressIndicator Indicates progress made by the master
    * @return the response of the allocate request
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public AllocateResponse allocate(float progressIndicator) 
-                           throws YarnRemoteException, IOException;
+                           throws YarnException, IOException;
   
   /**
    * Unregister the application master. This must be called in the end.
    * @param appStatus Success/Failure status of the master
    * @param appMessage Diagnostics message on failure
    * @param appTrackingUrl New URL to get master info
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public void unregisterApplicationMaster(FinalApplicationStatus appStatus,
                                            String appMessage,
                                            String appTrackingUrl) 
-               throws YarnRemoteException, IOException;
+               throws YarnException, IOException;
   
   /**
    * Request containers for resources before calling <code>allocate</code>

+ 8 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientAsync.java

@@ -31,7 +31,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.service.AbstractService;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -152,7 +152,7 @@ public class AMRMClientAsync<T extends ContainerRequest> extends AbstractService
   @Override
   public void stop() {
     if (Thread.currentThread() == handlerThread) {
-      throw new YarnException("Cannot call stop from callback handler thread!");
+      throw new YarnRuntimeException("Cannot call stop from callback handler thread!");
     }
     keepRunning = false;
     try {
@@ -184,12 +184,12 @@ public class AMRMClientAsync<T extends ContainerRequest> extends AbstractService
   /**
    * Registers this application master with the resource manager. On successful
    * registration, starts the heartbeating thread.
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public RegisterApplicationMasterResponse registerApplicationMaster(
       String appHostName, int appHostPort, String appTrackingUrl)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     RegisterApplicationMasterResponse response = client
         .registerApplicationMaster(appHostName, appHostPort, appTrackingUrl);
     heartbeatThread.start();
@@ -201,11 +201,11 @@ public class AMRMClientAsync<T extends ContainerRequest> extends AbstractService
    * @param appStatus Success/Failure status of the master
    * @param appMessage Diagnostics message on failure
    * @param appTrackingUrl New URL to get master info
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   public void unregisterApplicationMaster(FinalApplicationStatus appStatus,
-      String appMessage, String appTrackingUrl) throws YarnRemoteException,
+      String appMessage, String appTrackingUrl) throws YarnException,
       IOException {
     synchronized (unregisterHeartbeatLock) {
       keepRunning = false;
@@ -277,7 +277,7 @@ public class AMRMClientAsync<T extends ContainerRequest> extends AbstractService
             
           try {
             response = client.allocate(progress);
-          } catch (YarnRemoteException ex) {
+          } catch (YarnException ex) {
             LOG.error("Yarn exception on heartbeat", ex);
             savedException = ex;
             // interrupt handler thread in case it waiting on the queue

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java

@@ -40,7 +40,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
@@ -55,7 +55,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -168,7 +168,7 @@ public class AMRMClientImpl<T extends ContainerRequest>
     try {
       currentUser = UserGroupInformation.getCurrentUser();
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
 
     // CurrentUser should already have AMToken loaded.
@@ -194,7 +194,7 @@ public class AMRMClientImpl<T extends ContainerRequest>
   @Override
   public RegisterApplicationMasterResponse registerApplicationMaster(
       String appHostName, int appHostPort, String appTrackingUrl)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     // do this only once ???
     RegisterApplicationMasterRequest request = recordFactory
         .newRecordInstance(RegisterApplicationMasterRequest.class);
@@ -213,7 +213,7 @@ public class AMRMClientImpl<T extends ContainerRequest>
 
   @Override
   public AllocateResponse allocate(float progressIndicator) 
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     AllocateResponse allocateResponse = null;
     ArrayList<ResourceRequest> askList = null;
     ArrayList<ContainerId> releaseList = null;
@@ -267,7 +267,7 @@ public class AMRMClientImpl<T extends ContainerRequest>
 
   @Override
   public void unregisterApplicationMaster(FinalApplicationStatus appStatus,
-      String appMessage, String appTrackingUrl) throws YarnRemoteException,
+      String appMessage, String appTrackingUrl) throws YarnException,
       IOException {
     FinishApplicationMasterRequest request = recordFactory
                   .newRecordInstance(FinishApplicationMasterRequest.class);

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClient.java

@@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Token;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.service.Service;
 
 @InterfaceAudience.Public
@@ -51,12 +51,12 @@ public interface NMClient extends Service {
    *                               <code>NodeManager</code> to launch the
    *                               container
    * @return a map between the auxiliary service names and their outputs
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   Map<String, ByteBuffer> startContainer(Container container,
       ContainerLaunchContext containerLaunchContext)
-          throws YarnRemoteException, IOException;
+          throws YarnException, IOException;
 
   /**
    * <p>Stop an started container.</p>
@@ -65,11 +65,11 @@ public interface NMClient extends Service {
    * @param nodeId the Id of the <code>NodeManager</code>
    * @param containerToken the security token to verify authenticity of the
    *                       started container
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   void stopContainer(ContainerId containerId, NodeId nodeId,
-      Token containerToken) throws YarnRemoteException, IOException;
+      Token containerToken) throws YarnException, IOException;
 
   /**
    * <p>Query the status of a container.</p>
@@ -79,11 +79,11 @@ public interface NMClient extends Service {
    * @param containerToken the security token to verify authenticity of the
    *                       started container
    * @return the status of a container
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   ContainerStatus getContainerStatus(ContainerId containerId, NodeId nodeId,
-      Token containerToken) throws YarnRemoteException, IOException;
+      Token containerToken) throws YarnException, IOException;
 
   /**
    * <p>Set whether the containers that are started by this client, and are

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientAsync.java

@@ -51,7 +51,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AbstractEvent;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
@@ -446,7 +446,7 @@ public class NMClientAsync extends AbstractService {
                 + "Container " + containerId, thr);
           }
           return ContainerState.RUNNING;
-        } catch (YarnRemoteException e) {
+        } catch (YarnException e) {
           return onExceptionRaised(container, event, e);
         } catch (IOException e) {
           return onExceptionRaised(container, event, e);
@@ -490,7 +490,7 @@ public class NMClientAsync extends AbstractService {
                 + "Container " + event.getContainerId(), thr);
           }
           return ContainerState.DONE;
-        } catch (YarnRemoteException e) {
+        } catch (YarnException e) {
           return onExceptionRaised(container, event, e);
         } catch (IOException e) {
           return onExceptionRaised(container, event, e);
@@ -602,7 +602,7 @@ public class NMClientAsync extends AbstractService {
                 "Unchecked exception is thrown from onContainerStatusReceived" +
                     " for Container " + event.getContainerId(), thr);
           }
-        } catch (YarnRemoteException e) {
+        } catch (YarnException e) {
           onExceptionRaised(containerId, e);
         } catch (IOException e) {
           onExceptionRaised(containerId, e);

+ 13 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java

@@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
@@ -112,7 +112,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
         stopContainer(startedContainer.getContainerId(),
             startedContainer.getNodeId(),
             startedContainer.getContainerToken());
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         LOG.error("Failed to stop Container " +
             startedContainer.getContainerId() +
             "when stopping NMClientImpl");
@@ -213,7 +213,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
 
     public synchronized Map<String, ByteBuffer> startContainer(
         Container container, ContainerLaunchContext containerLaunchContext)
-            throws YarnRemoteException, IOException {
+            throws YarnException, IOException {
       if (!container.getId().equals(containerId)) {
         throw new IllegalArgumentException(
             "NMCommunicator's containerId  mismatches the given Container's");
@@ -228,7 +228,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Started Container " + containerId);
         }
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         LOG.warn("Container " + containerId + " failed to start", e);
         throw e;
       } catch (IOException e) {
@@ -238,7 +238,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
       return startResponse.getAllServiceResponse();
     }
 
-    public synchronized void stopContainer() throws YarnRemoteException,
+    public synchronized void stopContainer() throws YarnException,
         IOException {
       try {
         StopContainerRequest stopRequest =
@@ -248,7 +248,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Stopped Container " + containerId);
         }
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         LOG.warn("Container " + containerId + " failed to stop", e);
         throw e;
       } catch (IOException e) {
@@ -258,7 +258,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
     }
 
     public synchronized ContainerStatus getContainerStatus()
-        throws YarnRemoteException, IOException {
+        throws YarnException, IOException {
       GetContainerStatusResponse statusResponse = null;
       try {
         GetContainerStatusRequest statusRequest =
@@ -268,7 +268,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Got the status of Container " + containerId);
         }
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         LOG.warn(
             "Unable to get the status of Container " + containerId, e);
         throw e;
@@ -284,7 +284,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
   @Override
   public Map<String, ByteBuffer> startContainer(
       Container container, ContainerLaunchContext containerLaunchContext)
-          throws YarnRemoteException, IOException {
+          throws YarnException, IOException {
     // Do synchronization on StartedContainer to prevent race condition
     // between startContainer and stopContainer
     synchronized (addStartedContainer(container)) {
@@ -297,7 +297,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
         nmCommunicator.start();
         allServiceResponse =
             nmCommunicator.startContainer(container, containerLaunchContext);
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         // Remove the started container if it failed to start
         removeStartedContainer(container.getId());
         throw e;
@@ -326,7 +326,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
 
   @Override
   public void stopContainer(ContainerId containerId, NodeId nodeId,
-      Token containerToken) throws YarnRemoteException, IOException {
+      Token containerToken) throws YarnException, IOException {
     StartedContainer startedContainer = getStartedContainer(containerId);
     if (startedContainer == null) {
       throw RPCUtil.getRemoteException("Container " + containerId +
@@ -359,7 +359,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
   @Override
   public ContainerStatus getContainerStatus(ContainerId containerId,
       NodeId nodeId, Token containerToken)
-          throws YarnRemoteException, IOException {
+          throws YarnException, IOException {
     NMCommunicator nmCommunicator = null;
     try {
       nmCommunicator = new NMCommunicator(containerId, nodeId, containerToken);
@@ -375,7 +375,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
   }
 
   protected synchronized StartedContainer addStartedContainer(
-      Container container) throws YarnRemoteException, IOException {
+      Container container) throws YarnException, IOException {
     if (startedContainers.containsKey(container.getId())) {
       throw RPCUtil.getRemoteException("Container " + container.getId() +
           " is already started");

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/RMAdmin.java

@@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RefreshServiceAclsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -188,7 +188,7 @@ public class RMAdmin extends Configured implements Tool {
     return adminProtocol;
   }
   
-  private int refreshQueues() throws IOException, YarnRemoteException {
+  private int refreshQueues() throws IOException, YarnException {
     // Refresh the queue properties
     RMAdminProtocol adminProtocol = createAdminProtocol();
     RefreshQueuesRequest request = 
@@ -197,7 +197,7 @@ public class RMAdmin extends Configured implements Tool {
     return 0;
   }
 
-  private int refreshNodes() throws IOException, YarnRemoteException {
+  private int refreshNodes() throws IOException, YarnException {
     // Refresh the nodes
     RMAdminProtocol adminProtocol = createAdminProtocol();
     RefreshNodesRequest request = 
@@ -207,7 +207,7 @@ public class RMAdmin extends Configured implements Tool {
   }
   
   private int refreshUserToGroupsMappings() throws IOException,
-      YarnRemoteException {
+      YarnException {
     // Refresh the user-to-groups mappings
     RMAdminProtocol adminProtocol = createAdminProtocol();
     RefreshUserToGroupsMappingsRequest request = 
@@ -217,7 +217,7 @@ public class RMAdmin extends Configured implements Tool {
   }
   
   private int refreshSuperUserGroupsConfiguration() throws IOException,
-      YarnRemoteException {
+      YarnException {
     // Refresh the super-user groups
     RMAdminProtocol adminProtocol = createAdminProtocol();
     RefreshSuperUserGroupsConfigurationRequest request = 
@@ -226,7 +226,7 @@ public class RMAdmin extends Configured implements Tool {
     return 0;
   }
   
-  private int refreshAdminAcls() throws IOException, YarnRemoteException {
+  private int refreshAdminAcls() throws IOException, YarnException {
     // Refresh the admin acls
     RMAdminProtocol adminProtocol = createAdminProtocol();
     RefreshAdminAclsRequest request = 
@@ -235,7 +235,7 @@ public class RMAdmin extends Configured implements Tool {
     return 0;
   }
   
-  private int refreshServiceAcls() throws IOException, YarnRemoteException {
+  private int refreshServiceAcls() throws IOException, YarnException {
     // Refresh the service acls
     RMAdminProtocol adminProtocol = createAdminProtocol();
     RefreshServiceAclsRequest request = 

+ 27 - 27
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClient.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.service.Service;
 
 @InterfaceAudience.Public
@@ -58,10 +58,10 @@ public interface YarnClient extends Service {
    * 
    * @return response containing the new <code>ApplicationId</code> to be used
    *         to submit an application
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
-  GetNewApplicationResponse getNewApplication() throws YarnRemoteException,
+  GetNewApplicationResponse getNewApplication() throws YarnException,
       IOException;
 
   /**
@@ -75,12 +75,12 @@ public interface YarnClient extends Service {
    *          {@link ApplicationSubmissionContext} containing all the details
    *          needed to submit a new application
    * @return {@link ApplicationId} of the accepted application
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    * @see #getNewApplication()
    */
   ApplicationId submitApplication(ApplicationSubmissionContext appContext)
-      throws YarnRemoteException, IOException;
+      throws YarnException, IOException;
 
   /**
    * <p>
@@ -89,13 +89,13 @@ public interface YarnClient extends Service {
    * 
    * @param applicationId
    *          {@link ApplicationId} of the application that needs to be killed
-   * @throws YarnRemoteException
+   * @throws YarnException
    *           in case of errors or if YARN rejects the request due to
    *           access-control restrictions.
    * @throws IOException
    * @see #getQueueAclsInfo()
    */
-  void killApplication(ApplicationId applicationId) throws YarnRemoteException,
+  void killApplication(ApplicationId applicationId) throws YarnException,
       IOException;
 
   /**
@@ -125,11 +125,11 @@ public interface YarnClient extends Service {
    * @param appId
    *          {@link ApplicationId} of the application that needs a report
    * @return application report
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   ApplicationReport getApplicationReport(ApplicationId appId)
-      throws YarnRemoteException, IOException;
+      throws YarnException, IOException;
 
   /**
    * <p>
@@ -143,10 +143,10 @@ public interface YarnClient extends Service {
    * </p>
    * 
    * @return a list of reports of all running applications
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
-  List<ApplicationReport> getApplicationList() throws YarnRemoteException,
+  List<ApplicationReport> getApplicationList() throws YarnException,
       IOException;
 
   /**
@@ -155,10 +155,10 @@ public interface YarnClient extends Service {
    * </p>
    * 
    * @return cluster metrics
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
-  YarnClusterMetrics getYarnClusterMetrics() throws YarnRemoteException,
+  YarnClusterMetrics getYarnClusterMetrics() throws YarnException,
       IOException;
 
   /**
@@ -167,10 +167,10 @@ public interface YarnClient extends Service {
    * </p>
    * 
    * @return A list of report of all nodes
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
-  List<NodeReport> getNodeReports() throws YarnRemoteException, IOException;
+  List<NodeReport> getNodeReports() throws YarnException, IOException;
 
   /**
    * <p>
@@ -181,11 +181,11 @@ public interface YarnClient extends Service {
    *          securely talking to YARN.
    * @return a delegation token ({@link Token}) that can be used to
    *         talk to YARN
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   Token getRMDelegationToken(Text renewer)
-      throws YarnRemoteException, IOException;
+      throws YarnException, IOException;
 
   /**
    * <p>
@@ -195,12 +195,12 @@ public interface YarnClient extends Service {
    * @param queueName
    *          Name of the queue whose information is needed
    * @return queue information
-   * @throws YarnRemoteException
+   * @throws YarnException
    *           in case of errors or if YARN rejects the request due to
    *           access-control restrictions.
    * @throws IOException
    */
-  QueueInfo getQueueInfo(String queueName) throws YarnRemoteException,
+  QueueInfo getQueueInfo(String queueName) throws YarnException,
       IOException;
 
   /**
@@ -210,10 +210,10 @@ public interface YarnClient extends Service {
    * </p>
    * 
    * @return a list of queue-information for all queues
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
-  List<QueueInfo> getAllQueues() throws YarnRemoteException, IOException;
+  List<QueueInfo> getAllQueues() throws YarnException, IOException;
 
   /**
    * <p>
@@ -221,10 +221,10 @@ public interface YarnClient extends Service {
    * </p>
    * 
    * @return a list of queue-information for all the top-level queues
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
-  List<QueueInfo> getRootQueueInfos() throws YarnRemoteException, IOException;
+  List<QueueInfo> getRootQueueInfos() throws YarnException, IOException;
 
   /**
    * <p>
@@ -236,10 +236,10 @@ public interface YarnClient extends Service {
    *          Name of the queue whose child-queues' information is needed
    * @return a list of queue-information for all queues who are direct children
    *         of the given parent queue.
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
-  List<QueueInfo> getChildQueueInfos(String parent) throws YarnRemoteException,
+  List<QueueInfo> getChildQueueInfos(String parent) throws YarnException,
       IOException;
 
   /**
@@ -250,9 +250,9 @@ public interface YarnClient extends Service {
    * 
    * @return a list of queue acls ({@link QueueUserACLInfo}) for
    *         <em>current user</em>
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
-  List<QueueUserACLInfo> getQueueAclsInfo() throws YarnRemoteException,
+  List<QueueUserACLInfo> getQueueAclsInfo() throws YarnException,
       IOException;
 }

+ 14 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java

@@ -57,7 +57,7 @@ import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.util.Records;
@@ -121,7 +121,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
 
   @Override
   public GetNewApplicationResponse getNewApplication()
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     GetNewApplicationRequest request =
         Records.newRecord(GetNewApplicationRequest.class);
     return rmClient.getNewApplication(request);
@@ -130,7 +130,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
   @Override
   public ApplicationId
       submitApplication(ApplicationSubmissionContext appContext)
-          throws YarnRemoteException, IOException {
+          throws YarnException, IOException {
     ApplicationId applicationId = appContext.getApplicationId();
     appContext.setApplicationId(applicationId);
     SubmitApplicationRequest request =
@@ -167,7 +167,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
 
   @Override
   public void killApplication(ApplicationId applicationId)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     LOG.info("Killing application " + applicationId);
     KillApplicationRequest request =
         Records.newRecord(KillApplicationRequest.class);
@@ -177,7 +177,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
 
   @Override
   public ApplicationReport getApplicationReport(ApplicationId appId)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     GetApplicationReportRequest request =
         Records.newRecord(GetApplicationReportRequest.class);
     request.setApplicationId(appId);
@@ -188,7 +188,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
 
   @Override
   public List<ApplicationReport> getApplicationList()
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     GetAllApplicationsRequest request =
         Records.newRecord(GetAllApplicationsRequest.class);
     GetAllApplicationsResponse response = rmClient.getAllApplications(request);
@@ -196,7 +196,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
   }
 
   @Override
-  public YarnClusterMetrics getYarnClusterMetrics() throws YarnRemoteException,
+  public YarnClusterMetrics getYarnClusterMetrics() throws YarnException,
       IOException {
     GetClusterMetricsRequest request =
         Records.newRecord(GetClusterMetricsRequest.class);
@@ -205,7 +205,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
   }
 
   @Override
-  public List<NodeReport> getNodeReports() throws YarnRemoteException,
+  public List<NodeReport> getNodeReports() throws YarnException,
       IOException {
     GetClusterNodesRequest request =
         Records.newRecord(GetClusterNodesRequest.class);
@@ -215,7 +215,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
 
   @Override
   public Token getRMDelegationToken(Text renewer)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     /* get the token from RM */
     GetDelegationTokenRequest rmDTRequest =
         Records.newRecord(GetDelegationTokenRequest.class);
@@ -238,7 +238,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
   }
 
   @Override
-  public QueueInfo getQueueInfo(String queueName) throws YarnRemoteException,
+  public QueueInfo getQueueInfo(String queueName) throws YarnException,
       IOException {
     GetQueueInfoRequest request =
         getQueueInfoRequest(queueName, true, false, false);
@@ -247,7 +247,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
   }
 
   @Override
-  public List<QueueUserACLInfo> getQueueAclsInfo() throws YarnRemoteException,
+  public List<QueueUserACLInfo> getQueueAclsInfo() throws YarnException,
       IOException {
     GetQueueUserAclsInfoRequest request =
         Records.newRecord(GetQueueUserAclsInfoRequest.class);
@@ -255,7 +255,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
   }
 
   @Override
-  public List<QueueInfo> getAllQueues() throws YarnRemoteException,
+  public List<QueueInfo> getAllQueues() throws YarnException,
       IOException {
     List<QueueInfo> queues = new ArrayList<QueueInfo>();
 
@@ -267,7 +267,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
   }
 
   @Override
-  public List<QueueInfo> getRootQueueInfos() throws YarnRemoteException,
+  public List<QueueInfo> getRootQueueInfos() throws YarnException,
       IOException {
     List<QueueInfo> queues = new ArrayList<QueueInfo>();
 
@@ -280,7 +280,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
 
   @Override
   public List<QueueInfo> getChildQueueInfos(String parent)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     List<QueueInfo> queues = new ArrayList<QueueInfo>();
 
     QueueInfo parentQueue =

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

@@ -30,7 +30,7 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 public class ApplicationCLI extends YarnCLI {
@@ -90,10 +90,10 @@ public class ApplicationCLI extends YarnCLI {
   /**
    * Lists all the applications present in the Resource Manager
    * 
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
-  private void listAllApplications() throws YarnRemoteException, IOException {
+  private void listAllApplications() throws YarnException, IOException {
     PrintWriter writer = new PrintWriter(sysout);
     List<ApplicationReport> appsReport = client.getApplicationList();
 
@@ -117,11 +117,11 @@ public class ApplicationCLI extends YarnCLI {
    * Kills the application with the application id as appId
    * 
    * @param applicationId
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
   private void killApplication(String applicationId)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     ApplicationId appId = ConverterUtils.toApplicationId(applicationId);
     sysout.println("Killing application " + applicationId);
     client.killApplication(appId);
@@ -131,10 +131,10 @@ public class ApplicationCLI extends YarnCLI {
    * Prints the application report for an application id.
    * 
    * @param applicationId
-   * @throws YarnRemoteException
+   * @throws YarnException
    */
   private void printApplicationReport(String applicationId)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     ApplicationReport appReport = client.getApplicationReport(ConverterUtils
         .toApplicationId(applicationId));
     // Use PrintWriter.println, which uses correct platform line ending.

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java

@@ -31,7 +31,7 @@ import org.apache.commons.lang.time.DateFormatUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 public class NodeCLI extends YarnCLI {
@@ -83,10 +83,10 @@ public class NodeCLI extends YarnCLI {
   /**
    * Lists all the nodes present in the cluster
    * 
-   * @throws YarnRemoteException
+   * @throws YarnException
    * @throws IOException
    */
-  private void listClusterNodes() throws YarnRemoteException, IOException {
+  private void listClusterNodes() throws YarnException, IOException {
     PrintWriter writer = new PrintWriter(sysout);
     List<NodeReport> nodesReport = client.getNodeReports();
     writer.println("Total Nodes:" + nodesReport.size());
@@ -105,9 +105,9 @@ public class NodeCLI extends YarnCLI {
    * Prints the node report for node id.
    * 
    * @param nodeIdStr
-   * @throws YarnRemoteException
+   * @throws YarnException
    */
-  private void printNodeStatus(String nodeIdStr) throws YarnRemoteException,
+  private void printNodeStatus(String nodeIdStr) throws YarnException,
       IOException {
     NodeId nodeId = ConverterUtils.toNodeId(nodeIdStr);
     List<NodeReport> nodesReport = client.getNodeReports();

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java

@@ -53,7 +53,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.client.AMRMClient.StoredContainerRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.service.Service.STATE;
 import org.apache.hadoop.yarn.util.Records;
@@ -162,7 +162,7 @@ public class TestAMRMClient {
   }
   
   @Test (timeout=60000)
-  public void testAMRMClientMatchingFit() throws YarnRemoteException, IOException {
+  public void testAMRMClientMatchingFit() throws YarnException, IOException {
     AMRMClientImpl<StoredContainerRequest> amClient = null;
     try {
       // start am rm client
@@ -263,7 +263,7 @@ public class TestAMRMClient {
   }
 
   @Test (timeout=60000)
-  public void testAMRMClientMatchStorage() throws YarnRemoteException, IOException {
+  public void testAMRMClientMatchStorage() throws YarnException, IOException {
     AMRMClientImpl<StoredContainerRequest> amClient = null;
     try {
       // start am rm client
@@ -384,7 +384,7 @@ public class TestAMRMClient {
   }
 
   @Test (timeout=60000)
-  public void testAMRMClient() throws YarnRemoteException, IOException {
+  public void testAMRMClient() throws YarnException, IOException {
     AMRMClientImpl<ContainerRequest> amClient = null;
     try {
       // start am rm client
@@ -407,7 +407,7 @@ public class TestAMRMClient {
   }
     
   private void testAllocation(final AMRMClientImpl<ContainerRequest> amClient)  
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     // setup container request
     
     assertTrue(amClient.ask.size() == 0);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java

@@ -45,7 +45,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -153,7 +153,7 @@ public class TestAMRMClientAsync {
     @SuppressWarnings("unchecked")
     AMRMClient<ContainerRequest> client = mock(AMRMClientImpl.class);
     String exStr = "TestException";
-    YarnRemoteException mockException = mock(YarnRemoteException.class);
+    YarnException mockException = mock(YarnException.class);
     when(mockException.getMessage()).thenReturn(exStr);
     when(client.allocate(anyFloat())).thenThrow(mockException);
 

+ 10 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java

@@ -52,7 +52,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.service.Service.STATE;
 import org.apache.hadoop.yarn.util.Records;
@@ -71,7 +71,7 @@ public class TestNMClient {
   int nodeCount = 3;
 
   @Before
-  public void setup() throws YarnRemoteException, IOException {
+  public void setup() throws YarnException, IOException {
     // start minicluster
     conf = new YarnConfiguration();
     yarnCluster =
@@ -175,7 +175,7 @@ public class TestNMClient {
 
   @Test (timeout = 60000)
   public void testNMClient()
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
 
     rmClient.registerApplicationMaster("Host", 10000, "");
 
@@ -187,7 +187,7 @@ public class TestNMClient {
 
   private Set<Container> allocateContainers(
       AMRMClientImpl<ContainerRequest> rmClient, int num)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     // setup container request
     Resource capability = Resource.newInstance(1024, 0);
     Priority priority = Priority.newInstance(0);
@@ -228,7 +228,7 @@ public class TestNMClient {
   }
 
   private void testContainerManagement(NMClientImpl nmClient,
-      Set<Container> containers) throws YarnRemoteException, IOException {
+      Set<Container> containers) throws YarnException, IOException {
     int size = containers.size();
     int i = 0;
     for (Container container : containers) {
@@ -238,7 +238,7 @@ public class TestNMClient {
         nmClient.getContainerStatus(container.getId(), container.getNodeId(),
             container.getContainerToken());
         fail("Exception is expected");
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         assertTrue("The thrown exception is not expected",
             e.getMessage().contains("is not handled by this NodeManager"));
       }
@@ -249,7 +249,7 @@ public class TestNMClient {
         nmClient.stopContainer(container.getId(), container.getNodeId(),
             container.getContainerToken());
         fail("Exception is expected");
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         assertTrue("The thrown exception is not expected",
             e.getMessage().contains(
                 "is either not started yet or already stopped"));
@@ -265,7 +265,7 @@ public class TestNMClient {
       clc.setTokens(securityTokens);
       try {
         nmClient.startContainer(container, clc);
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         fail("Exception is not expected");
       }
 
@@ -278,7 +278,7 @@ public class TestNMClient {
         try {
           nmClient.stopContainer(container.getId(), container.getNodeId(),
               container.getContainerToken());
-        } catch (YarnRemoteException e) {
+        } catch (YarnException e) {
           fail("Exception is not expected");
         }
 
@@ -299,7 +299,7 @@ public class TestNMClient {
 
   private void testGetContainerStatus(Container container, int index,
       ContainerState state, String diagnostics, int exitStatus)
-          throws YarnRemoteException, IOException {
+          throws YarnException, IOException {
     while (true) {
       try {
         ContainerStatus status = nmClient.getContainerStatus(

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java

@@ -48,7 +48,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
@@ -136,7 +136,7 @@ public class TestNMClientAsync {
         Collections.synchronizedSet(new HashSet<String>());
 
     protected MockNMClientAsync1(int expectedSuccess, int expectedFailure)
-        throws YarnRemoteException, IOException {
+        throws YarnException, IOException {
       super(MockNMClientAsync1.class.getName(), mockNMClient(0),
           new TestCallbackHandler1(expectedSuccess, expectedFailure));
     }
@@ -361,7 +361,7 @@ public class TestNMClientAsync {
   }
 
   private NMClient mockNMClient(int mode)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     NMClient client = mock(NMClient.class);
     switch (mode) {
       case 0:
@@ -436,7 +436,7 @@ public class TestNMClientAsync {
     private CyclicBarrier barrierB;
 
     protected MockNMClientAsync2(CyclicBarrier barrierA, CyclicBarrier barrierB,
-        CyclicBarrier barrierC) throws YarnRemoteException, IOException {
+        CyclicBarrier barrierC) throws YarnException, IOException {
       super(MockNMClientAsync2.class.getName(), mockNMClient(0),
           new TestCallbackHandler2(barrierC));
       this.barrierA = barrierA;

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java

@@ -39,7 +39,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -95,7 +95,7 @@ public class TestYarnClient {
       ((MockYarnClient) client).setYarnApplicationState(exitStates[i]);
       try {
         client.submitApplication(context);
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         Assert.fail("Exception is not expected.");
       } catch (IOException e) {
         Assert.fail("Exception is not expected.");
@@ -153,7 +153,7 @@ public class TestYarnClient {
       try{
         when(rmClient.getApplicationReport(any(
             GetApplicationReportRequest.class))).thenReturn(mockResponse);
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         Assert.fail("Exception is not expected.");
       } catch (IOException e) {
         Assert.fail("Exception is not expected.");

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/AMRMProtocolPBClientImpl.java

@@ -39,7 +39,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMaste
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMasterResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterRequestProto;
@@ -68,7 +68,7 @@ public class AMRMProtocolPBClientImpl implements AMRMProtocol, Closeable {
 
   @Override
   public AllocateResponse allocate(AllocateRequest request)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     AllocateRequestProto requestProto =
         ((AllocateRequestPBImpl) request).getProto();
     try {
@@ -81,7 +81,7 @@ public class AMRMProtocolPBClientImpl implements AMRMProtocol, Closeable {
 
   @Override
   public FinishApplicationMasterResponse finishApplicationMaster(
-      FinishApplicationMasterRequest request) throws YarnRemoteException,
+      FinishApplicationMasterRequest request) throws YarnException,
       IOException {
     FinishApplicationMasterRequestProto requestProto =
         ((FinishApplicationMasterRequestPBImpl) request).getProto();
@@ -96,7 +96,7 @@ public class AMRMProtocolPBClientImpl implements AMRMProtocol, Closeable {
 
   @Override
   public RegisterApplicationMasterResponse registerApplicationMaster(
-      RegisterApplicationMasterRequest request) throws YarnRemoteException,
+      RegisterApplicationMasterRequest request) throws YarnException,
       IOException {
     RegisterApplicationMasterRequestProto requestProto =
         ((RegisterApplicationMasterRequestPBImpl) request).getProto();

+ 13 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java

@@ -78,7 +78,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRe
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllApplicationsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationReportRequestProto;
@@ -113,7 +113,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
 
   @Override
   public KillApplicationResponse forceKillApplication(
-      KillApplicationRequest request) throws YarnRemoteException, IOException {
+      KillApplicationRequest request) throws YarnException, IOException {
     KillApplicationRequestProto requestProto =
         ((KillApplicationRequestPBImpl) request).getProto();
     try {
@@ -127,7 +127,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
 
   @Override
   public GetApplicationReportResponse getApplicationReport(
-      GetApplicationReportRequest request) throws YarnRemoteException,
+      GetApplicationReportRequest request) throws YarnException,
       IOException {
     GetApplicationReportRequestProto requestProto =
         ((GetApplicationReportRequestPBImpl) request).getProto();
@@ -142,7 +142,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
 
   @Override
   public GetClusterMetricsResponse getClusterMetrics(
-      GetClusterMetricsRequest request) throws YarnRemoteException,
+      GetClusterMetricsRequest request) throws YarnException,
       IOException {
     GetClusterMetricsRequestProto requestProto =
         ((GetClusterMetricsRequestPBImpl) request).getProto();
@@ -157,7 +157,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
 
   @Override
   public GetNewApplicationResponse getNewApplication(
-      GetNewApplicationRequest request) throws YarnRemoteException,
+      GetNewApplicationRequest request) throws YarnException,
       IOException {
     GetNewApplicationRequestProto requestProto =
         ((GetNewApplicationRequestPBImpl) request).getProto();
@@ -172,7 +172,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
 
   @Override
   public SubmitApplicationResponse submitApplication(
-      SubmitApplicationRequest request) throws YarnRemoteException,
+      SubmitApplicationRequest request) throws YarnException,
       IOException {
     SubmitApplicationRequestProto requestProto =
         ((SubmitApplicationRequestPBImpl) request).getProto();
@@ -187,7 +187,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
 
   @Override
   public GetAllApplicationsResponse getAllApplications(
-      GetAllApplicationsRequest request) throws YarnRemoteException,
+      GetAllApplicationsRequest request) throws YarnException,
       IOException {
     GetAllApplicationsRequestProto requestProto =
         ((GetAllApplicationsRequestPBImpl) request).getProto();
@@ -203,7 +203,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
   @Override
   public GetClusterNodesResponse
       getClusterNodes(GetClusterNodesRequest request)
-          throws YarnRemoteException, IOException {
+          throws YarnException, IOException {
     GetClusterNodesRequestProto requestProto =
         ((GetClusterNodesRequestPBImpl) request).getProto();
     try {
@@ -217,7 +217,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
 
   @Override
   public GetQueueInfoResponse getQueueInfo(GetQueueInfoRequest request)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     GetQueueInfoRequestProto requestProto =
         ((GetQueueInfoRequestPBImpl) request).getProto();
     try {
@@ -231,7 +231,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
 
   @Override
   public GetQueueUserAclsInfoResponse getQueueUserAcls(
-      GetQueueUserAclsInfoRequest request) throws YarnRemoteException,
+      GetQueueUserAclsInfoRequest request) throws YarnException,
       IOException {
     GetQueueUserAclsInfoRequestProto requestProto =
         ((GetQueueUserAclsInfoRequestPBImpl) request).getProto();
@@ -246,7 +246,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
 
   @Override
   public GetDelegationTokenResponse getDelegationToken(
-      GetDelegationTokenRequest request) throws YarnRemoteException,
+      GetDelegationTokenRequest request) throws YarnException,
       IOException {
     GetDelegationTokenRequestProto requestProto =
         ((GetDelegationTokenRequestPBImpl) request).getProto();
@@ -261,7 +261,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
 
   @Override
   public RenewDelegationTokenResponse renewDelegationToken(
-      RenewDelegationTokenRequest request) throws YarnRemoteException,
+      RenewDelegationTokenRequest request) throws YarnException,
       IOException {
     RenewDelegationTokenRequestProto requestProto = 
         ((RenewDelegationTokenRequestPBImpl) request).getProto();
@@ -276,7 +276,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
 
   @Override
   public CancelDelegationTokenResponse cancelDelegationToken(
-      CancelDelegationTokenRequest request) throws YarnRemoteException,
+      CancelDelegationTokenRequest request) throws YarnException,
       IOException {
     CancelDelegationTokenRequestProto requestProto =
         ((CancelDelegationTokenRequestPBImpl) request).getProto();

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java

@@ -42,7 +42,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerResponse
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerResponsePBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
@@ -86,7 +86,7 @@ public class ContainerManagerPBClientImpl implements ContainerManager,
 
   @Override
   public GetContainerStatusResponse getContainerStatus(
-      GetContainerStatusRequest request) throws YarnRemoteException,
+      GetContainerStatusRequest request) throws YarnException,
       IOException {
     GetContainerStatusRequestProto requestProto =
         ((GetContainerStatusRequestPBImpl) request).getProto();
@@ -101,7 +101,7 @@ public class ContainerManagerPBClientImpl implements ContainerManager,
 
   @Override
   public StartContainerResponse startContainer(StartContainerRequest request)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     StartContainerRequestProto requestProto =
         ((StartContainerRequestPBImpl) request).getProto();
     try {
@@ -115,7 +115,7 @@ public class ContainerManagerPBClientImpl implements ContainerManager,
 
   @Override
   public StopContainerResponse stopContainer(StopContainerRequest request)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     StopContainerRequestProto requestProto =
         ((StopContainerRequestPBImpl) request).getProto();
     try {

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java

@@ -52,7 +52,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshSuperUserGroups
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshSuperUserGroupsConfigurationResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsResponsePBImpl;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserResponseProto;
@@ -87,7 +87,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
 
   @Override
   public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     RefreshQueuesRequestProto requestProto = 
       ((RefreshQueuesRequestPBImpl)request).getProto();
     try {
@@ -101,7 +101,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
 
   @Override
   public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
-  throws YarnRemoteException, IOException {
+  throws YarnException, IOException {
     RefreshNodesRequestProto requestProto = 
       ((RefreshNodesRequestPBImpl)request).getProto();
     try {
@@ -116,7 +116,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
   @Override
   public RefreshSuperUserGroupsConfigurationResponse refreshSuperUserGroupsConfiguration(
       RefreshSuperUserGroupsConfigurationRequest request)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     RefreshSuperUserGroupsConfigurationRequestProto requestProto = 
       ((RefreshSuperUserGroupsConfigurationRequestPBImpl)request).getProto();
     try {
@@ -130,7 +130,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
 
   @Override
   public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
-      RefreshUserToGroupsMappingsRequest request) throws YarnRemoteException,
+      RefreshUserToGroupsMappingsRequest request) throws YarnException,
       IOException {
     RefreshUserToGroupsMappingsRequestProto requestProto = 
       ((RefreshUserToGroupsMappingsRequestPBImpl)request).getProto();
@@ -145,7 +145,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
 
   @Override
   public RefreshAdminAclsResponse refreshAdminAcls(
-      RefreshAdminAclsRequest request) throws YarnRemoteException, IOException {
+      RefreshAdminAclsRequest request) throws YarnException, IOException {
     RefreshAdminAclsRequestProto requestProto = 
       ((RefreshAdminAclsRequestPBImpl)request).getProto();
     try {
@@ -159,7 +159,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
 
   @Override
   public RefreshServiceAclsResponse refreshServiceAcls(
-      RefreshServiceAclsRequest request) throws YarnRemoteException,
+      RefreshServiceAclsRequest request) throws YarnException,
       IOException {
     RefreshServiceAclsRequestProto requestProto = 
         ((RefreshServiceAclsRequestPBImpl)request).getProto();

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/AMRMProtocolPBServiceImpl.java

@@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMaste
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMasterResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterRequestProto;
@@ -57,7 +57,7 @@ public class AMRMProtocolPBServiceImpl implements AMRMProtocolPB {
     try {
       AllocateResponse response = real.allocate(request);
       return ((AllocateResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -72,7 +72,7 @@ public class AMRMProtocolPBServiceImpl implements AMRMProtocolPB {
     try {
       FinishApplicationMasterResponse response = real.finishApplicationMaster(request);
       return ((FinishApplicationMasterResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -87,7 +87,7 @@ public class AMRMProtocolPBServiceImpl implements AMRMProtocolPB {
     try {
       RegisterApplicationMasterResponse response = real.registerApplicationMaster(request);
       return ((RegisterApplicationMasterResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);

+ 13 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java

@@ -64,7 +64,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRe
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllApplicationsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllApplicationsResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationReportRequestProto;
@@ -102,7 +102,7 @@ public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
     try {
       KillApplicationResponse response = real.forceKillApplication(request);
       return ((KillApplicationResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -117,7 +117,7 @@ public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
     try {
       GetApplicationReportResponse response = real.getApplicationReport(request);
       return ((GetApplicationReportResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -131,7 +131,7 @@ public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
     try {
       GetClusterMetricsResponse response = real.getClusterMetrics(request);
       return ((GetClusterMetricsResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -146,7 +146,7 @@ public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
     try {
       GetNewApplicationResponse response = real.getNewApplication(request);
       return ((GetNewApplicationResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -160,7 +160,7 @@ public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
     try {
       SubmitApplicationResponse response = real.submitApplication(request);
       return ((SubmitApplicationResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -176,7 +176,7 @@ public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
     try {
       GetAllApplicationsResponse response = real.getAllApplications(request);
       return ((GetAllApplicationsResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -191,7 +191,7 @@ public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
     try {
       GetClusterNodesResponse response = real.getClusterNodes(request);
       return ((GetClusterNodesResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -206,7 +206,7 @@ public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
     try {
       GetQueueInfoResponse response = real.getQueueInfo(request);
       return ((GetQueueInfoResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -222,7 +222,7 @@ public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
     try {
       GetQueueUserAclsInfoResponse response = real.getQueueUserAcls(request);
       return ((GetQueueUserAclsInfoResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -238,7 +238,7 @@ public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
       try {
         GetDelegationTokenResponse response = real.getDelegationToken(request);
         return ((GetDelegationTokenResponsePBImpl)response).getProto();
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         throw new ServiceException(e);
       } catch (IOException e) {
         throw new ServiceException(e);
@@ -254,7 +254,7 @@ public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
       try {
         RenewDelegationTokenResponse response = real.renewDelegationToken(request);
         return ((RenewDelegationTokenResponsePBImpl)response).getProto();
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         throw new ServiceException(e);
       } catch (IOException e) {
         throw new ServiceException(e);
@@ -270,7 +270,7 @@ public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
       try {
         CancelDelegationTokenResponse response = real.cancelDelegationToken(request);
         return ((CancelDelegationTokenResponsePBImpl)response).getProto();
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         throw new ServiceException(e);
       } catch (IOException e) {
         throw new ServiceException(e);

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagerPBServiceImpl.java

@@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerRequestP
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerResponsePBImpl;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
@@ -57,7 +57,7 @@ public class ContainerManagerPBServiceImpl implements ContainerManagerPB  {
     try {
       GetContainerStatusResponse response = real.getContainerStatus(request);
       return ((GetContainerStatusResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -71,7 +71,7 @@ public class ContainerManagerPBServiceImpl implements ContainerManagerPB  {
     try {
       StartContainerResponse response = real.startContainer(request);
       return ((StartContainerResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -85,7 +85,7 @@ public class ContainerManagerPBServiceImpl implements ContainerManagerPB  {
     try {
       StopContainerResponse response = real.stopContainer(request);
       return ((StopContainerResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java

@@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshSuperUserGroups
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshSuperUserGroupsConfigurationResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsResponsePBImpl;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.*;
 
 import com.google.protobuf.RpcController;
@@ -61,7 +61,7 @@ public class RMAdminProtocolPBServiceImpl implements RMAdminProtocolPB {
     try {
       RefreshQueuesResponse response = real.refreshQueues(request);
       return ((RefreshQueuesResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -77,7 +77,7 @@ public class RMAdminProtocolPBServiceImpl implements RMAdminProtocolPB {
     try {
       RefreshAdminAclsResponse response = real.refreshAdminAcls(request);
       return ((RefreshAdminAclsResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -91,7 +91,7 @@ public class RMAdminProtocolPBServiceImpl implements RMAdminProtocolPB {
     try {
       RefreshNodesResponse response = real.refreshNodes(request);
       return ((RefreshNodesResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -110,7 +110,7 @@ public class RMAdminProtocolPBServiceImpl implements RMAdminProtocolPB {
       RefreshSuperUserGroupsConfigurationResponse response = 
         real.refreshSuperUserGroupsConfiguration(request);
       return ((RefreshSuperUserGroupsConfigurationResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -127,7 +127,7 @@ public class RMAdminProtocolPBServiceImpl implements RMAdminProtocolPB {
       RefreshUserToGroupsMappingsResponse response = 
         real.refreshUserToGroupsMappings(request);
       return ((RefreshUserToGroupsMappingsResponsePBImpl)response).getProto();
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -144,7 +144,7 @@ public class RMAdminProtocolPBServiceImpl implements RMAdminProtocolPB {
         RefreshServiceAclsResponse response = 
           real.refreshServiceAcls(request);
         return ((RefreshServiceAclsResponsePBImpl)response).getProto();
-      } catch (YarnRemoteException e) {
+      } catch (YarnException e) {
         throw new ServiceException(e);
       } catch (IOException e) {
         throw new ServiceException(e);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java

@@ -29,7 +29,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.service.AbstractService;
 
 /**
@@ -190,7 +190,7 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
         if (!stopped) {
           LOG.warn("AsyncDispatcher thread interrupted", e);
         }
-        throw new YarnException(e);
+        throw new YarnRuntimeException(e);
       }
     };
   }

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RecordFactoryPBImpl.java

@@ -24,7 +24,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 
 public class RecordFactoryPBImpl implements RecordFactory {
@@ -53,7 +53,7 @@ public class RecordFactoryPBImpl implements RecordFactory {
       try {
         pbClazz = localConf.getClassByName(getPBImplClassName(clazz));
       } catch (ClassNotFoundException e) {
-        throw new YarnException("Failed to load class: ["
+        throw new YarnRuntimeException("Failed to load class: ["
             + getPBImplClassName(clazz) + "]", e);
       }
       try {
@@ -61,18 +61,18 @@ public class RecordFactoryPBImpl implements RecordFactory {
         constructor.setAccessible(true);
         cache.putIfAbsent(clazz, constructor);
       } catch (NoSuchMethodException e) {
-        throw new YarnException("Could not find 0 argument constructor", e);
+        throw new YarnRuntimeException("Could not find 0 argument constructor", e);
       }
     }
     try {
       Object retObject = constructor.newInstance();
       return (T)retObject;
     } catch (InvocationTargetException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (IllegalAccessException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (InstantiationException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 

+ 8 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java

@@ -28,7 +28,7 @@ import java.util.concurrent.ConcurrentMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RpcClientFactory;
 
 public class RpcClientFactoryPBImpl implements RpcClientFactory {
@@ -59,7 +59,7 @@ public class RpcClientFactoryPBImpl implements RpcClientFactory {
       try {
         pbClazz = localConf.getClassByName(getPBImplClassName(protocol));
       } catch (ClassNotFoundException e) {
-        throw new YarnException("Failed to load class: ["
+        throw new YarnRuntimeException("Failed to load class: ["
             + getPBImplClassName(protocol) + "]", e);
       }
       try {
@@ -67,18 +67,18 @@ public class RpcClientFactoryPBImpl implements RpcClientFactory {
         constructor.setAccessible(true);
         cache.putIfAbsent(protocol, constructor);
       } catch (NoSuchMethodException e) {
-        throw new YarnException("Could not find constructor with params: " + Long.TYPE + ", " + InetSocketAddress.class + ", " + Configuration.class, e);
+        throw new YarnRuntimeException("Could not find constructor with params: " + Long.TYPE + ", " + InetSocketAddress.class + ", " + Configuration.class, e);
       }
     }
     try {
       Object retObject = constructor.newInstance(clientVersion, addr, conf);
       return retObject;
     } catch (InvocationTargetException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (IllegalAccessException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (InstantiationException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 
@@ -88,11 +88,11 @@ public class RpcClientFactoryPBImpl implements RpcClientFactory {
       Method closeMethod = proxy.getClass().getMethod("close");
       closeMethod.invoke(proxy);
     } catch (InvocationTargetException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (Exception e) {
       LOG.error("Cannot call close method due to Exception. "
           + "Ignoring.", e);
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 

+ 11 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcServerFactoryPBImpl.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RpcServerFactory;
 
 import com.google.protobuf.BlockingService;
@@ -81,7 +81,7 @@ public class RpcServerFactoryPBImpl implements RpcServerFactory {
         pbServiceImplClazz = localConf
             .getClassByName(getPbServiceImplClassName(protocol));
       } catch (ClassNotFoundException e) {
-        throw new YarnException("Failed to load class: ["
+        throw new YarnRuntimeException("Failed to load class: ["
             + getPbServiceImplClassName(protocol) + "]", e);
       }
       try {
@@ -89,7 +89,7 @@ public class RpcServerFactoryPBImpl implements RpcServerFactory {
         constructor.setAccessible(true);
         serviceCache.putIfAbsent(protocol, constructor);
       } catch (NoSuchMethodException e) {
-        throw new YarnException("Could not find constructor with params: "
+        throw new YarnRuntimeException("Could not find constructor with params: "
             + Long.TYPE + ", " + InetSocketAddress.class + ", "
             + Configuration.class, e);
       }
@@ -99,11 +99,11 @@ public class RpcServerFactoryPBImpl implements RpcServerFactory {
     try {
       service = constructor.newInstance(instance);
     } catch (InvocationTargetException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (IllegalAccessException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (InstantiationException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
 
     Class<?> pbProtocol = service.getClass().getInterfaces()[0];
@@ -113,7 +113,7 @@ public class RpcServerFactoryPBImpl implements RpcServerFactory {
       try {
         protoClazz = localConf.getClassByName(getProtoClassName(protocol));
       } catch (ClassNotFoundException e) {
-        throw new YarnException("Failed to load class: ["
+        throw new YarnRuntimeException("Failed to load class: ["
             + getProtoClassName(protocol) + "]", e);
       }
       try {
@@ -122,7 +122,7 @@ public class RpcServerFactoryPBImpl implements RpcServerFactory {
         method.setAccessible(true);
         protoCache.putIfAbsent(protocol, method);
       } catch (NoSuchMethodException e) {
-        throw new YarnException(e);
+        throw new YarnRuntimeException(e);
       }
     }
     
@@ -130,11 +130,11 @@ public class RpcServerFactoryPBImpl implements RpcServerFactory {
       return createServer(pbProtocol, addr, conf, secretManager, numHandlers,
           (BlockingService)method.invoke(null, service), portRangeConfig);
     } catch (InvocationTargetException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (IllegalAccessException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (IOException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
   

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RpcFactoryProvider.java

@@ -22,7 +22,7 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RpcClientFactory;
 import org.apache.hadoop.yarn.factories.RpcServerFactory;
@@ -61,13 +61,13 @@ public class RpcFactoryProvider {
       method.setAccessible(true);
       return method.invoke(null, null);
     } catch (ClassNotFoundException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (NoSuchMethodException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (InvocationTargetException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     } catch (IllegalAccessException e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
   

+ 12 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/RPCUtil.java

@@ -23,24 +23,24 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 
 import com.google.protobuf.ServiceException;
 
 public class RPCUtil {
 
   /**
-   * Returns an instance of YarnRemoteException 
+   * Returns an instance of {@link YarnException}
    */
-  public static YarnRemoteException getRemoteException(Throwable t) {
-    return new YarnRemoteException(t);
+  public static YarnException getRemoteException(Throwable t) {
+    return new YarnException(t);
   }
 
   /**
-   * Returns an instance of YarnRemoteException
+   * Returns an instance of {@link YarnException}
    */
-  public static YarnRemoteException getRemoteException(String message) {
-    return new YarnRemoteException(message);
+  public static YarnException getRemoteException(String message) {
+    return new YarnException(message);
   }
 
   private static <T extends Throwable> T instantiateException(
@@ -74,10 +74,10 @@ public class RPCUtil {
    * @param se
    *          ServiceException
    * @return An instance of the actual exception, which will be a subclass of
-   *         {@link YarnRemoteException} or {@link IOException}
+   *         {@link YarnException} or {@link IOException}
    */
   public static Void unwrapAndThrowException(ServiceException se)
-      throws IOException, YarnRemoteException {
+      throws IOException, YarnException {
     Throwable cause = se.getCause();
     if (cause == null) {
       // SE generated by the RPC layer itself.
@@ -92,12 +92,12 @@ public class RPCUtil {
           // Assume this to be a new exception type added to YARN. This isn't
           // absolutely correct since the RPC layer could add an exception as
           // well.
-          throw instantiateException(YarnRemoteException.class, re);
+          throw instantiateException(YarnException.class, re);
         }
 
-        if (YarnRemoteException.class.isAssignableFrom(realClass)) {
+        if (YarnException.class.isAssignableFrom(realClass)) {
           throw instantiateException(
-              realClass.asSubclass(YarnRemoteException.class), re);
+              realClass.asSubclass(YarnException.class), re);
         } else if (IOException.class.isAssignableFrom(realClass)) {
           throw instantiateException(realClass.asSubclass(IOException.class),
               re);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java

@@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 /**
@@ -63,7 +63,7 @@ public abstract class YarnRPC {
     try {
       return (YarnRPC) Class.forName(clazzName).newInstance();
     } catch (Exception e) {
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
   }
 

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java

@@ -54,7 +54,7 @@ import org.apache.hadoop.io.SecureIOUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.file.tfile.TFile;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -365,7 +365,7 @@ public class AggregatedLogFormat {
             try {
               aclString = valueStream.readUTF();
             } catch (EOFException e) {
-              throw new YarnException("Error reading ACLs", e);
+              throw new YarnRuntimeException("Error reading ACLs", e);
             }
             acls.put(ApplicationAccessType.valueOf(appAccessOp), aclString);
           }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/AdminACLsManager.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 
 public class AdminACLsManager {
 
@@ -69,7 +69,7 @@ public class AdminACLsManager {
       adminAcl.addUser(owner.getShortUserName());
     } catch (IOException e){
       LOG.warn("Could not add current user to admin:" + e);
-      throw new YarnException(e);
+      throw new YarnRuntimeException(e);
     }
 
     aclsEnabled = conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java

@@ -37,7 +37,7 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecret
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.util.Records;
 
@@ -104,7 +104,7 @@ public class RMDelegationTokenIdentifier extends AbstractDelegationTokenIdentifi
               Records.newRecord(RenewDelegationTokenRequest.class);
           request.setDelegationToken(convertToProtoToken(token));
           return rmClient.renewDelegationToken(request).getNextExpirationTime();
-        } catch (YarnRemoteException e) {
+        } catch (YarnException e) {
           throw new IOException(e);
         } finally {
           RPC.stopProxy(rmClient);
@@ -126,7 +126,7 @@ public class RMDelegationTokenIdentifier extends AbstractDelegationTokenIdentifi
               Records.newRecord(CancelDelegationTokenRequest.class);
           request.setDelegationToken(convertToProtoToken(token));
           rmClient.cancelDelegationToken(request);
-        } catch (YarnRemoteException e) {
+        } catch (YarnException e) {
           throw new IOException(e);
         } finally {
           RPC.stopProxy(rmClient);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/CompositeService.java

@@ -26,7 +26,7 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 
 /**
  * Composition of services.
@@ -75,7 +75,7 @@ public class CompositeService extends AbstractService {
       // call stop() on all services including failed service to make sure cleanup
       // happens.
       stop(i);
-      throw new YarnException("Failed to Start " + getName(), e);
+      throw new YarnRuntimeException("Failed to Start " + getName(), e);
     }
 
   }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java

@@ -18,9 +18,9 @@
 
 package org.apache.hadoop.yarn.state;
 
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 
-public class InvalidStateTransitonException extends YarnException {
+public class InvalidStateTransitonException extends YarnRuntimeException {
 
  private Enum<?> currentState;
   private Enum<?> event;

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java

@@ -22,7 +22,7 @@ import java.util.Iterator;
 import java.util.Map;
 
 import org.apache.hadoop.util.StringInterner;
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 
 import static org.apache.hadoop.yarn.util.StringHelper.*;
@@ -56,7 +56,7 @@ public class Apps {
   }
 
   public static void throwParseException(String name, String s) {
-    throw new YarnException(join("Error parsing ", name, ": ", s));
+    throw new YarnRuntimeException(join("Error parsing ", name, ": ", s));
   }
 
   public static void setEnvFromInputString(Map<String, String> env,

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebAppException.java

@@ -18,9 +18,9 @@
 
 package org.apache.hadoop.yarn.webapp;
 
-import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.YarnRuntimeException;
 
-public class WebAppException extends YarnException {
+public class WebAppException extends YarnRuntimeException {
 
   private static final long serialVersionUID = 1L;
 

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java

@@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
@@ -130,7 +130,7 @@ public class TestContainerLaunchRPC {
 
     @Override
     public GetContainerStatusResponse getContainerStatus(
-        GetContainerStatusRequest request) throws YarnRemoteException {
+        GetContainerStatusRequest request) throws YarnException {
       GetContainerStatusResponse response = recordFactory
           .newRecordInstance(GetContainerStatusResponse.class);
       response.setStatus(status);
@@ -139,23 +139,23 @@ public class TestContainerLaunchRPC {
 
     @Override
     public StartContainerResponse startContainer(StartContainerRequest request)
-        throws YarnRemoteException, IOException {
+        throws YarnException, IOException {
       try {
         // make the thread sleep to look like its not going to respond
         Thread.sleep(10000);
       } catch (Exception e) {
         LOG.error(e);
-        throw new YarnRemoteException(e);
+        throw new YarnException(e);
       }
-      throw new YarnRemoteException("Shouldn't happen!!");
+      throw new YarnException("Shouldn't happen!!");
     }
 
     @Override
     public StopContainerResponse stopContainer(StopContainerRequest request)
-        throws YarnRemoteException {
+        throws YarnException {
       Exception e = new Exception("Dummy function", new Exception(
           "Dummy function cause"));
-      throw new YarnRemoteException(e);
+      throw new YarnException(e);
     }
   }
 }

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java

@@ -50,7 +50,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
@@ -86,7 +86,7 @@ public class TestRPC {
       proxy.getNewApplication(Records
           .newRecord(GetNewApplicationRequest.class));
       Assert.fail("Excepted RPC call to fail with unknown method.");
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       Assert.assertTrue(e.getMessage().matches(
           "Unknown method getNewApplication called on.*"
               + "org.apache.hadoop.yarn.proto.ClientRMProtocol"
@@ -147,7 +147,7 @@ public class TestRPC {
       StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
       stopRequest.setContainerId(containerId);
       proxy.stopContainer(stopRequest);
-    } catch (YarnRemoteException e) {
+    } catch (YarnException e) {
       exception = true;
       Assert.assertTrue(e.getMessage().contains(EXCEPTION_MSG));
       Assert.assertTrue(e.getMessage().contains(EXCEPTION_CAUSE));
@@ -169,7 +169,7 @@ public class TestRPC {
     @Override
     public GetContainerStatusResponse getContainerStatus(
         GetContainerStatusRequest request)
-    throws YarnRemoteException {
+    throws YarnException {
       GetContainerStatusResponse response = 
           recordFactory.newRecordInstance(GetContainerStatusResponse.class);
       response.setStatus(status);
@@ -178,7 +178,7 @@ public class TestRPC {
 
     @Override
     public StartContainerResponse startContainer(StartContainerRequest request) 
-        throws YarnRemoteException {
+        throws YarnException {
       Token containerToken = request.getContainerToken();
       ContainerTokenIdentifier tokenId = null;
 
@@ -199,10 +199,10 @@ public class TestRPC {
 
     @Override
     public StopContainerResponse stopContainer(StopContainerRequest request) 
-    throws YarnRemoteException {
+    throws YarnException {
       Exception e = new Exception(EXCEPTION_MSG, 
           new Exception(EXCEPTION_CAUSE));
-      throw new YarnRemoteException(e);
+      throw new YarnException(e);
     }
   }
 

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPCFactories.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.impl.pb.RpcClientFactoryPBImpl;
 import org.apache.hadoop.yarn.factories.impl.pb.RpcServerFactoryPBImpl;
 import org.junit.Test;
@@ -61,7 +61,7 @@ public class TestRPCFactories {
         RpcServerFactoryPBImpl.get().getServer(
             AMRMProtocol.class, instance, addr, conf, null, 1);
       server.start();
-    } catch (YarnException e) {
+    } catch (YarnRuntimeException e) {
       e.printStackTrace();
       Assert.fail("Failed to create server");
     } finally {
@@ -89,12 +89,12 @@ public class TestRPCFactories {
       AMRMProtocol amrmClient = null;
       try {
         amrmClient = (AMRMProtocol) RpcClientFactoryPBImpl.get().getClient(AMRMProtocol.class, 1, NetUtils.getConnectAddress(server), conf);
-      } catch (YarnException e) {
+      } catch (YarnRuntimeException e) {
         e.printStackTrace();
         Assert.fail("Failed to create client");
       }
       
-    } catch (YarnException e) {
+    } catch (YarnRuntimeException e) {
       e.printStackTrace();
       Assert.fail("Failed to create server");
     } finally {
@@ -108,7 +108,7 @@ public class TestRPCFactories {
 
     @Override
     public RegisterApplicationMasterResponse registerApplicationMaster(
-        RegisterApplicationMasterRequest request) throws YarnRemoteException,
+        RegisterApplicationMasterRequest request) throws YarnException,
         IOException {
       // TODO Auto-generated method stub
       return null;
@@ -116,7 +116,7 @@ public class TestRPCFactories {
 
     @Override
     public FinishApplicationMasterResponse finishApplicationMaster(
-        FinishApplicationMasterRequest request) throws YarnRemoteException,
+        FinishApplicationMasterRequest request) throws YarnException,
         IOException {
       // TODO Auto-generated method stub
       return null;
@@ -124,7 +124,7 @@ public class TestRPCFactories {
 
     @Override
     public AllocateResponse allocate(AllocateRequest request)
-        throws YarnRemoteException, IOException {
+        throws YarnException, IOException {
       // TODO Auto-generated method stub
       return null;
     }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRecordFactory.java

@@ -38,7 +38,7 @@ public class TestRecordFactory {
       AllocateResponse response =
           pbRecordFactory.newRecordInstance(AllocateResponse.class);
       Assert.assertEquals(AllocateResponsePBImpl.class, response.getClass());
-    } catch (YarnException e) {
+    } catch (YarnRuntimeException e) {
       e.printStackTrace();
       Assert.fail("Failed to crete record");
     }
@@ -47,7 +47,7 @@ public class TestRecordFactory {
       AllocateRequest response =
           pbRecordFactory.newRecordInstance(AllocateRequest.class);
       Assert.assertEquals(AllocateRequestPBImpl.class, response.getClass());
-    } catch (YarnException e) {
+    } catch (YarnRuntimeException e) {
       e.printStackTrace();
       Assert.fail("Failed to crete record");
     }

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRpcFactoryProvider.java

@@ -50,12 +50,12 @@ public class TestRpcFactoryProvider {
     try {
       clientFactory = RpcFactoryProvider.getClientFactory(conf);
       Assert.fail("Expected an exception - unknown serializer");
-    } catch (YarnException e) {
+    } catch (YarnRuntimeException e) {
     }
     try {
       serverFactory = RpcFactoryProvider.getServerFactory(conf);
       Assert.fail("Expected an exception - unknown serializer");
-    } catch (YarnException e) {
+    } catch (YarnRuntimeException e) {
     }
     
     conf = new Configuration();
@@ -65,11 +65,11 @@ public class TestRpcFactoryProvider {
     try {
       clientFactory = RpcFactoryProvider.getClientFactory(conf);
       Assert.fail("Expected an exception - unknown class");
-    } catch (YarnException e) {
+    } catch (YarnRuntimeException e) {
     }
     try {
       serverFactory = RpcFactoryProvider.getServerFactory(conf);
-    } catch (YarnException e) {
+    } catch (YarnRuntimeException e) {
       Assert.fail("Error while loading factory using reflection: [" + RpcServerFactoryPBImpl.class.getName() + "]");
     }
   }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestYarnUncaughtExceptionHandler.java

@@ -30,7 +30,7 @@ public class TestYarnUncaughtExceptionHandler {
   private static final YarnUncaughtExceptionHandler exHandler =
         new YarnUncaughtExceptionHandler();
   /**
-   * Throw {@code YarnException} inside thread and
+   * Throw {@code YarnRuntimeException} inside thread and
    * check {@code YarnUncaughtExceptionHandler} instance
    *
    * @throws InterruptedException
@@ -39,7 +39,7 @@ public class TestYarnUncaughtExceptionHandler {
   public void testUncaughtExceptionHandlerWithRuntimeException()
       throws InterruptedException {
     final YarnUncaughtExceptionHandler spyYarnHandler = spy(exHandler);
-    final YarnException yarnException = new YarnException(
+    final YarnRuntimeException yarnException = new YarnRuntimeException(
         "test-yarn-runtime-exception");
     final Thread yarnThread = new Thread(new Runnable() {
       @Override

Неке датотеке нису приказане због велике количине промена