From d19cfe01642f9582e1fe5d567beb480399c37a01 Mon Sep 17 00:00:00 2001 From: Mahadev Konar Date: Tue, 25 Oct 2011 06:07:13 +0000 Subject: [PATCH] MAPREDUCE-2746. Yarn servers can't communicate with each other with hadoop.security.authorization set to true (acmurthy via mahadev) git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1188522 13f79535-47bb-0310-9956-ffa450edef68 --- .../hadoop/fs/CommonConfigurationKeys.java | 13 +++ .../java/org/apache/hadoop/ipc/Server.java | 10 +++ .../apache/hadoop/security/SecurityUtil.java | 52 +++++++----- .../apache/hadoop/security/token/Token.java | 10 ++- .../src/main/packages/hadoop-setup-hdfs.sh | 5 ++ .../packages/templates/conf/hadoop-policy.xml | 83 ++++++++++++++++++- .../hadoop/hdfs/HDFSPolicyProvider.java | 16 ++-- hadoop-mapreduce-project/CHANGES.txt | 3 + .../mapred/TaskAttemptListenerImpl.java | 19 ++++- .../v2/app/client/MRClientService.java | 23 +++-- .../app/launcher/ContainerLauncherImpl.java | 3 - .../authorize/MRAMPolicyProvider.java | 50 +++++++++++ .../org/apache/hadoop/mapreduce/Cluster.java | 56 +++++++------ .../apache/hadoop/mapreduce/MRJobConfig.java | 10 +++ .../hadoop/mapreduce/util/ConfigUtil.java | 9 ++ .../mapreduce/v2/hs/HistoryClientService.java | 14 +++- .../hadoop/mapred/TestClientRedirect.java | 4 +- .../hadoop-yarn/conf/yarn-env.sh | 3 + .../hadoop/yarn/conf/YarnConfiguration.java | 23 +++++ .../yarn/factories/RpcServerFactory.java | 2 +- .../impl/pb/RpcServerFactoryPBImpl.java | 2 +- .../apache/hadoop/yarn/ipc/AvroYarnRPC.java | 63 -------------- .../hadoop/yarn/ipc/HadoopYarnProtoRPC.java | 29 +------ .../apache/hadoop/yarn/ipc/HadoopYarnRPC.java | 27 +----- .../org/apache/hadoop/yarn/ipc/YarnRPC.java | 2 +- .../java/org/apache/hadoop/yarn/TestRPC.java | 4 +- .../yarn/conf/TestYarnConfiguration.java | 3 - .../ContainerManagerImpl.java | 20 ++++- .../ResourceLocalizationService.java | 19 +++-- .../security/authorize/NMPolicyProvider.java | 49 +++++++++++ .../impl/pb/TestPBLocalizerRPC.java | 4 +- .../TestResourceLocalizationService.java | 2 +- .../server/resourcemanager/AdminService.java | 60 +++++++++++++- .../ApplicationMasterService.java | 22 ++++- .../resourcemanager/ClientRMService.java | 21 ++++- .../resourcemanager/ResourceManager.java | 10 ++- .../ResourceTrackerService.java | 21 ++++- .../resourcemanager/api/RMAdminProtocol.java | 6 ++ .../client/RMAdminProtocolPBClientImpl.java | 24 ++++++ .../service/RMAdminProtocolPBServiceImpl.java | 20 +++++ .../RefreshServiceAclsRequest.java | 23 +++++ .../RefreshServiceAclsResponse.java | 23 +++++ .../pb/RefreshServiceAclsRequestPBImpl.java | 49 +++++++++++ .../pb/RefreshServiceAclsResponsePBImpl.java | 49 +++++++++++ .../security/admin/AdminSecurityInfo.java | 2 +- .../security/authorize/RMPolicyProvider.java | 62 ++++++++++++++ .../server/resourcemanager/tools/RMAdmin.java | 31 ++++++- .../src/main/proto/RMAdminProtocol.proto | 1 + ...erver_resourcemanager_service_protos.proto | 6 ++ .../org.apache.hadoop.security.SecurityInfo | 2 +- .../yarn/server/resourcemanager/MockRM.java | 10 ++- .../mapred/MapReducePolicyProvider.java | 16 ++-- 52 files changed, 856 insertions(+), 234 deletions(-) create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/security/authorize/MRAMPolicyProvider.java delete mode 100644 hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/AvroYarnRPC.java create mode 100644 hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/authorize/NMPolicyProvider.java create mode 100644 hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/RefreshServiceAclsRequest.java create mode 100644 hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/RefreshServiceAclsResponse.java create mode 100644 hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/impl/pb/RefreshServiceAclsRequestPBImpl.java create mode 100644 hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/impl/pb/RefreshServiceAclsResponsePBImpl.java rename hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/{ => server/resourcemanager}/security/admin/AdminSecurityInfo.java (96%) create mode 100644 hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/authorize/RMPolicyProvider.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index faea3d1946e..aaac349cd2b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -93,5 +93,18 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { /** Default value for IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY */ public static final int IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_DEFAULT = 256 * 1024; + + /** + * Service Authorization + */ + public static final String + HADOOP_SECURITY_SERVICE_AUTHORIZATION_REFRESH_POLICY = + "security.refresh.policy.protocol.acl"; + public static final String + HADOOP_SECURITY_SERVICE_AUTHORIZATION_GET_USER_MAPPINGS = + "security.get.user.mappings.protocol.acl"; + public static final String + HADOOP_SECURITY_SERVICE_AUTHORIZATION_REFRESH_USER_MAPPINGS = + "security.refresh.user.mappings.protocol.acl"; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index ddc761b5adf..78c0bc76a10 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -1810,6 +1810,16 @@ public abstract class Server { } } + /** + * Get the port on which the IPC Server is listening for incoming connections. + * This could be an ephemeral port too, in which case we return the real + * port on which the Server has bound. + * @return port on which IPC Server is listening + */ + public int getPort() { + return port; + } + /** * The number of open RPC conections * @return the number of open rpc connections diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java index 58f8bde4662..81a6112c9fb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java @@ -316,17 +316,23 @@ public class SecurityUtil { * @param conf configuration object * @return the KerberosInfo or null if it has no KerberosInfo defined */ - public static KerberosInfo getKerberosInfo(Class protocol, Configuration conf) { - for(SecurityInfo provider: testProviders) { - KerberosInfo result = provider.getKerberosInfo(protocol, conf); - if (result != null) { - return result; + public static KerberosInfo + getKerberosInfo(Class protocol, Configuration conf) { + synchronized (testProviders) { + for(SecurityInfo provider: testProviders) { + KerberosInfo result = provider.getKerberosInfo(protocol, conf); + if (result != null) { + return result; + } } } - for(SecurityInfo provider: securityInfoProviders) { - KerberosInfo result = provider.getKerberosInfo(protocol, conf); - if (result != null) { - return result; + + synchronized (securityInfoProviders) { + for(SecurityInfo provider: securityInfoProviders) { + KerberosInfo result = provider.getKerberosInfo(protocol, conf); + if (result != null) { + return result; + } } } return null; @@ -340,18 +346,24 @@ public class SecurityUtil { * @return the TokenInfo or null if it has no KerberosInfo defined */ public static TokenInfo getTokenInfo(Class protocol, Configuration conf) { - for(SecurityInfo provider: testProviders) { - TokenInfo result = provider.getTokenInfo(protocol, conf); - if (result != null) { - return result; - } - } - for(SecurityInfo provider: securityInfoProviders) { - TokenInfo result = provider.getTokenInfo(protocol, conf); - if (result != null) { - return result; + synchronized (testProviders) { + for(SecurityInfo provider: testProviders) { + TokenInfo result = provider.getTokenInfo(protocol, conf); + if (result != null) { + return result; + } } - } + } + + synchronized (securityInfoProviders) { + for(SecurityInfo provider: securityInfoProviders) { + TokenInfo result = provider.getTokenInfo(protocol, conf); + if (result != null) { + return result; + } + } + } + return null; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java index 32383da9a00..89652fc1fa8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java @@ -281,10 +281,12 @@ public class Token implements Writable { return renewer; } renewer = TRIVIAL_RENEWER; - for (TokenRenewer canidate: renewers) { - if (canidate.handleKind(this.kind)) { - renewer = canidate; - return renewer; + synchronized (renewers) { + for (TokenRenewer canidate : renewers) { + if (canidate.handleKind(this.kind)) { + renewer = canidate; + return renewer; + } } } LOG.warn("No TokenRenewer defined for token kind " + this.kind); diff --git a/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-hdfs.sh b/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-hdfs.sh index cd99463b370..fc4a7325c27 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-hdfs.sh +++ b/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-hdfs.sh @@ -70,6 +70,10 @@ while true ; do HADOOP_MR_USER=$2; shift 2 AUTOMATED=1 ;; + --yarn-user) + HADOOP_YARN_USER=$2; shift 2 + AUTOMATED=1 + ;; --hdfs-user-keytab) HDFS_KEYTAB=$2; shift 2 AUTOMATED=1 @@ -91,6 +95,7 @@ done HADOOP_GROUP=${HADOOP_GROUP:-hadoop} HADOOP_HDFS_USER=${HADOOP_HDFS_USER:-hdfs} +HADOOP_YARN_USER=${HADOOP_YARN_USER:-yarn} HADOOP_MAPREDUCE_USER=${HADOOP_MR_USER:-mapred} if [ "${KERBEROS_REALM}" != "" ]; then diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml index 7708f01aeca..8ac761e58e5 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml +++ b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml @@ -85,6 +85,7 @@ A special value of "*" means all users are allowed. + security.job.submission.protocol.acl * @@ -124,7 +125,7 @@ users are allowed. - + security.refresh.policy.protocol.acl ${HADOOP_HDFS_USER} ACL for RefreshAuthorizationPolicyProtocol, used by the @@ -135,5 +136,85 @@ + + + + security.resourcetracker.protocol.acl + ${HADOOP_YARN_USER} + ACL for ResourceTracker protocol, used by the + ResourceManager and NodeManager to communicate with each other. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.admin.protocol.acl + ${HADOOP_YARN_USER} + ACL for RMAdminProtocol, for admin commands. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.client.resourcemanager.protocol.acl + * + ACL for ClientRMProtocol, used by the ResourceManager + and applications submission clients to communicate with each other. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.applicationmaster.resourcemanager.protocol.acl + * + ACL for AMRMProtocol, used by the ResourceManager + and ApplicationMasters to communicate with each other. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.containermanager.protocol.acl + * + ACL for ContainerManager protocol, used by the NodeManager + and ApplicationMasters to communicate with each other. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.resourcelocalizer.protocol.acl + * + ACL for ResourceLocalizer protocol, used by the NodeManager + and ResourceLocalizer to communicate with each other. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.job.task.protocol.acl + * + ACL for TaskUmbilicalProtocol, used by the map and reduce + tasks to communicate with the parent tasktracker. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.job.client.protocol.acl + * + ACL for MRClientProtocol, used by job clients to + communciate with the MR ApplicationMaster to query job status etc. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java index edfc41ffc20..27702b5795b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol; @@ -43,12 +44,15 @@ public class HDFSPolicyProvider extends PolicyProvider { new Service("security.inter.datanode.protocol.acl", InterDatanodeProtocol.class), new Service("security.namenode.protocol.acl", NamenodeProtocol.class), - new Service("security.refresh.policy.protocol.acl", - RefreshAuthorizationPolicyProtocol.class), - new Service("security.refresh.user.mappings.protocol.acl", - RefreshUserMappingsProtocol.class), - new Service("security.get.user.mappings.protocol.acl", - GetUserMappingsProtocol.class) + new Service( + CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_REFRESH_POLICY, + RefreshAuthorizationPolicyProtocol.class), + new Service( + CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_REFRESH_USER_MAPPINGS, + RefreshUserMappingsProtocol.class), + new Service( + CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_GET_USER_MAPPINGS, + GetUserMappingsProtocol.class) }; @Override diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index 3edaf7ca9c6..41d048962d8 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -1758,6 +1758,9 @@ Release 0.23.0 - Unreleased MAPREDUCE-3248. Fixed log4j properties. (vinodkv via acmurthy) + MAPREDUCE-2746. Yarn servers can't communicate with each other with + hadoop.security.authorization set to true (acmurthy via mahadev) + Release 0.22.0 - Unreleased INCOMPATIBLE CHANGES diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java index ccaa773227d..66526445a9b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java @@ -29,10 +29,10 @@ import java.util.Map; 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.ProtocolSignature; import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.ipc.RPC.Server; -import org.apache.hadoop.ipc.VersionedProtocol; +import org.apache.hadoop.ipc.Server; import org.apache.hadoop.mapred.SortedRanges.Range; import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.TypeConverter; @@ -48,7 +48,9 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus; +import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider; import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.authorize.PolicyProvider; import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.service.CompositeService; @@ -107,6 +109,14 @@ public class TaskAttemptListenerImpl extends CompositeService conf.getInt(MRJobConfig.MR_AM_TASK_LISTENER_THREAD_COUNT, MRJobConfig.DEFAULT_MR_AM_TASK_LISTENER_THREAD_COUNT), false, conf, jobTokenSecretManager); + + // Enable service authorization? + if (conf.getBoolean( + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, + false)) { + refreshServiceAcls(conf, new MRAMPolicyProvider()); + } + server.start(); InetSocketAddress listenerAddress = server.getListenerAddress(); this.address = @@ -118,6 +128,11 @@ public class TaskAttemptListenerImpl extends CompositeService } } + void refreshServiceAcls(Configuration configuration, + PolicyProvider policyProvider) { + this.server.refreshServiceAcl(configuration, policyProvider); + } + @Override public void stop() { stopRpcServer(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java index faf11a117cc..c7300041afa 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java @@ -26,7 +26,8 @@ import java.security.AccessControlException; import java.util.Arrays; import java.util.Collection; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.ipc.Server; import org.apache.commons.codec.binary.Base64; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -72,20 +73,19 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType; +import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider; import org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authorize.PolicyProvider; import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnRemoteException; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier; -import org.apache.hadoop.yarn.security.SchedulerSecurityInfo; import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager; import org.apache.hadoop.yarn.service.AbstractService; import org.apache.hadoop.yarn.webapp.WebApp; @@ -140,6 +140,14 @@ public class MRClientService extends AbstractService conf, secretManager, conf.getInt(MRJobConfig.MR_AM_JOB_CLIENT_THREAD_COUNT, MRJobConfig.DEFAULT_MR_AM_JOB_CLIENT_THREAD_COUNT)); + + // Enable service authorization? + if (conf.getBoolean( + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, + false)) { + refreshServiceAcls(conf, new MRAMPolicyProvider()); + } + server.start(); this.bindAddress = NetUtils.createSocketAddr(hostNameResolved.getHostAddress() @@ -154,8 +162,13 @@ public class MRClientService extends AbstractService super.start(); } + void refreshServiceAcls(Configuration configuration, + PolicyProvider policyProvider) { + this.server.refreshServiceAcl(configuration, policyProvider); + } + public void stop() { - server.close(); + server.stop(); if (webApp != null) { webApp.stop(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java index 9a5cd1b5ae7..dfb5f4fd001 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java @@ -44,7 +44,6 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType; import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator; import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.StringUtils; @@ -56,11 +55,9 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerToken; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.ipc.YarnRPC; -import org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.service.AbstractService; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/security/authorize/MRAMPolicyProvider.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/security/authorize/MRAMPolicyProvider.java new file mode 100644 index 00000000000..3f6ecb4386b --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/security/authorize/MRAMPolicyProvider.java @@ -0,0 +1,50 @@ +/** + * 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.mapreduce.v2.app.security.authorize; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.mapred.TaskUmbilicalProtocol; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.security.authorize.PolicyProvider; +import org.apache.hadoop.security.authorize.Service; +import org.apache.hadoop.yarn.proto.MRClientProtocol; + +/** + * {@link PolicyProvider} for YARN MapReduce protocols. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class MRAMPolicyProvider extends PolicyProvider { + + private static final Service[] mapReduceApplicationMasterServices = + new Service[] { + new Service( + MRJobConfig.MR_AM_SECURITY_SERVICE_AUTHORIZATION_TASK_UMBILICAL, + TaskUmbilicalProtocol.class), + new Service( + MRJobConfig.MR_AM_SECURITY_SERVICE_AUTHORIZATION_CLIENT, + MRClientProtocol.MRClientProtocolService.BlockingInterface.class) + }; + + @Override + public Service[] getServices() { + return mapReduceApplicationMasterServices; + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java index 94d1fc78f02..2a198b49550 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java @@ -66,6 +66,9 @@ public class Cluster { private Path jobHistoryDir = null; private static final Log LOG = LogFactory.getLog(Cluster.class); + private static ServiceLoader frameworkLoader = + ServiceLoader.load(ClientProtocolProvider.class); + static { ConfigUtil.loadResources(); } @@ -84,33 +87,34 @@ public class Cluster { private void initialize(InetSocketAddress jobTrackAddr, Configuration conf) throws IOException { - for (ClientProtocolProvider provider : ServiceLoader - .load(ClientProtocolProvider.class)) { - LOG.debug("Trying ClientProtocolProvider : " - + provider.getClass().getName()); - ClientProtocol clientProtocol = null; - try { - if (jobTrackAddr == null) { - clientProtocol = provider.create(conf); - } else { - clientProtocol = provider.create(jobTrackAddr, conf); + synchronized (frameworkLoader) { + for (ClientProtocolProvider provider : frameworkLoader) { + LOG.debug("Trying ClientProtocolProvider : " + + provider.getClass().getName()); + ClientProtocol clientProtocol = null; + try { + if (jobTrackAddr == null) { + clientProtocol = provider.create(conf); + } else { + clientProtocol = provider.create(jobTrackAddr, conf); + } + + if (clientProtocol != null) { + clientProtocolProvider = provider; + client = clientProtocol; + LOG.debug("Picked " + provider.getClass().getName() + + " as the ClientProtocolProvider"); + break; + } + else { + LOG.info("Cannot pick " + provider.getClass().getName() + + " as the ClientProtocolProvider - returned null protocol"); + } + } + catch (Exception e) { + LOG.info("Failed to use " + provider.getClass().getName() + + " due to error: " + e.getMessage()); } - - if (clientProtocol != null) { - clientProtocolProvider = provider; - client = clientProtocol; - LOG.debug("Picked " + provider.getClass().getName() - + " as the ClientProtocolProvider"); - break; - } - else { - LOG.info("Cannot pick " + provider.getClass().getName() - + " as the ClientProtocolProvider - returned null protocol"); - } - } - catch (Exception e) { - LOG.info("Failed to use " + provider.getClass().getName() - + " due to error: " + e.getMessage()); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java index 142807f2271..ea2c63f5781 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java @@ -499,4 +499,14 @@ public interface MRJobConfig { public static final String MR_JOB_END_NOTIFICATION_MAX_RETRY_INTERVAL = "mapreduce.job.end-notification.max.retry.interval"; + /* + * MR AM Service Authorization + */ + public static final String + MR_AM_SECURITY_SERVICE_AUTHORIZATION_TASK_UMBILICAL = + "security.job.task.protocol.acl"; + public static final String + MR_AM_SECURITY_SERVICE_AUTHORIZATION_CLIENT = + "security.job.client.protocol.acl"; + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ConfigUtil.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ConfigUtil.java index 51ecc34b2c1..a7bd19ec148 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ConfigUtil.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ConfigUtil.java @@ -514,6 +514,15 @@ public class ConfigUtil { Configuration.addDeprecation("webinterface.private.actions", new String[]{JTConfig.PRIVATE_ACTIONS_KEY}); + + Configuration.addDeprecation("security.task.umbilical.protocol.acl", + new String[] { + MRJobConfig.MR_AM_SECURITY_SERVICE_AUTHORIZATION_TASK_UMBILICAL + }); + Configuration.addDeprecation("security.job.submission.protocol.acl", + new String[] { + MRJobConfig.MR_AM_SECURITY_SERVICE_AUTHORIZATION_CLIENT + }); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java index 6d5de7ecedb..5e7740d4685 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java @@ -27,11 +27,12 @@ import java.security.PrivilegedExceptionAction; import java.util.Arrays; import java.util.Collection; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.ipc.Server; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.mapreduce.JobACL; import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol; import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest; @@ -62,6 +63,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskId; import org.apache.hadoop.mapreduce.v2.api.records.TaskType; import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.Task; +import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider; import org.apache.hadoop.mapreduce.v2.hs.webapp.HsWebApp; import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; import org.apache.hadoop.mapreduce.v2.security.client.ClientHSSecurityInfo; @@ -121,6 +123,14 @@ public class HistoryClientService extends AbstractService { conf, null, conf.getInt(JHAdminConfig.MR_HISTORY_CLIENT_THREAD_COUNT, JHAdminConfig.DEFAULT_MR_HISTORY_CLIENT_THREAD_COUNT)); + + // Enable service authorization? + if (conf.getBoolean( + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, + false)) { + server.refreshServiceAcl(conf, new MRAMPolicyProvider()); + } + server.start(); this.bindAddress = NetUtils.createSocketAddr(hostNameResolved.getHostAddress() @@ -140,7 +150,7 @@ public class HistoryClientService extends AbstractService { @Override public void stop() { if (server != null) { - server.close(); + server.stop(); } if (webApp != null) { webApp.stop(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java index 5d839252eac..3ea3683497c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java @@ -25,7 +25,7 @@ import java.util.Iterator; import junit.framework.Assert; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.ipc.Server; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -379,7 +379,7 @@ public class TestClientRedirect { } public void stop() { - server.close(); + server.stop(); super.stop(); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/conf/yarn-env.sh b/hadoop-mapreduce-project/hadoop-yarn/conf/yarn-env.sh index e4fb7eb0142..b219eddf1a3 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/conf/yarn-env.sh +++ b/hadoop-mapreduce-project/hadoop-yarn/conf/yarn-env.sh @@ -13,6 +13,9 @@ # See the License for the specific language governing permissions and # limitations under the License. +# User for YARN daemons +export HADOOP_YARN_USER=${HADOOP_YARN_USER:-yarn} + # resolve links - $0 may be a softlink export YARN_CONF_DIR="${YARN_CONF_DIR:-$YARN_HOME/conf}" diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index 2c7af03ec7d..a51e93abd0a 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -372,6 +372,29 @@ public class YarnConfiguration extends Configuration { public static final int INVALID_CONTAINER_EXIT_STATUS = -1000; public static final int ABORTED_CONTAINER_EXIT_STATUS = -100; + /** + * YARN Service Level Authorization + */ + public static final String + YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER = + "security.resourcetracker.protocol.acl"; + public static final String + YARN_SECURITY_SERVICE_AUTHORIZATION_CLIENT_RESOURCEMANAGER = + "security.client.resourcemanager.protocol.acl"; + public static final String + YARN_SECURITY_SERVICE_AUTHORIZATION_ADMIN = + "security.admin.protocol.acl"; + public static final String + YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONMASTER_RESOURCEMANAGER = + "security.applicationmaster.resourcemanager.protocol.acl"; + + public static final String + YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGER = + "security.containermanager.protocol.acl"; + public static final String + YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER = + "security.resourcelocalizer.protocol.acl"; + public YarnConfiguration() { super(); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/RpcServerFactory.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/RpcServerFactory.java index 512f48f6ed4..28576d7628a 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/RpcServerFactory.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/RpcServerFactory.java @@ -21,7 +21,7 @@ package org.apache.hadoop.yarn.factories; import java.net.InetSocketAddress; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.ipc.RPC.Server; +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; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcServerFactoryPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcServerFactoryPBImpl.java index afb3d7e75ec..f1f28921628 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcServerFactoryPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcServerFactoryPBImpl.java @@ -27,8 +27,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.Server; import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.ipc.RPC.Server; import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.yarn.YarnException; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/AvroYarnRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/AvroYarnRPC.java deleted file mode 100644 index e72ff4dc1ba..00000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/AvroYarnRPC.java +++ /dev/null @@ -1,63 +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.ipc; - -import java.io.IOException; -import java.net.InetSocketAddress; - -import org.apache.avro.ipc.Server; -import org.apache.avro.ipc.SocketServer; -import org.apache.avro.ipc.SocketTransceiver; -import org.apache.avro.ipc.specific.SpecificRequestor; -import org.apache.avro.ipc.specific.SpecificResponder; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.token.SecretManager; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.yarn.YarnException; - -/* - * This uses Avro's simple Socket based RPC. Can be replaced with Netty based - * when Yarn is upgraded to Avro 1.4. - */ -public class AvroYarnRPC extends YarnRPC { - - @Override - public Object getProxy(Class protocol, - InetSocketAddress addr, Configuration conf) { - try { - return SpecificRequestor.getClient(protocol, new SocketTransceiver(addr)); - } catch (IOException e) { - throw new YarnException(e); - } - } - - @Override - public Server getServer(Class protocol, Object instance, - InetSocketAddress addr, Configuration conf, - SecretManager secretManager, - int numHandlers) { - try { - return new SocketServer(new SpecificResponder(protocol, instance), - addr); - } catch (IOException e) { - throw new YarnException(e); - } - } - -} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnProtoRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnProtoRPC.java index 885682111e3..b40ecad235b 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnProtoRPC.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnProtoRPC.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.ipc; import java.net.InetSocketAddress; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.ipc.Server; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -58,34 +58,9 @@ public class HadoopYarnProtoRPC extends YarnRPC { LOG.info("Creating a HadoopYarnProtoRpc server for protocol " + protocol + " with " + numHandlers + " handlers"); - final RPC.Server hadoopServer; - hadoopServer = - RpcFactoryProvider.getServerFactory(conf).getServer(protocol, instance, + return RpcFactoryProvider.getServerFactory(conf).getServer(protocol, instance, addr, conf, secretManager, numHandlers); - Server server = new Server() { - @Override - public void close() { - hadoopServer.stop(); - } - - @Override - public int getPort() { - return hadoopServer.getListenerAddress().getPort(); - } - - @Override - public void join() throws InterruptedException { - hadoopServer.join(); - } - - @Override - public void start() { - hadoopServer.start(); - } - }; - return server; - } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnRPC.java index 9bd2ff03f3f..3a5146ae1f0 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnRPC.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnRPC.java @@ -21,7 +21,7 @@ package org.apache.hadoop.yarn.ipc; import java.io.IOException; import java.net.InetSocketAddress; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.ipc.Server; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -30,7 +30,6 @@ 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.conf.YarnConfiguration; /** * This uses Hadoop RPC. Uses a tunnel AvroSpecificRpcEngine over @@ -70,29 +69,7 @@ public class HadoopYarnRPC extends YarnRPC { } catch (IOException e) { throw new YarnException(e); } - Server server = new Server() { - @Override - public void close() { - hadoopServer.stop(); - } - - @Override - public int getPort() { - return hadoopServer.getListenerAddress().getPort(); - } - - @Override - public void join() throws InterruptedException { - hadoopServer.join(); - } - - @Override - public void start() { - hadoopServer.start(); - } - }; - return server; - + return hadoopServer; } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java index bbd02a0c70b..c6de9ae1905 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.ipc; import java.net.InetSocketAddress; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.ipc.Server; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java index a855cc6f218..a53175b94de 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java @@ -22,7 +22,7 @@ import java.net.InetSocketAddress; import junit.framework.Assert; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.ipc.Server; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.yarn.api.ClientRMProtocol; @@ -162,7 +162,7 @@ public class TestRPC { } Assert.assertTrue(exception); - server.close(); + server.stop(); Assert.assertNotNull(status); Assert.assertEquals(ContainerState.RUNNING, status.getState()); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfiguration.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfiguration.java index 3d2a5769097..d776e630223 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfiguration.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfiguration.java @@ -18,12 +18,9 @@ package org.apache.hadoop.yarn.conf; -import java.net.InetSocketAddress; import junit.framework.Assert; -import org.apache.avro.ipc.Server; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.junit.Test; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java index e8b869eddb7..af4aa2309f0 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java @@ -27,7 +27,8 @@ import java.net.UnknownHostException; import java.nio.ByteBuffer; import java.util.Map; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.ipc.Server; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -35,6 +36,7 @@ import org.apache.hadoop.io.DataInputByteBuffer; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authorize.PolicyProvider; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.yarn.YarnException; @@ -87,6 +89,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.Contai import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl; import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; +import org.apache.hadoop.yarn.server.nodemanager.security.authorize.NMPolicyProvider; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.yarn.service.CompositeService; @@ -201,6 +204,14 @@ public class ContainerManagerImpl extends CompositeService implements this.containerTokenSecretManager, conf.getInt(YarnConfiguration.NM_CONTAINER_MGR_THREAD_COUNT, YarnConfiguration.DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT)); + + // Enable service authorization? + if (conf.getBoolean( + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, + false)) { + refreshServiceAcls(conf, new NMPolicyProvider()); + } + server.start(); InetAddress hostNameResolved = null; try { @@ -215,13 +226,18 @@ public class ContainerManagerImpl extends CompositeService implements super.start(); } + void refreshServiceAcls(Configuration configuration, + PolicyProvider policyProvider) { + this.server.refreshServiceAcl(configuration, policyProvider); + } + @Override public void stop() { if (auxiliaryServices.getServiceState() == STARTED) { auxiliaryServices.unregister(this); } if (server != null) { - server.close(); + server.stop(); } super.stop(); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java index 22c2d3d9358..ff55625b822 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java @@ -62,10 +62,11 @@ import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.ipc.Server; 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.fs.FileContext; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; @@ -104,8 +105,8 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.even import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceLocalizedEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceReleaseEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceRequestEvent; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerSecurityInfo; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerTokenSecretManager; +import org.apache.hadoop.yarn.server.nodemanager.security.authorize.NMPolicyProvider; import org.apache.hadoop.yarn.service.AbstractService; import org.apache.hadoop.yarn.service.CompositeService; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -251,17 +252,25 @@ public class ResourceLocalizationService extends CompositeService secretManager = new LocalizerTokenSecretManager(); } - return rpc.getServer(LocalizationProtocol.class, this, + Server server = rpc.getServer(LocalizationProtocol.class, this, localizationServerAddress, conf, secretManager, conf.getInt(YarnConfiguration.NM_LOCALIZER_CLIENT_THREAD_COUNT, YarnConfiguration.DEFAULT_NM_LOCALIZER_CLIENT_THREAD_COUNT)); - + + // Enable service authorization? + if (conf.getBoolean( + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, + false)) { + server.refreshServiceAcl(conf, new NMPolicyProvider()); + } + + return server; } @Override public void stop() { if (server != null) { - server.close(); + server.stop(); } cacheCleanup.shutdown(); super.stop(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/authorize/NMPolicyProvider.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/authorize/NMPolicyProvider.java new file mode 100644 index 00000000000..0f818bd3b29 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/authorize/NMPolicyProvider.java @@ -0,0 +1,49 @@ +/** + * 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.server.nodemanager.security.authorize; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.security.authorize.PolicyProvider; +import org.apache.hadoop.security.authorize.Service; +import org.apache.hadoop.yarn.proto.ContainerManager; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.proto.LocalizationProtocol; + +/** + * {@link PolicyProvider} for YARN NodeManager protocols. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class NMPolicyProvider extends PolicyProvider { + + private static final Service[] nodeManagerServices = + new Service[] { + new Service( + YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGER, + ContainerManager.ContainerManagerService.BlockingInterface.class), + new Service(YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER, + LocalizationProtocol.LocalizationProtocolService.BlockingInterface.class) + }; + + @Override + public Service[] getServices() { + return nodeManagerServices; + } + +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBLocalizerRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBLocalizerRPC.java index fdee8970953..39eecff3df9 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBLocalizerRPC.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBLocalizerRPC.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb; import java.net.InetSocketAddress; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.ipc.Server; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; @@ -61,7 +61,7 @@ public class TestPBLocalizerRPC { public void stop() { if (server != null) { - server.close(); + server.stop(); } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java index c4169e56b84..741a01a4dde 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java @@ -33,7 +33,7 @@ import java.util.Set; import junit.framework.Assert; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.ipc.Server; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.AbstractFileSystem; import org.apache.hadoop.fs.FSDataOutputStream; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java index 71eba779a58..c2a90185696 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java @@ -21,15 +21,17 @@ package org.apache.hadoop.yarn.server.resourcemanager; import java.io.IOException; import java.net.InetSocketAddress; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.ipc.Server; 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.net.NetUtils; import org.apache.hadoop.security.AccessControlException; 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.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnRemoteException; @@ -45,11 +47,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.Refresh import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshNodesResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshQueuesRequest; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshQueuesResponse; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshServiceAclsRequest; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshServiceAclsResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshUserToGroupsMappingsRequest; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshUserToGroupsMappingsResponse; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider; import org.apache.hadoop.yarn.service.AbstractService; public class AdminService extends AbstractService implements RMAdminProtocol { @@ -60,7 +65,11 @@ public class AdminService extends AbstractService implements RMAdminProtocol { 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 Server server; private InetSocketAddress masterServiceAddress; private AccessControlList adminAcl; @@ -69,12 +78,18 @@ public class AdminService extends AbstractService implements RMAdminProtocol { RecordFactoryProvider.getRecordFactory(null); public AdminService(Configuration conf, ResourceScheduler scheduler, - RMContext rmContext, NodesListManager nodesListManager) { + RMContext rmContext, NodesListManager nodesListManager, + ClientRMService clientRMService, + ApplicationMasterService applicationMasterService, + ResourceTrackerService resourceTrackerService) { super(AdminService.class.getName()); this.conf = conf; this.scheduler = scheduler; this.rmContext = rmContext; this.nodesListManager = nodesListManager; + this.clientRMService = clientRMService; + this.applicationMasterService = applicationMasterService; + this.resourceTrackerService = resourceTrackerService; } @Override @@ -99,6 +114,14 @@ public class AdminService extends AbstractService implements RMAdminProtocol { 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, + false)) { + refreshServiceAcls(conf, new RMPolicyProvider()); + } + this.server.start(); super.start(); } @@ -106,7 +129,7 @@ public class AdminService extends AbstractService implements RMAdminProtocol { @Override public void stop() { if (this.server != null) { - this.server.close(); + this.server.stop(); } super.stop(); } @@ -222,4 +245,33 @@ public class AdminService extends AbstractService implements RMAdminProtocol { return recordFactory.newRecordInstance(RefreshAdminAclsResponse.class); } + + @Override + public RefreshServiceAclsResponse refreshServiceAcls( + RefreshServiceAclsRequest request) throws YarnRemoteException { + Configuration conf = new Configuration(); + if (!conf.getBoolean( + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, + false)) { + throw RPCUtil.getRemoteException( + new IOException("Service Authorization (" + + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION + + ") not enabled.")); + } + + PolicyProvider policyProvider = new RMPolicyProvider(); + + refreshServiceAcls(conf, policyProvider); + clientRMService.refreshServiceAcls(conf, policyProvider); + applicationMasterService.refreshServiceAcls(conf, policyProvider); + resourceTrackerService.refreshServiceAcls(conf, policyProvider); + + return recordFactory.newRecordInstance(RefreshServiceAclsResponse.class); + } + + void refreshServiceAcls(Configuration configuration, + PolicyProvider policyProvider) { + this.server.refreshServiceAcl(configuration, policyProvider); + } + } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java index 44840e1eefe..0e4065d6297 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java @@ -23,13 +23,14 @@ import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import org.apache.avro.ipc.Server; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.ipc.Server; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.SecurityInfo; +import org.apache.hadoop.security.authorize.PolicyProvider; import org.apache.hadoop.yarn.api.AMRMProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; @@ -49,7 +50,6 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager; -import org.apache.hadoop.yarn.security.SchedulerSecurityInfo; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor; @@ -59,6 +59,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider; import org.apache.hadoop.yarn.service.AbstractService; @Private @@ -107,6 +108,14 @@ public class ApplicationMasterService extends AbstractService implements conf, this.appTokenManager, conf.getInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT, YarnConfiguration.DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT)); + + // Enable service authorization? + if (conf.getBoolean( + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, + false)) { + refreshServiceAcls(conf, new RMPolicyProvider()); + } + this.server.start(); super.start(); } @@ -258,10 +267,15 @@ public class ApplicationMasterService extends AbstractService implements } } + public void refreshServiceAcls(Configuration configuration, + PolicyProvider policyProvider) { + this.server.refreshServiceAcl(configuration, policyProvider); + } + @Override public void stop() { if (this.server != null) { - this.server.close(); + this.server.stop(); } super.stop(); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java index 525a07bc655..781837f3260 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java @@ -26,12 +26,14 @@ import java.util.Collection; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.avro.ipc.Server; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.ipc.Server; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authorize.PolicyProvider; import org.apache.hadoop.yarn.api.ClientRMProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; @@ -70,6 +72,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.service.AbstractService; @@ -129,6 +132,14 @@ public class ClientRMService extends AbstractService implements conf, null, conf.getInt(YarnConfiguration.RM_CLIENT_THREAD_COUNT, YarnConfiguration.DEFAULT_RM_CLIENT_THREAD_COUNT)); + + // Enable service authorization? + if (conf.getBoolean( + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, + false)) { + refreshServiceAcls(conf, new RMPolicyProvider()); + } + this.server.start(); super.start(); } @@ -401,11 +412,17 @@ public class ClientRMService extends AbstractService implements return response; } + void refreshServiceAcls(Configuration configuration, + PolicyProvider policyProvider) { + this.server.refreshServiceAcl(configuration, policyProvider); + } + @Override public void stop() { if (this.server != null) { - this.server.close(); + this.server.stop(); } super.stop(); } + } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index 77305045a3c..b51e0198037 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -188,7 +188,7 @@ public class ResourceManager extends CompositeService implements Recoverable { clientRM = createClientRMService(); addService(clientRM); - adminService = createAdminService(); + adminService = createAdminService(clientRM, masterService, resourceTracker); addService(adminService); this.applicationMasterLauncher = createAMLauncher(); @@ -466,9 +466,13 @@ public class ResourceManager extends CompositeService implements Recoverable { } - protected AdminService createAdminService() { + protected AdminService createAdminService( + ClientRMService clientRMService, + ApplicationMasterService applicationMasterService, + ResourceTrackerService resourceTrackerService) { return new AdminService(this.conf, scheduler, rmContext, - this.nodesListManager); + this.nodesListManager, clientRMService, applicationMasterService, + resourceTrackerService); } @Private diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java index 1947bb66fce..9b716f81460 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java @@ -23,13 +23,15 @@ import java.nio.ByteBuffer; import javax.crypto.SecretKey; -import org.apache.avro.ipc.Server; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.ipc.Server; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.Node; import org.apache.hadoop.security.SecurityInfo; +import org.apache.hadoop.security.authorize.PolicyProvider; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; @@ -53,6 +55,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent; +import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider; import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.yarn.service.AbstractService; import org.apache.hadoop.yarn.util.RackResolver; @@ -119,14 +122,21 @@ public class ResourceTrackerService extends AbstractService implements conf, null, conf.getInt(YarnConfiguration.RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT, YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT)); - this.server.start(); + + // Enable service authorization? + if (conf.getBoolean( + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, + false)) { + refreshServiceAcls(conf, new RMPolicyProvider()); + } + this.server.start(); } @Override public synchronized void stop() { if (this.server != null) { - this.server.close(); + this.server.stop(); } super.stop(); } @@ -289,4 +299,9 @@ public class ResourceTrackerService extends AbstractService implements return RackResolver.resolve(hostName); } + void refreshServiceAcls(Configuration configuration, + PolicyProvider policyProvider) { + this.server.refreshServiceAcl(configuration, policyProvider); + } + } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/RMAdminProtocol.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/RMAdminProtocol.java index c1578a6b882..267a6cb8a29 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/RMAdminProtocol.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/RMAdminProtocol.java @@ -25,6 +25,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.Refresh import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshNodesResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshQueuesRequest; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshQueuesResponse; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshServiceAclsRequest; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshServiceAclsResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshUserToGroupsMappingsRequest; @@ -49,4 +51,8 @@ public interface RMAdminProtocol { public RefreshAdminAclsResponse refreshAdminAcls( RefreshAdminAclsRequest request) throws YarnRemoteException; + + public RefreshServiceAclsResponse refreshServiceAcls( + RefreshServiceAclsRequest request) + throws YarnRemoteException; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/impl/pb/client/RMAdminProtocolPBClientImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/impl/pb/client/RMAdminProtocolPBClientImpl.java index 6e8ca2680a0..cf2ce894ee5 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/impl/pb/client/RMAdminProtocolPBClientImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/impl/pb/client/RMAdminProtocolPBClientImpl.java @@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.proto.RMAdminProtocol.RMAdminProtocolService; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesRequestProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesRequestProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsRequestProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshSuperUserGroupsConfigurationRequestProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshUserToGroupsMappingsRequestProto; import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocol; @@ -39,6 +40,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.Refresh import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshNodesResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshQueuesRequest; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshQueuesResponse; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshServiceAclsRequest; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshServiceAclsResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshUserToGroupsMappingsRequest; @@ -49,6 +52,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshNodesResponsePBImpl; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshQueuesRequestPBImpl; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshQueuesResponsePBImpl; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshServiceAclsRequestPBImpl; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshServiceAclsResponsePBImpl; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshSuperUserGroupsConfigurationRequestPBImpl; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshSuperUserGroupsConfigurationResponsePBImpl; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsRequestPBImpl; @@ -165,5 +170,24 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol { } } + @Override + public RefreshServiceAclsResponse refreshServiceAcls( + RefreshServiceAclsRequest request) throws YarnRemoteException { + RefreshServiceAclsRequestProto requestProto = + ((RefreshServiceAclsRequestPBImpl)request).getProto(); + try { + return new RefreshServiceAclsResponsePBImpl( + proxy.refreshServiceAcls(null, requestProto)); + } catch (ServiceException e) { + if (e.getCause() instanceof YarnRemoteException) { + throw (YarnRemoteException)e.getCause(); + } else if (e.getCause() instanceof UndeclaredThrowableException) { + throw (UndeclaredThrowableException)e.getCause(); + } else { + throw new UndeclaredThrowableException(e); + } + } + } + } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java index 4d0dde2c3e1..f6b6760b539 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java @@ -20,11 +20,14 @@ package org.apache.hadoop.yarn.server.resourcemanager.api.impl.pb.service; import org.apache.hadoop.yarn.exceptions.YarnRemoteException; import org.apache.hadoop.yarn.proto.RMAdminProtocol.RMAdminProtocolService.BlockingInterface; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsRequestProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsResponseProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.*; import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocol; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshAdminAclsResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshNodesResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshQueuesResponse; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshServiceAclsResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshUserToGroupsMappingsResponse; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshAdminAclsRequestPBImpl; @@ -33,6 +36,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshNodesResponsePBImpl; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshQueuesRequestPBImpl; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshQueuesResponsePBImpl; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshServiceAclsRequestPBImpl; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshServiceAclsResponsePBImpl; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshSuperUserGroupsConfigurationRequestPBImpl; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshSuperUserGroupsConfigurationResponsePBImpl; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsRequestPBImpl; @@ -119,4 +124,19 @@ public class RMAdminProtocolPBServiceImpl implements BlockingInterface { } } + @Override + public RefreshServiceAclsResponseProto refreshServiceAcls( + RpcController controller, RefreshServiceAclsRequestProto proto) + throws ServiceException { + RefreshServiceAclsRequestPBImpl request = + new RefreshServiceAclsRequestPBImpl(proto); + try { + RefreshServiceAclsResponse response = + real.refreshServiceAcls(request); + return ((RefreshServiceAclsResponsePBImpl)response).getProto(); + } catch (YarnRemoteException e) { + throw new ServiceException(e); + } + } + } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/RefreshServiceAclsRequest.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/RefreshServiceAclsRequest.java new file mode 100644 index 00000000000..b016a71c035 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/RefreshServiceAclsRequest.java @@ -0,0 +1,23 @@ +/** +* 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.server.resourcemanager.api.protocolrecords; + +public interface RefreshServiceAclsRequest { + +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/RefreshServiceAclsResponse.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/RefreshServiceAclsResponse.java new file mode 100644 index 00000000000..dd6ef338e62 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/RefreshServiceAclsResponse.java @@ -0,0 +1,23 @@ +/** +* 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.server.resourcemanager.api.protocolrecords; + +public interface RefreshServiceAclsResponse { + +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/impl/pb/RefreshServiceAclsRequestPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/impl/pb/RefreshServiceAclsRequestPBImpl.java new file mode 100644 index 00000000000..0d558fe01f0 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/impl/pb/RefreshServiceAclsRequestPBImpl.java @@ -0,0 +1,49 @@ +/** +* 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.server.resourcemanager.api.protocolrecords.impl.pb; + +import org.apache.hadoop.yarn.api.records.ProtoBase; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsRequestProto; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshServiceAclsRequest; + +public class RefreshServiceAclsRequestPBImpl +extends ProtoBase +implements RefreshServiceAclsRequest { + + RefreshServiceAclsRequestProto proto = + RefreshServiceAclsRequestProto.getDefaultInstance(); + RefreshServiceAclsRequestProto.Builder builder = null; + boolean viaProto = false; + + public RefreshServiceAclsRequestPBImpl() { + builder = RefreshServiceAclsRequestProto.newBuilder(); + } + + public RefreshServiceAclsRequestPBImpl( + RefreshServiceAclsRequestProto proto) { + this.proto = proto; + viaProto = true; + } + + public RefreshServiceAclsRequestProto getProto() { + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/impl/pb/RefreshServiceAclsResponsePBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/impl/pb/RefreshServiceAclsResponsePBImpl.java new file mode 100644 index 00000000000..f09bc5cd860 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/api/protocolrecords/impl/pb/RefreshServiceAclsResponsePBImpl.java @@ -0,0 +1,49 @@ +/** +* 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.server.resourcemanager.api.protocolrecords.impl.pb; + +import org.apache.hadoop.yarn.api.records.ProtoBase; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsResponseProto; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshServiceAclsResponse; + +public class RefreshServiceAclsResponsePBImpl +extends ProtoBase +implements RefreshServiceAclsResponse { + + RefreshServiceAclsResponseProto proto = + RefreshServiceAclsResponseProto.getDefaultInstance(); + RefreshServiceAclsResponseProto.Builder builder = null; + boolean viaProto = false; + + public RefreshServiceAclsResponsePBImpl() { + builder = RefreshServiceAclsResponseProto.newBuilder(); + } + + public RefreshServiceAclsResponsePBImpl( + RefreshServiceAclsResponseProto proto) { + this.proto = proto; + viaProto = true; + } + + public RefreshServiceAclsResponseProto getProto() { + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/admin/AdminSecurityInfo.java similarity index 96% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/admin/AdminSecurityInfo.java index 2fd8eb3b178..48eda6930a7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/admin/AdminSecurityInfo.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.security.admin; +package org.apache.hadoop.yarn.server.resourcemanager.security.admin; import java.lang.annotation.Annotation; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/authorize/RMPolicyProvider.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/authorize/RMPolicyProvider.java new file mode 100644 index 00000000000..6fe2c1912e8 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/authorize/RMPolicyProvider.java @@ -0,0 +1,62 @@ +/** + * 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.server.resourcemanager.security.authorize; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.security.authorize.PolicyProvider; +import org.apache.hadoop.security.authorize.Service; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.proto.ContainerManager; +import org.apache.hadoop.yarn.proto.ResourceTracker; +import org.apache.hadoop.yarn.proto.RMAdminProtocol; +import org.apache.hadoop.yarn.proto.ClientRMProtocol; +import org.apache.hadoop.yarn.proto.AMRMProtocol; + +/** + * {@link PolicyProvider} for YARN ResourceManager protocols. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class RMPolicyProvider extends PolicyProvider { + + private static final Service[] resourceManagerServices = + new Service[] { + new Service( + YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER, + ResourceTracker.ResourceTrackerService.BlockingInterface.class), + new Service( + YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CLIENT_RESOURCEMANAGER, + ClientRMProtocol.ClientRMProtocolService.BlockingInterface.class), + new Service( + YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONMASTER_RESOURCEMANAGER, + AMRMProtocol.AMRMProtocolService.BlockingInterface.class), + new Service( + YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_ADMIN, + RMAdminProtocol.RMAdminProtocolService.BlockingInterface.class), + new Service( + YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGER, + ContainerManager.ContainerManagerService.BlockingInterface.class), + }; + + @Override + public Service[] getServices() { + return resourceManagerServices; + } + +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/tools/RMAdmin.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/tools/RMAdmin.java index 9c4fee2e8ab..4d429306dd2 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/tools/RMAdmin.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/tools/RMAdmin.java @@ -26,7 +26,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; @@ -34,11 +33,11 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.ipc.YarnRPC; -import org.apache.hadoop.yarn.security.admin.AdminSecurityInfo; import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocol; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshAdminAclsRequest; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshNodesRequest; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshQueuesRequest; +import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshServiceAclsRequest; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshUserToGroupsMappingsRequest; @@ -64,6 +63,7 @@ public class RMAdmin extends Configured implements Tool { " [-refreshSuperUserGroupsConfiguration]" + " [-refreshUserToGroupsMappings]" + " [-refreshAdminAcls]" + + " [-refreshServiceAcl]" + " [-help [cmd]]\n"; String refreshQueues = @@ -85,6 +85,10 @@ public class RMAdmin extends Configured implements Tool { String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" + "\t\tis specified.\n"; + String refreshServiceAcl = + "-refreshServiceAcl: Reload the service-level authorization policy file\n" + + "\t\tResoureceManager will reload the authorization policy file.\n"; + if ("refreshQueues".equals(cmd)) { System.out.println(refreshQueues); } else if ("refreshNodes".equals(cmd)) { @@ -95,11 +99,18 @@ public class RMAdmin extends Configured implements Tool { System.out.println(refreshSuperUserGroupsConfiguration); } else if ("refreshAdminAcls".equals(cmd)) { System.out.println(refreshAdminAcls); + } else if ("refreshServiceAcl".equals(cmd)) { + System.out.println(refreshServiceAcl); } else if ("help".equals(cmd)) { System.out.println(help); } else { System.out.println(summary); System.out.println(refreshQueues); + System.out.println(refreshNodes); + System.out.println(refreshUserToGroupsMappings); + System.out.println(refreshSuperUserGroupsConfiguration); + System.out.println(refreshAdminAcls); + System.out.println(refreshServiceAcl); System.out.println(help); System.out.println(); ToolRunner.printGenericCommandUsage(System.out); @@ -121,6 +132,8 @@ public class RMAdmin extends Configured implements Tool { System.err.println("Usage: java RMAdmin" + " [-refreshSuperUserGroupsConfiguration]"); } else if ("-refreshAdminAcls".equals(cmd)){ System.err.println("Usage: java RMAdmin" + " [-refreshAdminAcls]"); + } else if ("-refreshService".equals(cmd)){ + System.err.println("Usage: java RMAdmin" + " [-refreshServiceAcl]"); } else { System.err.println("Usage: java RMAdmin"); System.err.println(" [-refreshQueues]"); @@ -128,6 +141,7 @@ public class RMAdmin extends Configured implements Tool { System.err.println(" [-refreshUserToGroupsMappings]"); System.err.println(" [-refreshSuperUserGroupsConfiguration]"); System.err.println(" [-refreshAdminAcls]"); + System.err.println(" [-refreshServiceAcl]"); System.err.println(" [-help [cmd]]"); System.err.println(); ToolRunner.printGenericCommandUsage(System.err); @@ -210,6 +224,15 @@ public class RMAdmin extends Configured implements Tool { return 0; } + private int refreshServiceAcls() throws IOException { + // Refresh the service acls + RMAdminProtocol adminProtocol = createAdminProtocol(); + RefreshServiceAclsRequest request = + recordFactory.newRecordInstance(RefreshServiceAclsRequest.class); + adminProtocol.refreshServiceAcls(request); + return 0; + } + @Override public int run(String[] args) throws Exception { if (args.length < 1) { @@ -224,7 +247,7 @@ public class RMAdmin extends Configured implements Tool { // verify that we have enough command line parameters // if ("-refreshAdminAcls".equals(cmd) || "-refreshQueues".equals(cmd) || - "-refreshNodes".equals(cmd) || + "-refreshNodes".equals(cmd) || "-refreshServiceAcl".equals(cmd) || "-refreshUserToGroupsMappings".equals(cmd) || "-refreshSuperUserGroupsConfiguration".equals(cmd)) { if (args.length != 1) { @@ -245,6 +268,8 @@ public class RMAdmin extends Configured implements Tool { exitCode = refreshSuperUserGroupsConfiguration(); } else if ("-refreshAdminAcls".equals(cmd)) { exitCode = refreshAdminAcls(); + } else if ("-refreshServiceAcl".equals(cmd)) { + exitCode = refreshServiceAcls(); } else if ("-help".equals(cmd)) { if (i < args.length) { printUsage(args[i]); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/proto/RMAdminProtocol.proto b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/proto/RMAdminProtocol.proto index 09e9419a7f5..8ef9b84210b 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/proto/RMAdminProtocol.proto +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/proto/RMAdminProtocol.proto @@ -29,4 +29,5 @@ service RMAdminProtocolService { rpc refreshSuperUserGroupsConfiguration(RefreshSuperUserGroupsConfigurationRequestProto) returns (RefreshSuperUserGroupsConfigurationResponseProto); rpc refreshUserToGroupsMappings(RefreshUserToGroupsMappingsRequestProto) returns (RefreshUserToGroupsMappingsResponseProto); rpc refreshAdminAcls(RefreshAdminAclsRequestProto) returns (RefreshAdminAclsResponseProto); + rpc refreshServiceAcls(RefreshServiceAclsRequestProto) returns (RefreshServiceAclsResponseProto); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/proto/yarn_server_resourcemanager_service_protos.proto b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/proto/yarn_server_resourcemanager_service_protos.proto index 58c541ee775..1eea68732be 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/proto/yarn_server_resourcemanager_service_protos.proto +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/proto/yarn_server_resourcemanager_service_protos.proto @@ -46,3 +46,9 @@ message RefreshAdminAclsRequestProto { } message RefreshAdminAclsResponseProto { } + +message RefreshServiceAclsRequestProto { +} +message RefreshServiceAclsResponseProto { +} + diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo index a77d06146fb..1f589775198 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo @@ -1 +1 @@ -org.apache.hadoop.yarn.security.admin.AdminSecurityInfo +org.apache.hadoop.yarn.server.resourcemanager.security.admin.AdminSecurityInfo diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java index 17298799531..f522b78c1a5 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java @@ -203,9 +203,13 @@ public class MockRM extends ResourceManager { } @Override - protected AdminService createAdminService() { - return new AdminService(getConfig(), scheduler, getRMContext(), - this.nodesListManager){ + protected AdminService createAdminService( + ClientRMService clientRMService, + ApplicationMasterService applicationMasterService, + ResourceTrackerService resourceTrackerService) { + return new AdminService( + getConfig(), scheduler, getRMContext(), this.nodesListManager, + clientRMService, applicationMasterService, resourceTrackerService){ @Override public void start() { //override to not start rpc handler diff --git a/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java b/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java index 18c40f3dd59..89bb9b37752 100644 --- a/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java +++ b/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java @@ -19,6 +19,7 @@ package org.apache.hadoop.mapred; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.mapreduce.protocol.ClientProtocol; import org.apache.hadoop.security.RefreshUserMappingsProtocol; import org.apache.hadoop.security.authorize.PolicyProvider; @@ -40,14 +41,17 @@ public class MapReducePolicyProvider extends PolicyProvider { ClientProtocol.class), new Service("security.task.umbilical.protocol.acl", TaskUmbilicalProtocol.class), - new Service("security.refresh.policy.protocol.acl", - RefreshAuthorizationPolicyProtocol.class), - new Service("security.refresh.user.mappings.protocol.acl", - RefreshUserMappingsProtocol.class), + new Service( + CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_REFRESH_POLICY, + RefreshAuthorizationPolicyProtocol.class), + new Service( + CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_REFRESH_USER_MAPPINGS, + RefreshUserMappingsProtocol.class), new Service("security.admin.operations.protocol.acl", AdminOperationsProtocol.class), - new Service("security.get.user.mappings.protocol.acl", - GetUserMappingsProtocol.class) + new Service( + CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_GET_USER_MAPPINGS, + GetUserMappingsProtocol.class) }; @Override