瀏覽代碼

YARN-630. Changed AMRMProtocol api to throw IOException and YarnRemoteException. Contributed by Xuan Gong.
MAPREDUCE-5226. Handling YarnRemoteException separately from IOException in MR App's use of AMRMProtocol after YARN-630. Contributed by Xuan Gong.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1480529 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli 12 年之前
父節點
當前提交
9c4f86879c
共有 16 個文件被更改,包括 75 次插入29 次删除
  1. 4 1
      hadoop-mapreduce-project/CHANGES.txt
  2. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
  3. 4 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
  4. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
  5. 3 0
      hadoop-yarn-project/CHANGES.txt
  6. 8 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/AMRMProtocol.java
  7. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  8. 8 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClient.java
  9. 8 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientAsync.java
  10. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java
  11. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java
  12. 5 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/AMRMProtocolPBClientImpl.java
  13. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/AMRMProtocolPBServiceImpl.java
  14. 6 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPCFactories.java
  15. 5 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  16. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

+ 4 - 1
hadoop-mapreduce-project/CHANGES.txt

@@ -390,7 +390,10 @@ Release 2.0.5-beta - UNRELEASED
     (Tsuyoshi OZAWA via cdouglas)
 
     MAPREDUCE-5212. Handling YarnRemoteException separately from IOException in
-    MR App after YARN-631. (Xuan Gong via vinodkv)
+    MR App's use of ClientRMProtocol after YARN-631. (Xuan Gong via vinodkv)
+
+    MAPREDUCE-5226. Handling YarnRemoteException separately from IOException in
+    MR App's use of AMRMProtocol after YARN-630. (Xuan Gong via vinodkv)
 
 Release 2.0.4-alpha - 2013-04-25
 

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

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.app.rm;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -144,7 +145,8 @@ public abstract class RMContainerRequestor extends RMCommunicator {
     LOG.info("blacklistDisablePercent is " + blacklistDisablePercent);
   }
 
