|
@@ -21,18 +21,31 @@ package org.apache.hadoop.yarn.server.resourcemanager;
|
|
|
import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
+import com.google.protobuf.BlockingService;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
-import org.apache.hadoop.ipc.Server;
|
|
|
+import org.apache.hadoop.ha.HAServiceProtocol;
|
|
|
+import org.apache.hadoop.ha.HAServiceStatus;
|
|
|
+import org.apache.hadoop.ha.HealthCheckFailedException;
|
|
|
+import org.apache.hadoop.ha.ServiceFailedException;
|
|
|
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
|
|
|
+import org.apache.hadoop.ha.protocolPB.HAServiceProtocolPB;
|
|
|
+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.security.Groups;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.authorize.AccessControlList;
|
|
|
import org.apache.hadoop.security.authorize.PolicyProvider;
|
|
|
import org.apache.hadoop.security.authorize.ProxyUsers;
|
|
|
import org.apache.hadoop.service.AbstractService;
|
|
|
+import org.apache.hadoop.yarn.conf.HAUtil;
|
|
|
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;
|
|
@@ -51,22 +64,20 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsC
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
|
|
|
|
|
|
-public class AdminService extends AbstractService implements ResourceManagerAdministrationProtocol {
|
|
|
+public class AdminService extends AbstractService implements
|
|
|
+ HAServiceProtocol, ResourceManagerAdministrationProtocol {
|
|
|
|
|
|
private static final Log LOG = LogFactory.getLog(AdminService.class);
|
|
|
|
|
|
- private final Configuration conf;
|
|
|
- private final ResourceScheduler scheduler;
|
|
|
private final RMContext rmContext;
|
|
|
- private final NodesListManager nodesListManager;
|
|
|
-
|
|
|
- private final ClientRMService clientRMService;
|
|
|
- private final ApplicationMasterService applicationMasterService;
|
|
|
- private final ResourceTrackerService resourceTrackerService;
|
|
|
-
|
|
|
+ private final ResourceManager rm;
|
|
|
+ @VisibleForTesting
|
|
|
+ protected HAServiceProtocol.HAServiceState
|
|
|
+ haState = HAServiceProtocol.HAServiceState.INITIALIZING;
|
|
|
+ boolean haEnabled;
|
|
|
+
|
|
|
private Server server;
|
|
|
private InetSocketAddress masterServiceAddress;
|
|
|
private AccessControlList adminAcl;
|
|
@@ -74,23 +85,21 @@ public class AdminService extends AbstractService implements ResourceManagerAdmi
|
|
|
private final RecordFactory recordFactory =
|
|
|
RecordFactoryProvider.getRecordFactory(null);
|
|
|
|
|
|
- public AdminService(Configuration conf, ResourceScheduler scheduler,
|
|
|
- RMContext rmContext, NodesListManager nodesListManager,
|
|
|
- ClientRMService clientRMService,
|
|
|
- ApplicationMasterService applicationMasterService,
|
|
|
- ResourceTrackerService resourceTrackerService) {
|
|
|
+ public AdminService(ResourceManager rm, RMContext rmContext) {
|
|
|
super(AdminService.class.getName());
|
|
|
- this.conf = conf;
|
|
|
- this.scheduler = scheduler;
|
|
|
+ this.rm = rm;
|
|
|
this.rmContext = rmContext;
|
|
|
- this.nodesListManager = nodesListManager;
|
|
|
- this.clientRMService = clientRMService;
|
|
|
- this.applicationMasterService = applicationMasterService;
|
|
|
- this.resourceTrackerService = resourceTrackerService;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void serviceInit(Configuration conf) throws Exception {
|
|
|
+ public synchronized void serviceInit(Configuration conf) throws Exception {
|
|
|
+ haEnabled = HAUtil.isHAEnabled(conf);
|
|
|
+ if (haEnabled) {
|
|
|
+ HAUtil.verifyAndSetConfiguration(conf);
|
|
|
+ rm.setConf(conf);
|
|
|
+ }
|
|
|
+ rm.createAndInitActiveServices();
|
|
|
+
|
|
|
masterServiceAddress = conf.getSocketAddr(
|
|
|
YarnConfiguration.RM_ADMIN_ADDRESS,
|
|
|
YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
|
|
@@ -102,50 +111,185 @@ public class AdminService extends AbstractService implements ResourceManagerAdmi
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- protected void serviceStart() throws Exception {
|
|
|
+ protected synchronized void serviceStart() throws Exception {
|
|
|
+ if (haEnabled) {
|
|
|
+ transitionToStandby(true);
|
|
|
+ } else {
|
|
|
+ transitionToActive();
|
|
|
+ }
|
|
|
+ startServer();
|
|
|
+ super.serviceStart();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected synchronized void serviceStop() throws Exception {
|
|
|
+ stopServer();
|
|
|
+ transitionToStandby(false);
|
|
|
+ haState = HAServiceState.STOPPING;
|
|
|
+ super.serviceStop();
|
|
|
+ }
|
|
|
+
|
|
|
+ protected void startServer() throws Exception {
|
|
|
Configuration conf = getConfig();
|
|
|
YarnRPC rpc = YarnRPC.create(conf);
|
|
|
- this.server =
|
|
|
- rpc.getServer(ResourceManagerAdministrationProtocol.class, this, masterServiceAddress,
|
|
|
- conf, null,
|
|
|
- conf.getInt(YarnConfiguration.RM_ADMIN_CLIENT_THREAD_COUNT,
|
|
|
- YarnConfiguration.DEFAULT_RM_ADMIN_CLIENT_THREAD_COUNT));
|
|
|
-
|
|
|
+ this.server = (Server) rpc.getServer(
|
|
|
+ ResourceManagerAdministrationProtocol.class, this, masterServiceAddress,
|
|
|
+ conf, null,
|
|
|
+ conf.getInt(YarnConfiguration.RM_ADMIN_CLIENT_THREAD_COUNT,
|
|
|
+ YarnConfiguration.DEFAULT_RM_ADMIN_CLIENT_THREAD_COUNT));
|
|
|
+
|
|
|
// Enable service authorization?
|
|
|
if (conf.getBoolean(
|
|
|
- CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
|
|
|
+ CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
|
|
|
false)) {
|
|
|
refreshServiceAcls(conf, new RMPolicyProvider());
|
|
|
}
|
|
|
|
|
|
+ if (haEnabled) {
|
|
|
+ RPC.setProtocolEngine(conf, HAServiceProtocolPB.class,
|
|
|
+ ProtobufRpcEngine.class);
|
|
|
+
|
|
|
+ HAServiceProtocolServerSideTranslatorPB haServiceProtocolXlator =
|
|
|
+ new HAServiceProtocolServerSideTranslatorPB(this);
|
|
|
+ BlockingService haPbService =
|
|
|
+ HAServiceProtocolProtos.HAServiceProtocolService
|
|
|
+ .newReflectiveBlockingService(haServiceProtocolXlator);
|
|
|
+ server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
|
|
|
+ HAServiceProtocol.class, haPbService);
|
|
|
+ }
|
|
|
+
|
|
|
this.server.start();
|
|
|
conf.updateConnectAddr(YarnConfiguration.RM_ADMIN_ADDRESS,
|
|
|
- server.getListenerAddress());
|
|
|
- super.serviceStart();
|
|
|
+ server.getListenerAddress());
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- protected void serviceStop() throws Exception {
|
|
|
+ protected void stopServer() throws Exception {
|
|
|
if (this.server != null) {
|
|
|
this.server.stop();
|
|
|
}
|
|
|
- super.serviceStop();
|
|
|
+ }
|
|
|
+
|
|
|
+ private UserGroupInformation checkAccess(String method) throws IOException {
|
|
|
+ return RMServerUtils.verifyAccess(adminAcl, method, LOG);
|
|
|
}
|
|
|
|
|
|
private UserGroupInformation checkAcls(String method) throws YarnException {
|
|
|
try {
|
|
|
- return RMServerUtils.verifyAccess(adminAcl, method, LOG);
|
|
|
+ return checkAccess(method);
|
|
|
} catch (IOException ioe) {
|
|
|
throw RPCUtil.getRemoteException(ioe);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ private synchronized boolean isRMActive() {
|
|
|
+ return HAServiceState.ACTIVE == haState;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void monitorHealth()
|
|
|
+ throws IOException {
|
|
|
+ checkAccess("monitorHealth");
|
|
|
+ if (haState == HAServiceProtocol.HAServiceState.ACTIVE && !rm.areActiveServicesRunning()) {
|
|
|
+ throw new HealthCheckFailedException(
|
|
|
+ "Active ResourceManager services are not running!");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ synchronized void transitionToActive() throws Exception {
|
|
|
+ if (haState == HAServiceProtocol.HAServiceState.ACTIVE) {
|
|
|
+ LOG.info("Already in active state");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ LOG.info("Transitioning to active");
|
|
|
+ rm.startActiveServices();
|
|
|
+ haState = HAServiceProtocol.HAServiceState.ACTIVE;
|
|
|
+ LOG.info("Transitioned to active");
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void transitionToActive(HAServiceProtocol.StateChangeRequestInfo reqInfo)
|
|
|
+ throws IOException {
|
|
|
+ UserGroupInformation user = checkAccess("transitionToActive");
|
|
|
+ // TODO (YARN-1177): When automatic failover is enabled,
|
|
|
+ // check if transition should be allowed for this request
|
|
|
+ try {
|
|
|
+ transitionToActive();
|
|
|
+ RMAuditLogger.logSuccess(user.getShortUserName(),
|
|
|
+ "transitionToActive", "RMHAProtocolService");
|
|
|
+ } catch (Exception e) {
|
|
|
+ RMAuditLogger.logFailure(user.getShortUserName(), "transitionToActive",
|
|
|
+ adminAcl.toString(), "RMHAProtocolService",
|
|
|
+ "Exception transitioning to active");
|
|
|
+ throw new ServiceFailedException(
|
|
|
+ "Error when transitioning to Active mode", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ synchronized void transitionToStandby(boolean initialize)
|
|
|
+ throws Exception {
|
|
|
+ if (haState == HAServiceProtocol.HAServiceState.STANDBY) {
|
|
|
+ LOG.info("Already in standby state");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ LOG.info("Transitioning to standby");
|
|
|
+ if (haState == HAServiceProtocol.HAServiceState.ACTIVE) {
|
|
|
+ rm.stopActiveServices();
|
|
|
+ if (initialize) {
|
|
|
+ rm.createAndInitActiveServices();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ haState = HAServiceProtocol.HAServiceState.STANDBY;
|
|
|
+ LOG.info("Transitioned to standby");
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void transitionToStandby(HAServiceProtocol.StateChangeRequestInfo reqInfo)
|
|
|
+ throws IOException {
|
|
|
+ UserGroupInformation user = checkAccess("transitionToStandby");
|
|
|
+ // TODO (YARN-1177): When automatic failover is enabled,
|
|
|
+ // check if transition should be allowed for this request
|
|
|
+ try {
|
|
|
+ transitionToStandby(true);
|
|
|
+ RMAuditLogger.logSuccess(user.getShortUserName(),
|
|
|
+ "transitionToStandby", "RMHAProtocolService");
|
|
|
+ } catch (Exception e) {
|
|
|
+ RMAuditLogger.logFailure(user.getShortUserName(), "transitionToStandby",
|
|
|
+ adminAcl.toString(), "RMHAProtocolService",
|
|
|
+ "Exception transitioning to standby");
|
|
|
+ throw new ServiceFailedException(
|
|
|
+ "Error when transitioning to Standby mode", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized HAServiceStatus getServiceStatus() throws IOException {
|
|
|
+ checkAccess("getServiceState");
|
|
|
+ HAServiceStatus ret = new HAServiceStatus(haState);
|
|
|
+ if (haState == HAServiceProtocol.HAServiceState.ACTIVE || haState ==
|
|
|
+ HAServiceProtocol.HAServiceState.STANDBY) {
|
|
|
+ ret.setReadyToBecomeActive();
|
|
|
+ } else {
|
|
|
+ ret.setNotReadyToBecomeActive("State is " + haState);
|
|
|
+ }
|
|
|
+ return ret;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
|
|
|
throws YarnException {
|
|
|
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();
|
|
|
+ }
|
|
|
+
|
|
|
try {
|
|
|
- scheduler.reinitialize(conf, this.rmContext);
|
|
|
+ rmContext.getScheduler().reinitialize(getConfig(), this.rmContext);
|
|
|
RMAuditLogger.logSuccess(user.getShortUserName(), "refreshQueues",
|
|
|
"AdminService");
|
|
|
return recordFactory.newRecordInstance(RefreshQueuesResponse.class);
|
|
@@ -162,8 +306,16 @@ public class AdminService extends AbstractService implements ResourceManagerAdmi
|
|
|
public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
|
|
|
throws YarnException {
|
|
|
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();
|
|
|
+ }
|
|
|
+
|
|
|
try {
|
|
|
- this.nodesListManager.refreshNodes(new YarnConfiguration());
|
|
|
+ rmContext.getNodesListManager().refreshNodes(new YarnConfiguration());
|
|
|
RMAuditLogger.logSuccess(user.getShortUserName(), "refreshNodes",
|
|
|
"AdminService");
|
|
|
return recordFactory.newRecordInstance(RefreshNodesResponse.class);
|
|
@@ -180,7 +332,16 @@ public class AdminService extends AbstractService implements ResourceManagerAdmi
|
|
|
RefreshSuperUserGroupsConfigurationRequest request)
|
|
|
throws YarnException {
|
|
|
UserGroupInformation user = checkAcls("refreshSuperUserGroupsConfiguration");
|
|
|
-
|
|
|
+
|
|
|
+ // TODO (YARN-1459): Revisit handling super-user-groups on Standby RM
|
|
|
+ if (!isRMActive()) {
|
|
|
+ RMAuditLogger.logFailure(user.getShortUserName(),
|
|
|
+ "refreshSuperUserGroupsConfiguration",
|
|
|
+ adminAcl.toString(), "AdminService",
|
|
|
+ "ResourceManager is not active. Can not refresh super-user-groups.");
|
|
|
+ throw new RMNotYetActiveException();
|
|
|
+ }
|
|
|
+
|
|
|
ProxyUsers.refreshSuperUserGroupsConfiguration(new Configuration());
|
|
|
RMAuditLogger.logSuccess(user.getShortUserName(),
|
|
|
"refreshSuperUserGroupsConfiguration", "AdminService");
|
|
@@ -193,7 +354,16 @@ public class AdminService extends AbstractService implements ResourceManagerAdmi
|
|
|
public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
|
|
|
RefreshUserToGroupsMappingsRequest request) throws YarnException {
|
|
|
UserGroupInformation user = checkAcls("refreshUserToGroupsMappings");
|
|
|
-
|
|
|
+
|
|
|
+ // TODO (YARN-1459): Revisit handling user-groups on Standby RM
|
|
|
+ if (!isRMActive()) {
|
|
|
+ RMAuditLogger.logFailure(user.getShortUserName(),
|
|
|
+ "refreshUserToGroupsMapping",
|
|
|
+ adminAcl.toString(), "AdminService",
|
|
|
+ "ResourceManager is not active. Can not refresh user-groups.");
|
|
|
+ throw new RMNotYetActiveException();
|
|
|
+ }
|
|
|
+
|
|
|
Groups.getUserToGroupsMappingService().refresh();
|
|
|
RMAuditLogger.logSuccess(user.getShortUserName(),
|
|
|
"refreshUserToGroupsMappings", "AdminService");
|
|
@@ -233,9 +403,16 @@ public class AdminService extends AbstractService implements ResourceManagerAdmi
|
|
|
PolicyProvider policyProvider = new RMPolicyProvider();
|
|
|
|
|
|
refreshServiceAcls(conf, policyProvider);
|
|
|
- clientRMService.refreshServiceAcls(conf, policyProvider);
|
|
|
- applicationMasterService.refreshServiceAcls(conf, policyProvider);
|
|
|
- resourceTrackerService.refreshServiceAcls(conf, policyProvider);
|
|
|
+ if (isRMActive()) {
|
|
|
+ rmContext.getClientRMService().refreshServiceAcls(conf, policyProvider);
|
|
|
+ rmContext.getApplicationMasterService().refreshServiceAcls(
|
|
|
+ conf, policyProvider);
|
|
|
+ rmContext.getResourceTrackerService().refreshServiceAcls(
|
|
|
+ conf, policyProvider);
|
|
|
+ } else {
|
|
|
+ LOG.warn("ResourceManager is not active. Not refreshing ACLs for " +
|
|
|
+ "Clients, ApplicationMasters and NodeManagers");
|
|
|
+ }
|
|
|
|
|
|
return recordFactory.newRecordInstance(RefreshServiceAclsResponse.class);
|
|
|
}
|
|
@@ -249,5 +426,4 @@ public class AdminService extends AbstractService implements ResourceManagerAdmi
|
|
|
public String[] getGroupsForUser(String user) throws IOException {
|
|
|
return UserGroupInformation.createRemoteUser(user).getGroupNames();
|
|
|
}
|
|
|
-
|
|
|
}
|