You are viewing a plain text version of this content. The canonical link for it is here.
Posted to yarn-commits@hadoop.apache.org by vi...@apache.org on 2013/06/04 06:50:22 UTC

svn commit: r1489288 [2/5] - in /hadoop/common/branches/branch-2/hadoop-yarn-project: ./ hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/ hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ hadoop-yarn/hadoop-yarn-api...

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java Tue Jun  4 04:50:16 2013
@@ -57,7 +57,7 @@ import org.apache.hadoop.yarn.api.record
 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 Abst
 
   @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 Abst
   @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 Abst
 
   @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 Abst
 
   @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 Abst
 
   @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 Abst
   }
 
   @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 Abst
   }
 
   @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 Abst
 
   @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 Abst
   }
 
   @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 Abst
   }
 
   @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 Abst
   }
 
   @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 Abst
   }
 
   @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 Abst
 
   @Override
   public List<QueueInfo> getChildQueueInfos(String parent)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     List<QueueInfo> queues = new ArrayList<QueueInfo>();
 
     QueueInfo parentQueue =

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java Tue Jun  4 04:50:16 2013
@@ -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 Yarn
   /**
    * 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 Yarn
    * 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 Yarn
    * 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.

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java Tue Jun  4 04:50:16 2013
@@ -31,7 +31,7 @@ import org.apache.commons.lang.time.Date
 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();

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java Tue Jun  4 04:50:16 2013
@@ -53,7 +53,7 @@ import org.apache.hadoop.yarn.api.record
 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);

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java Tue Jun  4 04:50:16 2013
@@ -45,7 +45,7 @@ import org.apache.hadoop.yarn.api.record
 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);
 

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java Tue Jun  4 04:50:16 2013
@@ -52,7 +52,7 @@ import org.apache.hadoop.yarn.api.record
 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(

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java Tue Jun  4 04:50:16 2013
@@ -48,7 +48,7 @@ import org.apache.hadoop.yarn.api.record
 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;

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java Tue Jun  4 04:50:16 2013
@@ -39,7 +39,7 @@ import org.apache.hadoop.yarn.api.record
 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.");

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/AMRMProtocolPBClientImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/AMRMProtocolPBClientImpl.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/AMRMProtocolPBClientImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/AMRMProtocolPBClientImpl.java Tue Jun  4 04:50:16 2013
@@ -39,7 +39,7 @@ import org.apache.hadoop.yarn.api.protoc
 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 im
 
   @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 im
 
   @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 im
 
   @Override
   public RegisterApplicationMasterResponse registerApplicationMaster(
-      RegisterApplicationMasterRequest request) throws YarnRemoteException,
+      RegisterApplicationMasterRequest request) throws YarnException,
       IOException {
     RegisterApplicationMasterRequestProto requestProto =
         ((RegisterApplicationMasterRequestPBImpl) request).getProto();

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java Tue Jun  4 04:50:16 2013
@@ -78,7 +78,7 @@ import org.apache.hadoop.yarn.api.protoc
 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 ClientRMProtocolPBClientImp
 
   @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 ClientRMProtocolPBClientImp
 
   @Override
   public GetApplicationReportResponse getApplicationReport(
-      GetApplicationReportRequest request) throws YarnRemoteException,
+      GetApplicationReportRequest request) throws YarnException,
       IOException {
     GetApplicationReportRequestProto requestProto =
         ((GetApplicationReportRequestPBImpl) request).getProto();
@@ -142,7 +142,7 @@ public class ClientRMProtocolPBClientImp
 
   @Override
   public GetClusterMetricsResponse getClusterMetrics(
-      GetClusterMetricsRequest request) throws YarnRemoteException,
+      GetClusterMetricsRequest request) throws YarnException,
       IOException {
     GetClusterMetricsRequestProto requestProto =
         ((GetClusterMetricsRequestPBImpl) request).getProto();
@@ -157,7 +157,7 @@ public class ClientRMProtocolPBClientImp
 
   @Override
   public GetNewApplicationResponse getNewApplication(
-      GetNewApplicationRequest request) throws YarnRemoteException,
+      GetNewApplicationRequest request) throws YarnException,
       IOException {
     GetNewApplicationRequestProto requestProto =
         ((GetNewApplicationRequestPBImpl) request).getProto();
@@ -172,7 +172,7 @@ public class ClientRMProtocolPBClientImp
 
   @Override
   public SubmitApplicationResponse submitApplication(
-      SubmitApplicationRequest request) throws YarnRemoteException,
+      SubmitApplicationRequest request) throws YarnException,
       IOException {
     SubmitApplicationRequestProto requestProto =
         ((SubmitApplicationRequestPBImpl) request).getProto();
@@ -187,7 +187,7 @@ public class ClientRMProtocolPBClientImp
 
   @Override
   public GetAllApplicationsResponse getAllApplications(
-      GetAllApplicationsRequest request) throws YarnRemoteException,
+      GetAllApplicationsRequest request) throws YarnException,
       IOException {
     GetAllApplicationsRequestProto requestProto =
         ((GetAllApplicationsRequestPBImpl) request).getProto();
@@ -203,7 +203,7 @@ public class ClientRMProtocolPBClientImp
   @Override
   public GetClusterNodesResponse
       getClusterNodes(GetClusterNodesRequest request)
-          throws YarnRemoteException, IOException {
+          throws YarnException, IOException {
     GetClusterNodesRequestProto requestProto =
         ((GetClusterNodesRequestPBImpl) request).getProto();
     try {
@@ -217,7 +217,7 @@ public class ClientRMProtocolPBClientImp
 
   @Override
   public GetQueueInfoResponse getQueueInfo(GetQueueInfoRequest request)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     GetQueueInfoRequestProto requestProto =
         ((GetQueueInfoRequestPBImpl) request).getProto();
     try {
@@ -231,7 +231,7 @@ public class ClientRMProtocolPBClientImp
 
   @Override
   public GetQueueUserAclsInfoResponse getQueueUserAcls(
-      GetQueueUserAclsInfoRequest request) throws YarnRemoteException,
+      GetQueueUserAclsInfoRequest request) throws YarnException,
       IOException {
     GetQueueUserAclsInfoRequestProto requestProto =
         ((GetQueueUserAclsInfoRequestPBImpl) request).getProto();
@@ -246,7 +246,7 @@ public class ClientRMProtocolPBClientImp
 
   @Override
   public GetDelegationTokenResponse getDelegationToken(
-      GetDelegationTokenRequest request) throws YarnRemoteException,
+      GetDelegationTokenRequest request) throws YarnException,
       IOException {
     GetDelegationTokenRequestProto requestProto =
         ((GetDelegationTokenRequestPBImpl) request).getProto();
@@ -261,7 +261,7 @@ public class ClientRMProtocolPBClientImp
 
   @Override
   public RenewDelegationTokenResponse renewDelegationToken(
-      RenewDelegationTokenRequest request) throws YarnRemoteException,
+      RenewDelegationTokenRequest request) throws YarnException,
       IOException {
     RenewDelegationTokenRequestProto requestProto = 
         ((RenewDelegationTokenRequestPBImpl) request).getProto();
@@ -276,7 +276,7 @@ public class ClientRMProtocolPBClientImp
 
   @Override
   public CancelDelegationTokenResponse cancelDelegationToken(
-      CancelDelegationTokenRequest request) throws YarnRemoteException,
+      CancelDelegationTokenRequest request) throws YarnException,
       IOException {
     CancelDelegationTokenRequestProto requestProto =
         ((CancelDelegationTokenRequestPBImpl) request).getProto();

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java Tue Jun  4 04:50:16 2013
@@ -42,7 +42,7 @@ import org.apache.hadoop.yarn.api.protoc
 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 ContainerManagerPBClientImp
 
   @Override
   public GetContainerStatusResponse getContainerStatus(
-      GetContainerStatusRequest request) throws YarnRemoteException,
+      GetContainerStatusRequest request) throws YarnException,
       IOException {
     GetContainerStatusRequestProto requestProto =
         ((GetContainerStatusRequestPBImpl) request).getProto();
@@ -101,7 +101,7 @@ public class ContainerManagerPBClientImp
 
   @Override
   public StartContainerResponse startContainer(StartContainerRequest request)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     StartContainerRequestProto requestProto =
         ((StartContainerRequestPBImpl) request).getProto();
     try {
@@ -115,7 +115,7 @@ public class ContainerManagerPBClientImp
 
   @Override
   public StopContainerResponse stopContainer(StopContainerRequest request)
-      throws YarnRemoteException, IOException {
+      throws YarnException, IOException {
     StopContainerRequestProto requestProto =
         ((StopContainerRequestPBImpl) request).getProto();
     try {

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java Tue Jun  4 04:50:16 2013
@@ -52,7 +52,7 @@ import org.apache.hadoop.yarn.api.protoc
 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
 
   @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
 
   @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
   @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
 
   @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
 
   @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
 
   @Override
   public RefreshServiceAclsResponse refreshServiceAcls(
-      RefreshServiceAclsRequest request) throws YarnRemoteException,
+      RefreshServiceAclsRequest request) throws YarnException,
       IOException {
     RefreshServiceAclsRequestProto requestProto = 
         ((RefreshServiceAclsRequestPBImpl)request).getProto();

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/AMRMProtocolPBServiceImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/AMRMProtocolPBServiceImpl.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/AMRMProtocolPBServiceImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/AMRMProtocolPBServiceImpl.java Tue Jun  4 04:50:16 2013
@@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.api.protoc
 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 i
     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 i
     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 i
     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);

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java Tue Jun  4 04:50:16 2013
@@ -64,7 +64,7 @@ import org.apache.hadoop.yarn.api.protoc
 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 ClientRMProtocolPBServiceIm
     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 ClientRMProtocolPBServiceIm
     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 ClientRMProtocolPBServiceIm
     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 ClientRMProtocolPBServiceIm
     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 ClientRMProtocolPBServiceIm
     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 ClientRMProtocolPBServiceIm
     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 ClientRMProtocolPBServiceIm
     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 ClientRMProtocolPBServiceIm
     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 ClientRMProtocolPBServiceIm
     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 ClientRMProtocolPBServiceIm
       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 ClientRMProtocolPBServiceIm
       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 ClientRMProtocolPBServiceIm
       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);

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagerPBServiceImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagerPBServiceImpl.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagerPBServiceImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagerPBServiceImpl.java Tue Jun  4 04:50:16 2013
@@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.api.protoc
 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 ContainerManagerPBServiceIm
     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 ContainerManagerPBServiceIm
     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 ContainerManagerPBServiceIm
     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);

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java Tue Jun  4 04:50:16 2013
@@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.api.protoc
 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 RMAdminProtocolPBServiceImp
     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 RMAdminProtocolPBServiceImp
     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 RMAdminProtocolPBServiceImp
     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 RMAdminProtocolPBServiceImp
       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 RMAdminProtocolPBServiceImp
       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 RMAdminProtocolPBServiceImp
         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);

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java Tue Jun  4 04:50:16 2013
@@ -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 Abs
         if (!stopped) {
           LOG.warn("AsyncDispatcher thread interrupted", e);
         }
-        throw new YarnException(e);
+        throw new YarnRuntimeException(e);
       }
     };
   }

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RecordFactoryPBImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RecordFactoryPBImpl.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RecordFactoryPBImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RecordFactoryPBImpl.java Tue Jun  4 04:50:16 2013
@@ -24,7 +24,7 @@ import java.util.concurrent.ConcurrentHa
 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 impleme
       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 impleme
         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);
     }
   }
 

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java Tue Jun  4 04:50:16 2013
@@ -28,7 +28,7 @@ import java.util.concurrent.ConcurrentMa
 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 impl
       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 impl
         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 impl
       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);
     }
   }
 

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcServerFactoryPBImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcServerFactoryPBImpl.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcServerFactoryPBImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcServerFactoryPBImpl.java Tue Jun  4 04:50:16 2013
@@ -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 impl
         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 impl
         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 impl
     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 impl
       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 impl
         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 impl
       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);
     }
   }
   

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RpcFactoryProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RpcFactoryProvider.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RpcFactoryProvider.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RpcFactoryProvider.java Tue Jun  4 04:50:16 2013
@@ -22,7 +22,7 @@ import java.lang.reflect.InvocationTarge
 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);
     }
   }
   

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/RPCUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/RPCUtil.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/RPCUtil.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/RPCUtil.java Tue Jun  4 04:50:16 2013
@@ -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);

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java Tue Jun  4 04:50:16 2013
@@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configurat
 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);
     }
   }
 

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java Tue Jun  4 04:50:16 2013
@@ -54,7 +54,7 @@ import org.apache.hadoop.io.SecureIOUtil
 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);
           }

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/AdminACLsManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/AdminACLsManager.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/AdminACLsManager.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/AdminACLsManager.java Tue Jun  4 04:50:16 2013
@@ -27,7 +27,7 @@ import org.apache.hadoop.conf.Configurat
 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,

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java Tue Jun  4 04:50:16 2013
@@ -37,7 +37,7 @@ import org.apache.hadoop.security.token.
 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
               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
               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);

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/CompositeService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/CompositeService.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/CompositeService.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/CompositeService.java Tue Jun  4 04:50:16 2013
@@ -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 Ab
       // 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);
     }
 
   }

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java Tue Jun  4 04:50:16 2013
@@ -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;

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java Tue Jun  4 04:50:16 2013
@@ -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,

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebAppException.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebAppException.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebAppException.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebAppException.java Tue Jun  4 04:50:16 2013
@@ -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;
 

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java?rev=1489288&r1=1489287&r2=1489288&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java Tue Jun  4 04:50:16 2013
@@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.api.record
 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);
     }
   }
 }