-  protected AllocateResponse makeRemoteRequest() throws YarnRemoteException {
+  protected AllocateResponse makeRemoteRequest() throws YarnRemoteException,
+      IOException {
     AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest(
         applicationAttemptId, lastResponseID, super.getApplicationProgress(),
         new ArrayList<ResourceRequest>(ask), new ArrayList<ContainerId>(

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

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.app;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.BlockingQueue;
@@ -202,7 +203,7 @@ public class MRAppBenchmark {
               public RegisterApplicationMasterResponse
                   registerApplicationMaster(
                       RegisterApplicationMasterRequest request)
-                      throws YarnRemoteException {
+                      throws YarnRemoteException, IOException {
                 RegisterApplicationMasterResponse response =
                     Records.newRecord(RegisterApplicationMasterResponse.class);
                 response.setMinimumResourceCapability(BuilderUtils
@@ -215,7 +216,7 @@ public class MRAppBenchmark {
               @Override
               public FinishApplicationMasterResponse finishApplicationMaster(
                   FinishApplicationMasterRequest request)
-                  throws YarnRemoteException {
+                  throws YarnRemoteException, IOException {
                 FinishApplicationMasterResponse response =
                     Records.newRecord(FinishApplicationMasterResponse.class);
                 return response;
@@ -223,7 +224,7 @@ public class MRAppBenchmark {
 
               @Override
               public AllocateResponse allocate(AllocateRequest request)
-                  throws YarnRemoteException {
+                  throws YarnRemoteException, IOException {
 
                 AllocateResponse response =
                     Records.newRecord(AllocateResponse.class);

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

@@ -100,6 +100,7 @@ public class TestLocalContainerAllocator {
         when(scheduler.allocate(isA(AllocateRequest.class)))
           .thenThrow(RPCUtil.getRemoteException(new IOException("forcefail")));
       } catch (YarnRemoteException e) {
+      } catch (IOException e) {
       }
       return scheduler;
     }

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

@@ -121,6 +121,9 @@ Release 2.0.5-beta - UNRELEASED
     YARN-631. Changed ClientRMProtocol api to throw IOException and
     YarnRemoteException. (Xuan Gong via vinodkv)
 
+    YARN-630. Changed AMRMProtocol api to throw IOException and
+    YarnRemoteException. (Xuan Gong via vinodkv)
+
   NEW FEATURES
 
     YARN-482. FS: Extend SchedulingMode to intermediate queues. 

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

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.api;
 
+import java.io.IOException;
+
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
@@ -57,10 +59,11 @@ public interface AMRMProtocol {
    * @param request registration request
    * @return registration respose
    * @throws YarnRemoteException
+   * @throws IOException
    */
   public RegisterApplicationMasterResponse registerApplicationMaster(
       RegisterApplicationMasterRequest request) 
-  throws YarnRemoteException;
+  throws YarnRemoteException, IOException;
   
   /**
    * <p>The interface used by an <code>ApplicationMaster</code> to notify the 
@@ -76,10 +79,11 @@ public interface AMRMProtocol {
    * @param request completion request
    * @return completion response
    * @throws YarnRemoteException
+   * @throws IOException
    */
   public FinishApplicationMasterResponse finishApplicationMaster(
       FinishApplicationMasterRequest request) 
-  throws YarnRemoteException;
+  throws YarnRemoteException, IOException;
   
   /**
    * <p>The main interface between an <code>ApplicationMaster</code> 
@@ -105,7 +109,8 @@ public interface AMRMProtocol {
    * @param request allocation request
    * @return allocation response
    * @throws YarnRemoteException
+   * @throws IOException
    */
   public AllocateResponse allocate(AllocateRequest request) 
-  throws YarnRemoteException;
+  throws YarnRemoteException, IOException;
 }

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

@@ -433,8 +433,9 @@ public class ApplicationMaster {
    * Main run function for the application master
    *
    * @throws YarnRemoteException
+   * @throws IOException
    */
-  public boolean run() throws YarnRemoteException {
+  public boolean run() throws YarnRemoteException, IOException {
     LOG.info("Starting ApplicationMaster");
 
     AMRMClientAsync.CallbackHandler allocListener = new RMCallbackHandler();
@@ -533,6 +534,8 @@ public class ApplicationMaster {
       resourceManager.unregisterApplicationMaster(appStatus, appMessage, null);
     } catch (YarnRemoteException ex) {
       LOG.error("Failed to unregister application", ex);
+    } catch (IOException e) {
+      LOG.error("Failed to unregister application", e);
     }
     
     done = true;

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

@@ -19,6 +19,8 @@
 package org.apache.hadoop.yarn.client;
 
 
+import java.io.IOException;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
@@ -72,12 +74,13 @@ public interface AMRMClient extends Service {
    * @param appTrackingUrl URL at which the master info can be seen
    * @return <code>RegisterApplicationMasterResponse</code>
    * @throws YarnRemoteException
+   * @throws IOException
    */
   public RegisterApplicationMasterResponse 
                registerApplicationMaster(String appHostName,
                                          int appHostPort,
                                          String appTrackingUrl) 
-               throws YarnRemoteException;
+               throws YarnRemoteException, IOException;
   
   /**
    * Request additional containers and receive new container allocations.
@@ -92,9 +95,10 @@ public interface AMRMClient extends Service {
    * @param progressIndicator Indicates progress made by the master
    * @return the response of the allocate request
    * @throws YarnRemoteException
+   * @throws IOException
    */
   public AllocateResponse allocate(float progressIndicator) 
-                           throws YarnRemoteException;
+                           throws YarnRemoteException, IOException;
   
   /**
    * Unregister the application master. This must be called in the end.
@@ -102,11 +106,12 @@ public interface AMRMClient extends Service {
    * @param appMessage Diagnostics message on failure
    * @param appTrackingUrl New URL to get master info
    * @throws YarnRemoteException
+   * @throws IOException
    */
   public void unregisterApplicationMaster(FinalApplicationStatus appStatus,
                                            String appMessage,
                                            String appTrackingUrl) 
-               throws YarnRemoteException;
+               throws YarnRemoteException, IOException;
   
   /**
    * Request containers for resources before calling <code>allocate</code>

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

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.client;
 
+import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -173,10 +174,12 @@ public class AMRMClientAsync extends AbstractService {
   /**
    * Registers this application master with the resource manager. On successful
    * registration, starts the heartbeating thread.
+   * @throws YarnRemoteException
+   * @throws IOException
    */
   public RegisterApplicationMasterResponse registerApplicationMaster(
       String appHostName, int appHostPort, String appTrackingUrl)
-      throws YarnRemoteException {
+      throws YarnRemoteException, IOException {
     RegisterApplicationMasterResponse response =
         client.registerApplicationMaster(appHostName, appHostPort, appTrackingUrl);
     heartbeatThread.start();
@@ -189,9 +192,10 @@ public class AMRMClientAsync extends AbstractService {
    * @param appMessage Diagnostics message on failure
    * @param appTrackingUrl New URL to get master info
    * @throws YarnRemoteException
+   * @throws IOException
    */
   public void unregisterApplicationMaster(FinalApplicationStatus appStatus,
-      String appMessage, String appTrackingUrl) throws YarnRemoteException {
+      String appMessage, String appTrackingUrl) throws YarnRemoteException, IOException {
     synchronized (client) {
       keepRunning = false;
       client.unregisterApplicationMaster(appStatus, appMessage, appTrackingUrl);
@@ -264,6 +268,8 @@ public class AMRMClientAsync extends AbstractService {
             response = client.allocate(progress);
           } catch (YarnRemoteException ex) {
             LOG.error("Failed to heartbeat", ex);
+          } catch (IOException e) {
+            LOG.error("Failed to heartbeat", e);
           }
         }
         if (response != null) {

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

@@ -134,7 +134,7 @@ public class AMRMClientImpl extends AbstractService implements AMRMClient {
   @Override
   public RegisterApplicationMasterResponse registerApplicationMaster(
       String appHostName, int appHostPort, String appTrackingUrl)
-      throws YarnRemoteException {
+      throws YarnRemoteException, IOException {
     // do this only once ???
     RegisterApplicationMasterRequest request = recordFactory
         .newRecordInstance(RegisterApplicationMasterRequest.class);
@@ -153,7 +153,7 @@ public class AMRMClientImpl extends AbstractService implements AMRMClient {
 
   @Override
   public AllocateResponse allocate(float progressIndicator) 
-      throws YarnRemoteException {
+      throws YarnRemoteException, IOException {
     AllocateResponse allocateResponse = null;
     ArrayList<ResourceRequest> askList = null;
     ArrayList<ContainerId> releaseList = null;
@@ -207,7 +207,8 @@ public class AMRMClientImpl extends AbstractService implements AMRMClient {
 
   @Override
   public void unregisterApplicationMaster(FinalApplicationStatus appStatus,
-      String appMessage, String appTrackingUrl) throws YarnRemoteException {
+      String appMessage, String appTrackingUrl) throws YarnRemoteException,
+      IOException {
     FinishApplicationMasterRequest request = recordFactory
                   .newRecordInstance(FinishApplicationMasterRequest.class);
     request.setAppAttemptId(appAttemptId);

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

@@ -135,7 +135,7 @@ public class TestAMRMClient {
   }
 
   @Test (timeout=60000)
-  public void testAMRMClient() throws YarnRemoteException {
+  public void testAMRMClient() throws YarnRemoteException, IOException {
     AMRMClientImpl amClient = null;
     try {
       // start am rm client
@@ -159,7 +159,7 @@ public class TestAMRMClient {
   
   
   private void testAllocation(final AMRMClientImpl amClient)  
-      throws YarnRemoteException {
+      throws YarnRemoteException, IOException {
     // setup container request
     final Resource capability = Records.newRecord(Resource.class);
     final Priority priority = Records.newRecord(Priority.class);

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

@@ -68,7 +68,7 @@ public class AMRMProtocolPBClientImpl implements AMRMProtocol, Closeable {
 
   @Override
   public AllocateResponse allocate(AllocateRequest request)
-      throws YarnRemoteException {
+      throws YarnRemoteException, IOException {
     AllocateRequestProto requestProto =
         ((AllocateRequestPBImpl) request).getProto();
     try {
@@ -80,7 +80,8 @@ public class AMRMProtocolPBClientImpl implements AMRMProtocol, Closeable {
 
   @Override
   public FinishApplicationMasterResponse finishApplicationMaster(
-      FinishApplicationMasterRequest request) throws YarnRemoteException {
+      FinishApplicationMasterRequest request) throws YarnRemoteException,
+      IOException {
     FinishApplicationMasterRequestProto requestProto =
         ((FinishApplicationMasterRequestPBImpl) request).getProto();
     try {
@@ -93,7 +94,8 @@ public class AMRMProtocolPBClientImpl implements AMRMProtocol, Closeable {
 
   @Override
   public RegisterApplicationMasterResponse registerApplicationMaster(
-      RegisterApplicationMasterRequest request) throws YarnRemoteException {
+      RegisterApplicationMasterRequest request) throws YarnRemoteException,
+      IOException {
     RegisterApplicationMasterRequestProto requestProto =
         ((RegisterApplicationMasterRequestPBImpl) request).getProto();
     try {

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

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.api.impl.pb.service;
 
+import java.io.IOException;
+
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.AMRMProtocolPB;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
@@ -57,6 +59,8 @@ public class AMRMProtocolPBServiceImpl implements AMRMProtocolPB {
       return ((AllocateResponsePBImpl)response).getProto();
     } catch (YarnRemoteException e) {
       throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
     }
   }
 
@@ -70,6 +74,8 @@ public class AMRMProtocolPBServiceImpl implements AMRMProtocolPB {
       return ((FinishApplicationMasterResponsePBImpl)response).getProto();
     } catch (YarnRemoteException e) {
       throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
     }
   }
 
@@ -83,6 +89,8 @@ public class AMRMProtocolPBServiceImpl implements AMRMProtocolPB {
       return ((RegisterApplicationMasterResponsePBImpl)response).getProto();
     } catch (YarnRemoteException e) {
       throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
     }
   }
 }

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

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn;
 
+import java.io.IOException;
 import java.net.InetSocketAddress;
 
 import junit.framework.Assert;
@@ -107,21 +108,23 @@ public class TestRPCFactories {
 
     @Override
     public RegisterApplicationMasterResponse registerApplicationMaster(
-        RegisterApplicationMasterRequest request) throws YarnRemoteException {
+        RegisterApplicationMasterRequest request) throws YarnRemoteException,
+        IOException {
       // TODO Auto-generated method stub
       return null;
     }
 
     @Override
     public FinishApplicationMasterResponse finishApplicationMaster(
-        FinishApplicationMasterRequest request) throws YarnRemoteException {
+        FinishApplicationMasterRequest request) throws YarnRemoteException,
+        IOException {
       // TODO Auto-generated method stub
       return null;
     }
 
     @Override
     public AllocateResponse allocate(AllocateRequest request)
-        throws YarnRemoteException {
+        throws YarnRemoteException, IOException {
       // TODO Auto-generated method stub
       return null;
     }

+ 5 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java

@@ -162,7 +162,8 @@ public class ApplicationMasterService extends AbstractService implements
 
   @Override
   public RegisterApplicationMasterResponse registerApplicationMaster(
-      RegisterApplicationMasterRequest request) throws YarnRemoteException {
+      RegisterApplicationMasterRequest request) throws YarnRemoteException,
+      IOException {
 
     ApplicationAttemptId applicationAttemptId = request
         .getApplicationAttemptId();
@@ -211,7 +212,8 @@ public class ApplicationMasterService extends AbstractService implements
 
   @Override
   public FinishApplicationMasterResponse finishApplicationMaster(
-      FinishApplicationMasterRequest request) throws YarnRemoteException {
+      FinishApplicationMasterRequest request) throws YarnRemoteException,
+      IOException {
 
     ApplicationAttemptId applicationAttemptId = request
         .getApplicationAttemptId();
@@ -243,7 +245,7 @@ public class ApplicationMasterService extends AbstractService implements
 
   @Override
   public AllocateResponse allocate(AllocateRequest request)
-      throws YarnRemoteException {
+      throws YarnRemoteException, IOException {
 
     ApplicationAttemptId appAttemptId = request.getApplicationAttemptId();
     authorizeRequest(appAttemptId);

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

@@ -485,7 +485,8 @@ public class TestContainerManagerSecurity {
   }
 
   private Container requestAndGetContainer(AMRMProtocol scheduler,
-      ApplicationId appID) throws YarnRemoteException, InterruptedException {
+      ApplicationId appID) throws YarnRemoteException, InterruptedException,
+      IOException {
 
     // Request a container allocation.
     List<ResourceRequest> ask = new ArrayList<ResourceRequest>();