Prechádzať zdrojové kódy

YARN-1523. Use StandbyException instead of RMNotYetReadyException (kasha)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1553620 13f79535-47bb-0310-9956-ffa450edef68
Karthik Kambatla 11 rokov pred
rodič
commit
b99dd5de74

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

@@ -171,6 +171,8 @@ Release 2.4.0 - UNRELEASED
 
     YARN-1172. Convert SecretManagers in RM to services (Tsuyoshi OZAWA via kasha)
 
+    YARN-1523. Use StandbyException instead of RMNotYetReadyException (kasha)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 0 - 36
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/RMNotYetActiveException.java

@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.exceptions;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Exception to be thrown when an Active-Only operation is attempted on a
- * ResourceManager that is not Active.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class RMNotYetActiveException extends YarnException {
-  private static final long serialVersionUID = 1L;
-
-  public RMNotYetActiveException() {
-    super("ResourceManager is not yet Active!");
-  }
-}

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java

@@ -24,10 +24,10 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
-import org.apache.hadoop.yarn.exceptions.RMNotYetActiveException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
@@ -51,25 +51,25 @@ public interface ResourceManagerAdministrationProtocol extends GetUserMappingsPr
   @Public
   @Stable
   public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request) 
-  throws RMNotYetActiveException, YarnException, IOException;
+  throws StandbyException, YarnException, IOException;
 
   @Public
   @Stable
   public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
-  throws RMNotYetActiveException, YarnException, IOException;
+  throws StandbyException, YarnException, IOException;
 
   @Public
   @Stable
   public RefreshSuperUserGroupsConfigurationResponse 
   refreshSuperUserGroupsConfiguration(
       RefreshSuperUserGroupsConfigurationRequest request)
-  throws RMNotYetActiveException, YarnException, IOException;
+  throws StandbyException, YarnException, IOException;
 
   @Public
   @Stable
   public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
       RefreshUserToGroupsMappingsRequest request)
-  throws RMNotYetActiveException, YarnException, IOException;
+  throws StandbyException, YarnException, IOException;
 
   @Public
   @Stable

+ 16 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC.Server;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
@@ -49,7 +50,6 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.RMNotYetActiveException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -82,6 +82,7 @@ public class AdminService extends AbstractService implements
 
   private final RMContext rmContext;
   private final ResourceManager rm;
+  private String rmId;
 
   private Server server;
   private InetSocketAddress masterServiceAddress;
@@ -105,6 +106,7 @@ public class AdminService extends AbstractService implements
     adminAcl = new AccessControlList(conf.get(
         YarnConfiguration.YARN_ADMIN_ACL,
         YarnConfiguration.DEFAULT_YARN_ADMIN_ACL));
+    rmId = conf.get(YarnConfiguration.RM_HA_ID);
     super.serviceInit(conf);
   }
 
@@ -176,6 +178,10 @@ public class AdminService extends AbstractService implements
     return HAServiceState.ACTIVE == rmContext.getHAServiceState();
   }
 
+  private void throwStandbyException() throws StandbyException {
+    throw new StandbyException("ResourceManager " + rmId + " is not Active!");
+  }
+
   @Override
   public synchronized void monitorHealth()
       throws IOException {
@@ -239,14 +245,14 @@ public class AdminService extends AbstractService implements
 
   @Override
   public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
-      throws YarnException {
+      throws YarnException, StandbyException {
     UserGroupInformation user = checkAcls("refreshQueues");
 
     if (!isRMActive()) {
       RMAuditLogger.logFailure(user.getShortUserName(), "refreshQueues",
           adminAcl.toString(), "AdminService",
           "ResourceManager is not active. Can not refresh queues.");
-      throw new RMNotYetActiveException();
+      throwStandbyException();
     }
 
     try {
@@ -265,14 +271,14 @@ public class AdminService extends AbstractService implements
 
   @Override
   public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
-      throws YarnException {
+      throws YarnException, StandbyException {
     UserGroupInformation user = checkAcls("refreshNodes");
 
     if (!isRMActive()) {
       RMAuditLogger.logFailure(user.getShortUserName(), "refreshNodes",
           adminAcl.toString(), "AdminService",
           "ResourceManager is not active. Can not refresh nodes.");
-      throw new RMNotYetActiveException();
+      throwStandbyException();
     }
 
     try {
@@ -291,7 +297,7 @@ public class AdminService extends AbstractService implements
   @Override
   public RefreshSuperUserGroupsConfigurationResponse refreshSuperUserGroupsConfiguration(
       RefreshSuperUserGroupsConfigurationRequest request)
-      throws YarnException {
+      throws YarnException, StandbyException {
     UserGroupInformation user = checkAcls("refreshSuperUserGroupsConfiguration");
 
     // TODO (YARN-1459): Revisit handling super-user-groups on Standby RM
@@ -300,7 +306,7 @@ public class AdminService extends AbstractService implements
           "refreshSuperUserGroupsConfiguration",
           adminAcl.toString(), "AdminService",
           "ResourceManager is not active. Can not refresh super-user-groups.");
-      throw new RMNotYetActiveException();
+      throwStandbyException();
     }
 
     ProxyUsers.refreshSuperUserGroupsConfiguration(new Configuration());
@@ -313,7 +319,8 @@ public class AdminService extends AbstractService implements
 
   @Override
   public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
-      RefreshUserToGroupsMappingsRequest request) throws YarnException {
+      RefreshUserToGroupsMappingsRequest request)
+      throws YarnException, StandbyException {
     UserGroupInformation user = checkAcls("refreshUserToGroupsMappings");
 
     // TODO (YARN-1459): Revisit handling user-groups on Standby RM
@@ -322,7 +329,7 @@ public class AdminService extends AbstractService implements
           "refreshUserToGroupsMapping",
           adminAcl.toString(), "AdminService",
           "ResourceManager is not active. Can not refresh user-groups.");
-      throw new RMNotYetActiveException();
+      throwStandbyException();
     }
 
     Groups.getUserToGroupsMappingService().refresh();