From 68807a65c2fb0165664dee15997bf77f235a3512 Mon Sep 17 00:00:00 2001 From: Arun Murthy Date: Mon, 17 Jun 2013 02:29:56 +0000 Subject: [PATCH] Merge -c 1493623 from trunk to branch-2 to fix YARN-387. Renamed YARN protocols for consistency. Contributed by Vinod K V. git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1493624 13f79535-47bb-0310-9956-ffa450edef68 --- .../src/main/conf/hadoop-policy.xml | 18 ++++----- .../app/launcher/ContainerLauncherImpl.java | 16 ++++---- .../mapreduce/v2/app/rm/RMCommunicator.java | 12 +++--- .../mapreduce/v2/app/MRAppBenchmark.java | 6 +-- .../hadoop/mapreduce/v2/app/TestFail.java | 4 +- .../v2/app/TestRMContainerAllocator.java | 8 ++-- .../app/launcher/TestContainerLauncher.java | 12 +++--- .../launcher/TestContainerLauncherImpl.java | 20 +++++----- .../local/TestLocalContainerAllocator.java | 6 +-- .../resources/job_1329348432655_0001_conf.xml | 2 +- .../hadoop/mapred/ResourceMgrDelegate.java | 4 +- .../hadoop/mapred/TestClientRedirect.java | 6 +-- .../mapred/TestResourceMgrDelegate.java | 6 +-- .../apache/hadoop/mapred/TestYARNRunner.java | 4 +- .../TestYarnClientProtocolProvider.java | 4 +- hadoop-yarn-project/CHANGES.txt | 6 +++ .../hadoop-yarn/hadoop-yarn-api/pom.xml | 8 ++-- ...ol.java => ApplicationClientProtocol.java} | 2 +- ....java => ApplicationClientProtocolPB.java} | 6 +-- ...ol.java => ApplicationMasterProtocol.java} | 2 +- ....java => ApplicationMasterProtocolPB.java} | 7 ++-- ....java => ContainerManagementProtocol.java} | 2 +- ...ava => ContainerManagementProtocolPB.java} | 6 +-- ...esourceManagerAdministrationProtocol.java} | 2 +- ...ourceManagerAdministrationProtocolPB.java} | 7 ++-- .../api/protocolrecords/AllocateRequest.java | 4 +- .../api/protocolrecords/AllocateResponse.java | 4 +- .../FinishApplicationMasterRequest.java | 4 +- .../FinishApplicationMasterResponse.java | 4 +- .../GetAllApplicationsRequest.java | 4 +- .../GetAllApplicationsResponse.java | 4 +- .../GetApplicationReportRequest.java | 4 +- .../GetApplicationReportResponse.java | 4 +- .../GetClusterMetricsRequest.java | 4 +- .../GetClusterMetricsResponse.java | 4 +- .../GetClusterNodesRequest.java | 4 +- .../GetClusterNodesResponse.java | 4 +- .../GetContainerStatusRequest.java | 4 +- .../GetContainerStatusResponse.java | 4 +- .../GetNewApplicationRequest.java | 4 +- .../GetNewApplicationResponse.java | 4 +- .../protocolrecords/GetQueueInfoRequest.java | 4 +- .../protocolrecords/GetQueueInfoResponse.java | 4 +- .../GetQueueUserAclsInfoRequest.java | 4 +- .../GetQueueUserAclsInfoResponse.java | 4 +- .../KillApplicationRequest.java | 4 +- .../KillApplicationResponse.java | 4 +- .../RegisterApplicationMasterRequest.java | 4 +- .../RegisterApplicationMasterResponse.java | 4 +- .../StartContainerRequest.java | 4 +- .../StartContainerResponse.java | 4 +- .../protocolrecords/StopContainerRequest.java | 4 +- .../StopContainerResponse.java | 4 +- .../SubmitApplicationRequest.java | 4 +- .../SubmitApplicationResponse.java | 4 +- .../yarn/api/records/ApplicationReport.java | 4 +- .../records/ApplicationSubmissionContext.java | 4 +- .../hadoop/yarn/api/records/Container.java | 14 +++---- .../api/records/ContainerLaunchContext.java | 4 +- .../yarn/api/records/LocalResource.java | 4 +- .../yarn/api/records/LocalResourceType.java | 4 +- .../api/records/LocalResourceVisibility.java | 4 +- .../hadoop/yarn/api/records/NodeReport.java | 4 +- .../hadoop/yarn/api/records/QueueACL.java | 4 +- .../hadoop/yarn/api/records/QueueInfo.java | 4 +- .../hadoop/yarn/api/records/QueueState.java | 4 +- .../yarn/api/records/QueueUserACLInfo.java | 4 +- .../hadoop/yarn/api/records/Resource.java | 4 +- .../api/records/ResourceBlacklistRequest.java | 4 +- .../yarn/api/records/ResourceRequest.java | 4 +- .../hadoop/yarn/conf/YarnConfiguration.java | 18 ++++----- ...proto => applicationclient_protocol.proto} | 4 +- ...proto => applicationmaster_protocol.proto} | 4 +- ...oto => containermanagement_protocol.proto} | 4 +- ...urcemanager_administration_protocol.proto} | 4 +- .../distributedshell/ApplicationMaster.java | 14 +++---- .../applications/distributedshell/Client.java | 4 +- .../hadoop/yarn/client/AMRMClientImpl.java | 10 ++--- .../hadoop/yarn/client/NMClientImpl.java | 10 ++--- .../hadoop/yarn/client/YarnClientImpl.java | 8 ++-- .../hadoop/yarn/client/cli/RMAdminCLI.java | 26 ++++++------ .../yarn/client/GetGroupsForTesting.java | 6 +-- .../hadoop/yarn/client/TestAMRMClient.java | 6 +-- .../hadoop/yarn/client/TestYarnClient.java | 4 +- ...pplicationClientProtocolPBClientImpl.java} | 14 +++---- ...pplicationMasterProtocolPBClientImpl.java} | 14 +++---- ...tainerManagementProtocolPBClientImpl.java} | 14 +++---- ...erAdministrationProtocolPBClientImpl.java} | 16 ++++---- ...plicationClientProtocolPBServiceImpl.java} | 10 ++--- ...plicationMasterProtocolPBServiceImpl.java} | 10 ++--- ...ainerManagementProtocolPBServiceImpl.java} | 10 ++--- ...rAdministrationProtocolPBServiceImpl.java} | 10 ++--- .../ContainerManagerSecurityInfo.java | 4 +- .../yarn/security/SchedulerSecurityInfo.java | 4 +- .../security/admin/AdminSecurityInfo.java | 4 +- .../security/client/ClientRMSecurityInfo.java | 6 +-- .../client/RMDelegationTokenIdentifier.java | 10 ++--- .../hadoop/yarn/TestContainerLaunchRPC.java | 10 ++--- .../java/org/apache/hadoop/yarn/TestRPC.java | 26 ++++++------ .../apache/hadoop/yarn/TestRPCFactories.java | 16 ++++---- .../server/api/records/NodeHealthStatus.java | 4 +- .../yarn/server/nodemanager/Context.java | 4 +- .../yarn/server/nodemanager/NodeManager.java | 8 ++-- .../ContainerManagerImpl.java | 6 +-- .../security/authorize/NMPolicyProvider.java | 6 +-- .../nodemanager/TestNodeManagerReboot.java | 4 +- .../nodemanager/TestNodeManagerShutdown.java | 10 ++--- .../BaseContainerManagerTest.java | 6 +-- .../server/resourcemanager/AdminService.java | 6 +-- .../ApplicationMasterService.java | 6 +-- .../resourcemanager/ClientRMService.java | 6 +-- .../amlauncher/AMLauncher.java | 12 +++--- .../security/authorize/RMPolicyProvider.java | 26 ++++++------ .../yarn/server/resourcemanager/MockAM.java | 8 ++-- .../yarn/server/resourcemanager/MockRM.java | 12 +++--- .../MockRMWithCustomAMLauncher.java | 10 ++--- .../server/resourcemanager/NodeManager.java | 4 +- .../resourcemanager/TestAMAuthorization.java | 30 +++++++------- .../resourcemanager/TestApplicationACLs.java | 26 ++++++------ .../TestApplicationMasterLauncher.java | 4 +- .../resourcemanager/TestClientRMService.java | 8 ++-- .../resourcemanager/TestClientRMTokens.java | 20 +++++----- .../scheduler/TestSchedulerUtils.java | 10 ++--- .../security/TestApplicationTokens.java | 14 +++---- .../security/TestClientTokens.java | 4 +- .../server/TestContainerManagerSecurity.java | 34 ++++++++-------- .../server/webproxy/AppReportFetcher.java | 8 ++-- .../site/apt/WritingYarnApplications.apt.vm | 40 +++++++++---------- 128 files changed, 504 insertions(+), 498 deletions(-) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/{ClientRMProtocol.java => ApplicationClientProtocol.java} (99%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/{AMRMProtocolPB.java => ApplicationClientProtocolPB.java} (76%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/{AMRMProtocol.java => ApplicationMasterProtocol.java} (99%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/{RMAdminProtocolPB.java => ApplicationMasterProtocolPB.java} (76%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/{ContainerManager.java => ContainerManagementProtocol.java} (99%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/{ContainerManagerPB.java => ContainerManagementProtocolPB.java} (76%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/{RMAdminProtocol.java => ResourceManagerAdministrationProtocol.java} (96%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/{ClientRMProtocolPB.java => ResourceManagerAdministrationProtocolPB.java} (72%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/{client_RM_protocol.proto => applicationclient_protocol.proto} (95%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/{AM_RM_protocol.proto => applicationmaster_protocol.proto} (92%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/{container_manager.proto => containermanagement_protocol.proto} (92%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/{RMAdminProtocol.proto => resourcemanager_administration_protocol.proto} (93%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/{ClientRMProtocolPBClientImpl.java => ApplicationClientProtocolPBClientImpl.java} (96%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/{AMRMProtocolPBClientImpl.java => ApplicationMasterProtocolPBClientImpl.java} (88%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/{ContainerManagerPBClientImpl.java => ContainerManagementProtocolPBClientImpl.java} (90%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/{RMAdminProtocolPBClientImpl.java => ResourceManagerAdministrationProtocolPBClientImpl.java} (92%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/{ClientRMProtocolPBServiceImpl.java => ApplicationClientProtocolPBServiceImpl.java} (97%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/{AMRMProtocolPBServiceImpl.java => ApplicationMasterProtocolPBServiceImpl.java} (92%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/{ContainerManagerPBServiceImpl.java => ContainerManagementProtocolPBServiceImpl.java} (91%) rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/{RMAdminProtocolPBServiceImpl.java => ResourceManagerAdministrationProtocolPBServiceImpl.java} (94%) diff --git a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml index c17966d7f08..491dbe7dbe9 100644 --- a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml +++ b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml @@ -140,7 +140,7 @@ security.resourcetracker.protocol.acl * - ACL for ResourceTracker protocol, used by the + ACL for ResourceTrackerProtocol, 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". @@ -148,18 +148,18 @@ - security.admin.protocol.acl + security.resourcemanager-administration.protocol.acl * - ACL for RMAdminProtocol, for admin commands. + ACL for ResourceManagerAdministrationProtocol, 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 + security.applicationclient.protocol.acl * - ACL for ClientRMProtocol, used by the ResourceManager + ACL for ApplicationClientProtocol, 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". @@ -167,9 +167,9 @@ - security.applicationmaster.resourcemanager.protocol.acl + security.applicationmaster.protocol.acl * - ACL for AMRMProtocol, used by the ResourceManager + ACL for ApplicationMasterProtocol, 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". @@ -177,9 +177,9 @@ - security.containermanager.protocol.acl + security.containermanagement.protocol.acl * - ACL for ContainerManager protocol, used by the NodeManager + ACL for ContainerManagementProtocol 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". 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 4605e9c10c9..4a985423600 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 @@ -48,7 +48,7 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.StringUtils; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse; import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest; @@ -140,7 +140,7 @@ public class ContainerLauncherImpl extends AbstractService implements return; } - ContainerManager proxy = null; + ContainerManagementProtocol proxy = null; try { proxy = getCMProxy(containerID, containerMgrAddress, @@ -198,7 +198,7 @@ public class ContainerLauncherImpl extends AbstractService implements } else if (!isCompletelyDone()) { LOG.info("KILLING " + taskAttemptID); - ContainerManager proxy = null; + ContainerManagementProtocol proxy = null; try { proxy = getCMProxy(this.containerID, this.containerMgrAddress, this.containerToken); @@ -348,7 +348,7 @@ public class ContainerLauncherImpl extends AbstractService implements return new EventProcessor(event); } - protected ContainerManager getCMProxy(ContainerId containerID, + protected ContainerManagementProtocol getCMProxy(ContainerId containerID, final String containerManagerBindAddr, org.apache.hadoop.yarn.api.records.Token containerToken) throws IOException { @@ -364,11 +364,11 @@ public class ContainerLauncherImpl extends AbstractService implements ProtoUtils.convertFromProtoFormat(containerToken, cmAddr); user.addToken(token); - ContainerManager proxy = user - .doAs(new PrivilegedAction() { + ContainerManagementProtocol proxy = user + .doAs(new PrivilegedAction() { @Override - public ContainerManager run() { - return (ContainerManager) rpc.getProxy(ContainerManager.class, + public ContainerManagementProtocol run() { + return (ContainerManagementProtocol) rpc.getProxy(ContainerManagementProtocol.class, cmAddr, getConfig()); } }); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java index 87b6ed6f735..8c5d5609cfa 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java @@ -39,7 +39,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal; import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl; import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; @@ -69,7 +69,7 @@ public abstract class RMCommunicator extends AbstractService protected Thread allocatorThread; @SuppressWarnings("rawtypes") protected EventHandler eventHandler; - protected AMRMProtocol scheduler; + protected ApplicationMasterProtocol scheduler; private final ClientService clientService; protected int lastResponseID; private Resource maxContainerCapability; @@ -254,7 +254,7 @@ public abstract class RMCommunicator extends AbstractService allocatorThread.start(); } - protected AMRMProtocol createSchedulerProxy() { + protected ApplicationMasterProtocol createSchedulerProxy() { final Configuration conf = getConfig(); final YarnRPC rpc = YarnRPC.create(conf); final InetSocketAddress serviceAddr = conf.getSocketAddr( @@ -270,10 +270,10 @@ public abstract class RMCommunicator extends AbstractService } // CurrentUser should already have AMToken loaded. - return currentUser.doAs(new PrivilegedAction() { + return currentUser.doAs(new PrivilegedAction() { @Override - public AMRMProtocol run() { - return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class, + public ApplicationMasterProtocol run() { + return (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class, serviceAddr, conf); } }); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java index 7a579ca0c87..758133d5eab 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java @@ -33,7 +33,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssigned import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator; import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent; import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; @@ -194,8 +194,8 @@ public class MRAppBenchmark { ClientService clientService, AppContext context) { return new RMContainerAllocator(clientService, context) { @Override - protected AMRMProtocol createSchedulerProxy() { - return new AMRMProtocol() { + protected ApplicationMasterProtocol createSchedulerProxy() { + return new ApplicationMasterProtocol() { @Override public RegisterApplicationMasterResponse diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java index 2eddbbf4ec4..0bf6305aed9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java @@ -44,7 +44,7 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher; import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent; import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.Token; import org.junit.Test; @@ -225,7 +225,7 @@ public class TestFail { } @Override - protected ContainerManager getCMProxy(ContainerId contianerID, + protected ContainerManagementProtocol getCMProxy(ContainerId contianerID, String containerManagerBindAddr, Token containerToken) throws IOException { try { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java index a5dc3598c6c..3ec20a476a7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java @@ -74,7 +74,7 @@ import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.yarn.Clock; import org.apache.hadoop.yarn.ClusterInfo; import org.apache.hadoop.yarn.SystemClock; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; @@ -1387,7 +1387,7 @@ public class TestRMContainerAllocator { } @Override - protected AMRMProtocol createSchedulerProxy() { + protected ApplicationMasterProtocol createSchedulerProxy() { return this.rm.getApplicationMasterService(); } @@ -1600,8 +1600,8 @@ public class TestRMContainerAllocator { protected void register() { } @Override - protected AMRMProtocol createSchedulerProxy() { - return mock(AMRMProtocol.class); + protected ApplicationMasterProtocol createSchedulerProxy() { + return mock(ApplicationMasterProtocol.class); } @Override protected synchronized void heartbeat() throws Exception { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java index 3089997b6e5..e33ffd0ec0f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java @@ -49,7 +49,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.TaskAttemptStateInternal; import org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl; import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; @@ -232,7 +232,7 @@ public class TestContainerLauncher { YarnRPC rpc = YarnRPC.create(conf); String bindAddr = "localhost:0"; InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); - server = rpc.getServer(ContainerManager.class, new DummyContainerManager(), + server = rpc.getServer(ContainerManagementProtocol.class, new DummyContainerManager(), addr, conf, null, 1); server.start(); @@ -345,12 +345,12 @@ public class TestContainerLauncher { protected ContainerLauncher createContainerLauncher(AppContext context) { return new ContainerLauncherImpl(context) { @Override - protected ContainerManager getCMProxy(ContainerId containerID, + protected ContainerManagementProtocol getCMProxy(ContainerId containerID, String containerManagerBindAddr, Token containerToken) throws IOException { // make proxy connect to our local containerManager server - ContainerManager proxy = (ContainerManager) rpc.getProxy( - ContainerManager.class, + ContainerManagementProtocol proxy = (ContainerManagementProtocol) rpc.getProxy( + ContainerManagementProtocol.class, NetUtils.getConnectAddress(server), conf); return proxy; } @@ -359,7 +359,7 @@ public class TestContainerLauncher { }; } - public class DummyContainerManager implements ContainerManager { + public class DummyContainerManager implements ContainerManagementProtocol { private ContainerStatus status = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java index 141e0b0bc30..c75ef92df9a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java @@ -46,7 +46,7 @@ import org.apache.hadoop.mapreduce.v2.app.MRApp; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType; import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher.EventType; import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; @@ -147,8 +147,8 @@ public class TestContainerLauncherImpl { EventHandler mockEventHandler = mock(EventHandler.class); when(mockContext.getEventHandler()).thenReturn(mockEventHandler); - ContainerManager mockCM = mock(ContainerManager.class); - when(mockRpc.getProxy(eq(ContainerManager.class), + ContainerManagementProtocol mockCM = mock(ContainerManagementProtocol.class); + when(mockRpc.getProxy(eq(ContainerManagementProtocol.class), any(InetSocketAddress.class), any(Configuration.class))) .thenReturn(mockCM); @@ -213,8 +213,8 @@ public class TestContainerLauncherImpl { EventHandler mockEventHandler = mock(EventHandler.class); when(mockContext.getEventHandler()).thenReturn(mockEventHandler); - ContainerManager mockCM = mock(ContainerManager.class); - when(mockRpc.getProxy(eq(ContainerManager.class), + ContainerManagementProtocol mockCM = mock(ContainerManagementProtocol.class); + when(mockRpc.getProxy(eq(ContainerManagementProtocol.class), any(InetSocketAddress.class), any(Configuration.class))) .thenReturn(mockCM); @@ -279,8 +279,8 @@ public class TestContainerLauncherImpl { EventHandler mockEventHandler = mock(EventHandler.class); when(mockContext.getEventHandler()).thenReturn(mockEventHandler); - ContainerManager mockCM = mock(ContainerManager.class); - when(mockRpc.getProxy(eq(ContainerManager.class), + ContainerManagementProtocol mockCM = mock(ContainerManagementProtocol.class); + when(mockRpc.getProxy(eq(ContainerManagementProtocol.class), any(InetSocketAddress.class), any(Configuration.class))) .thenReturn(mockCM); @@ -338,8 +338,8 @@ public class TestContainerLauncherImpl { EventHandler mockEventHandler = mock(EventHandler.class); when(mockContext.getEventHandler()).thenReturn(mockEventHandler); - ContainerManager mockCM = new ContainerManagerForTest(startLaunchBarrier, completeLaunchBarrier); - when(mockRpc.getProxy(eq(ContainerManager.class), + ContainerManagementProtocol mockCM = new ContainerManagerForTest(startLaunchBarrier, completeLaunchBarrier); + when(mockRpc.getProxy(eq(ContainerManagementProtocol.class), any(InetSocketAddress.class), any(Configuration.class))) .thenReturn(mockCM); @@ -417,7 +417,7 @@ public class TestContainerLauncherImpl { currentTime + 10000L, 123, currentTime)); } - private static class ContainerManagerForTest implements ContainerManager { + private static class ContainerManagerForTest implements ContainerManagementProtocol { private CyclicBarrier startLaunchBarrier; private CyclicBarrier completeLaunchBarrier; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java index 86cf98ece99..a7d1750bc54 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java @@ -30,7 +30,7 @@ import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.app.client.ClientService; import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.yarn.ClusterInfo; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -94,8 +94,8 @@ public class TestLocalContainerAllocator { } @Override - protected AMRMProtocol createSchedulerProxy() { - AMRMProtocol scheduler = mock(AMRMProtocol.class); + protected ApplicationMasterProtocol createSchedulerProxy() { + ApplicationMasterProtocol scheduler = mock(ApplicationMasterProtocol.class); try { when(scheduler.allocate(isA(AllocateRequest.class))) .thenThrow(RPCUtil.getRemoteException(new IOException("forcefail"))); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1329348432655_0001_conf.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1329348432655_0001_conf.xml index f3ec383f918..0bff22cdf69 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1329348432655_0001_conf.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1329348432655_0001_conf.xml @@ -258,7 +258,7 @@ mapreduce.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst mapreduce.tasktracker.http.address0.0.0.0:50060 mapreduce.jobhistory.webapp.address0.0.0.0:19888 -rpc.engine.org.apache.hadoop.yarn.proto.AMRMProtocol$AMRMProtocolService$BlockingInterfaceorg.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine +rpc.engine.org.apache.hadoop.yarn.proto.ApplicationMasterProtocol$ApplicationMasterProtocolService$BlockingInterfaceorg.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine yarn.ipc.rpc.classorg.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC mapreduce.job.nameSleep job kfs.blocksize67108864 diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java index dee5d183b8a..349a0194592 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java @@ -37,7 +37,7 @@ import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.v2.util.MRApps; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; @@ -54,7 +54,7 @@ public class ResourceMgrDelegate extends YarnClientImpl { private ApplicationId applicationId; /** - * Delegate responsible for communicating with the Resource Manager's {@link ClientRMProtocol}. + * Delegate responsible for communicating with the Resource Manager's {@link ApplicationClientProtocol}. * @param conf the configuration object. */ public ResourceMgrDelegate(YarnConfiguration conf) { 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 017b9938856..64bb45d71c8 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 @@ -68,7 +68,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobState; import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; @@ -223,7 +223,7 @@ public class TestClientRedirect { Assert.assertEquals(1, counters.countCounters()); } - class RMService extends AbstractService implements ClientRMProtocol { + class RMService extends AbstractService implements ApplicationClientProtocol { private String clientServiceBindAddress; InetSocketAddress clientBindAddress; private Server server; @@ -250,7 +250,7 @@ public class TestClientRedirect { // Kerberos if security is enabled, so no secretManager. YarnRPC rpc = YarnRPC.create(getConfig()); Configuration clientServerConf = new Configuration(getConfig()); - this.server = rpc.getServer(ClientRMProtocol.class, this, + this.server = rpc.getServer(ApplicationClientProtocol.class, this, clientBindAddress, clientServerConf, null, 1); this.server.start(); super.serviceStart(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java index 57b09c57dbd..816804d6719 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java @@ -26,7 +26,7 @@ import junit.framework.Assert; import org.apache.hadoop.mapreduce.JobStatus; import org.apache.hadoop.mapreduce.JobStatus.State; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; @@ -52,7 +52,7 @@ public class TestResourceMgrDelegate { */ @Test public void testGetRootQueues() throws IOException, InterruptedException { - final ClientRMProtocol applicationsManager = Mockito.mock(ClientRMProtocol.class); + final ApplicationClientProtocol applicationsManager = Mockito.mock(ApplicationClientProtocol.class); GetQueueInfoResponse response = Mockito.mock(GetQueueInfoResponse.class); org.apache.hadoop.yarn.api.records.QueueInfo queueInfo = Mockito.mock(org.apache.hadoop.yarn.api.records.QueueInfo.class); @@ -90,7 +90,7 @@ public class TestResourceMgrDelegate { @Test public void tesAllJobs() throws Exception { - final ClientRMProtocol applicationsManager = Mockito.mock(ClientRMProtocol.class); + final ApplicationClientProtocol applicationsManager = Mockito.mock(ApplicationClientProtocol.class); GetAllApplicationsResponse allApplicationsResponse = Records .newRecord(GetAllApplicationsResponse.class); List applications = new ArrayList(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java index e25f3cd24a5..0ccfcc306a1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java @@ -59,7 +59,7 @@ import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; @@ -197,7 +197,7 @@ public class TestYARNRunner extends TestCase { @Test(timeout=20000) public void testResourceMgrDelegate() throws Exception { /* we not want a mock of resource mgr delegate */ - final ClientRMProtocol clientRMProtocol = mock(ClientRMProtocol.class); + final ApplicationClientProtocol clientRMProtocol = mock(ApplicationClientProtocol.class); ResourceMgrDelegate delegate = new ResourceMgrDelegate(conf) { @Override protected void serviceStart() { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java index 43dd22778b1..289d0f21e5b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java @@ -34,7 +34,7 @@ import org.apache.hadoop.mapred.ResourceMgrDelegate; import org.apache.hadoop.mapred.YARNRunner; import org.apache.hadoop.mapreduce.protocol.ClientProtocol; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -104,7 +104,7 @@ public class TestYarnClientProtocolProvider extends TestCase { rmDTToken.setPassword(ByteBuffer.wrap("testcluster".getBytes())); rmDTToken.setService("0.0.0.0:8032"); getDTResponse.setRMDelegationToken(rmDTToken); - final ClientRMProtocol cRMProtocol = mock(ClientRMProtocol.class); + final ApplicationClientProtocol cRMProtocol = mock(ApplicationClientProtocol.class); when(cRMProtocol.getDelegationToken(any( GetDelegationTokenRequest.class))).thenReturn(getDTResponse); ResourceMgrDelegate rmgrDelegate = new ResourceMgrDelegate( diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 45ab9801abf..457c57e27e8 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -140,6 +140,12 @@ Release 2.1.0-beta - UNRELEASED YARN-823. Moved RMAdmin from yarn.client to yarn.client.cli and renamed it to be RMAdminCLI. (Jian He via vinodkv) + YARN-387. Renamed YARN protocols for consistency. + ClientRMProtocol -> ApplicationClientProtocol + AMRMProtocol -> ApplicationMasterProtocol + ContainerManager -> ContainerManagementProtocol + (vinodkv via acmurthy) + NEW FEATURES YARN-482. FS: Extend SchedulingMode to intermediate queues. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml index 50fc8051159..3c98e521c76 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml @@ -54,11 +54,11 @@ yarn_protos.proto yarn_service_protos.proto - AM_RM_protocol.proto - client_RM_protocol.proto - container_manager.proto + applicationmaster_protocol.proto + applicationclient_protocol.proto + containermanagement_protocol.proto yarn_server_resourcemanager_service_protos.proto - RMAdminProtocol.proto + resourcemanager_administration_protocol.proto ${project.build.directory}/generated-sources/java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java similarity index 99% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java index 593a8ea1b79..f9ed776eacb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java @@ -63,7 +63,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException; */ @Public @Stable -public interface ClientRMProtocol { +public interface ApplicationClientProtocol { /** *

The interface used by clients to obtain a new {@link ApplicationId} for * submitting new applications.

diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/AMRMProtocolPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocolPB.java similarity index 76% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/AMRMProtocolPB.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocolPB.java index d8f0ab77975..7f8cd38e9c7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/AMRMProtocolPB.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocolPB.java @@ -18,10 +18,10 @@ package org.apache.hadoop.yarn.api; import org.apache.hadoop.ipc.ProtocolInfo; -import org.apache.hadoop.yarn.proto.AMRMProtocol.AMRMProtocolService; +import org.apache.hadoop.yarn.proto.ApplicationClientProtocol.ApplicationClientProtocolService; -@ProtocolInfo(protocolName = "org.apache.hadoop.yarn.api.AMRMProtocolPB", +@ProtocolInfo(protocolName = "org.apache.hadoop.yarn.api.ApplicationClientProtocolPB", protocolVersion = 1) -public interface AMRMProtocolPB extends AMRMProtocolService.BlockingInterface { +public interface ApplicationClientProtocolPB extends ApplicationClientProtocolService.BlockingInterface { } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/AMRMProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocol.java similarity index 99% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/AMRMProtocol.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocol.java index b5f188df800..4baa8748bbe 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/AMRMProtocol.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocol.java @@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; */ @Public @Stable -public interface AMRMProtocol { +public interface ApplicationMasterProtocol { /** *

The interface used by a new ApplicationMaster to register diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/RMAdminProtocolPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocolPB.java similarity index 76% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/RMAdminProtocolPB.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocolPB.java index d8aaa1fc5e8..027a6f03238 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/RMAdminProtocolPB.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocolPB.java @@ -18,11 +18,10 @@ package org.apache.hadoop.yarn.api; import org.apache.hadoop.ipc.ProtocolInfo; -import org.apache.hadoop.yarn.proto.RMAdminProtocol.RMAdminProtocolService; +import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService; -@ProtocolInfo( - protocolName = "org.apache.hadoop.yarn.api.RMAdminProtocolPB", +@ProtocolInfo(protocolName = "org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB", protocolVersion = 1) -public interface RMAdminProtocolPB extends RMAdminProtocolService.BlockingInterface { +public interface ApplicationMasterProtocolPB extends ApplicationMasterProtocolService.BlockingInterface { } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java similarity index 99% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManager.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java index eab2e4654a7..5ae23a4a2ed 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java @@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException; */ @Public @Stable -public interface ContainerManager { +public interface ContainerManagementProtocol { /** *

The ApplicationMaster requests a NodeManager * to start a {@link Container} allocated to it using this interface. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagerPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocolPB.java similarity index 76% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagerPB.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocolPB.java index 306eaf637e3..86a9edcf3b5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagerPB.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocolPB.java @@ -18,11 +18,11 @@ package org.apache.hadoop.yarn.api; import org.apache.hadoop.ipc.ProtocolInfo; -import org.apache.hadoop.yarn.proto.ContainerManager.ContainerManagerService; +import org.apache.hadoop.yarn.proto.ContainerManagementProtocol.ContainerManagementProtocolService; @ProtocolInfo( - protocolName = "org.apache.hadoop.yarn.api.ContainerManagerPB", + protocolName = "org.apache.hadoop.yarn.api.ContainerManagementProtocolPB", protocolVersion = 1) -public interface ContainerManagerPB extends ContainerManagerService.BlockingInterface { +public interface ContainerManagementProtocolPB extends ContainerManagementProtocolService.BlockingInterface { } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/RMAdminProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocol.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/RMAdminProtocol.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocol.java index 7fba4de707f..76916b07ce0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/RMAdminProtocol.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocol.java @@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RefreshSuperUserGroupsConfigur import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsRequest; import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsResponse; -public interface RMAdminProtocol extends GetUserMappingsProtocol { +public interface ResourceManagerAdministrationProtocol extends GetUserMappingsProtocol { public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request) throws YarnException, IOException; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocolPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocolPB.java similarity index 72% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocolPB.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocolPB.java index d5930873efc..50427262eeb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocolPB.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocolPB.java @@ -18,10 +18,11 @@ package org.apache.hadoop.yarn.api; import org.apache.hadoop.ipc.ProtocolInfo; -import org.apache.hadoop.yarn.proto.ClientRMProtocol.ClientRMProtocolService; +import org.apache.hadoop.yarn.proto.ResourceManagerAdministrationProtocol.ResourceManagerAdministrationProtocolService; -@ProtocolInfo(protocolName = "org.apache.hadoop.yarn.api.ClientRMProtocolPB", +@ProtocolInfo( + protocolName = "org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocolPB", protocolVersion = 1) -public interface ClientRMProtocolPB extends ClientRMProtocolService.BlockingInterface { +public interface ResourceManagerAdministrationProtocolPB extends ResourceManagerAdministrationProtocolService.BlockingInterface { } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java index 763e1f0a350..15ab399ded3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java @@ -22,7 +22,7 @@ import java.util.List; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; import org.apache.hadoop.yarn.api.records.Container; @@ -53,7 +53,7 @@ import org.apache.hadoop.yarn.util.Records; * *

* - * @see AMRMProtocol#allocate(AllocateRequest) + * @see ApplicationMasterProtocol#allocate(AllocateRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java index 844924ef986..689368f25e2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java @@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.records.AMCommand; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerStatus; @@ -57,7 +57,7 @@ import org.apache.hadoop.yarn.util.Records; * *

* - * @see AMRMProtocol#allocate(AllocateRequest) + * @see ApplicationMasterProtocol#allocate(AllocateRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java index 0904a85d851..c67d8bdb1e3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.util.Records; @@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.util.Records; * *

* - * @see AMRMProtocol#finishApplicationMaster(FinishApplicationMasterRequest) + * @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest) */ public abstract class FinishApplicationMasterRequest { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java index 260be20cce9..68864ac4165 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records; * *

Currently, this is empty.

* - * @see AMRMProtocol#finishApplicationMaster(FinishApplicationMasterRequest) + * @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAllApplicationsRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAllApplicationsRequest.java index b97c8c35602..61cea28a285 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAllApplicationsRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAllApplicationsRequest.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records; * *

Currently, this is empty.

* - * @see ClientRMProtocol#getAllApplications(GetAllApplicationsRequest) + * @see ApplicationClientProtocol#getAllApplications(GetAllApplicationsRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAllApplicationsResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAllApplicationsResponse.java index b805e2693f1..523b4462728 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAllApplicationsResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAllApplicationsResponse.java @@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.util.Records; @@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.util.Records; * is running, RPC port, tracking URL, diagnostics, start time etc.

* * @see ApplicationReport - * @see ClientRMProtocol#getAllApplications(GetAllApplicationsRequest) + * @see ApplicationClientProtocol#getAllApplications(GetAllApplicationsRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationReportRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationReportRequest.java index d8ea51b420e..6355200d1c0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationReportRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationReportRequest.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.util.Records; @@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.util.Records; *

The request should include the {@link ApplicationId} of the * application.

* - * @see ClientRMProtocol#getApplicationReport(GetApplicationReportRequest) + * @see ApplicationClientProtocol#getApplicationReport(GetApplicationReportRequest) * @see ApplicationReport */ @Public diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationReportResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationReportResponse.java index 28daddec96d..78c62febe41 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationReportResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationReportResponse.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.util.Records; @@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.util.Records; * as user, queue, name, host on which the ApplicationMaster is * running, RPC port, tracking URL, diagnostics, start time etc.

* - * @see ClientRMProtocol#getApplicationReport(GetApplicationReportRequest) + * @see ApplicationClientProtocol#getApplicationReport(GetApplicationReportRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsRequest.java index 6ce4ffef66c..ba364dff057 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsRequest.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records; * *

Currently, this is empty.

* - * @see ClientRMProtocol#getClusterMetrics(GetClusterMetricsRequest) + * @see ApplicationClientProtocol#getClusterMetrics(GetClusterMetricsRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java index 3ad7d22d44c..7541ea7fa4a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; import org.apache.hadoop.yarn.util.Records; @@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.util.Records; * requesting cluster metrics.

* * @see YarnClusterMetrics - * @see ClientRMProtocol#getClusterMetrics(GetClusterMetricsRequest) + * @see ApplicationClientProtocol#getClusterMetrics(GetClusterMetricsRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodesRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodesRequest.java index 2c962261096..0c245ed9eef 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodesRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodesRequest.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records; * *

Currently, this is empty.

* - * @see ClientRMProtocol#getClusterNodes(GetClusterNodesRequest) + * @see ApplicationClientProtocol#getClusterNodes(GetClusterNodesRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodesResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodesResponse.java index 6d574b56006..a471f7efb0d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodesResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodesResponse.java @@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.util.Records; @@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.util.Records; * status etc. * * @see NodeReport - * @see ClientRMProtocol#getClusterNodes(GetClusterNodesRequest) + * @see ApplicationClientProtocol#getClusterNodes(GetClusterNodesRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusRequest.java index c621ca6852d..894139cbe9f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusRequest.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.util.Records; @@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records; *

The request sent by the ApplicationMaster to the * NodeManager to get {@link ContainerStatus} of a container.

* - * @see ContainerManager#getContainerStatus(GetContainerStatusRequest) + * @see ContainerManagementProtocol#getContainerStatus(GetContainerStatusRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusResponse.java index 61ee6bcd792..64f3d402873 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusResponse.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.util.Records; @@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.util.Records; * ApplicationMaster when asked to obtain the status * of a container.

* - * @see ContainerManager#getContainerStatus(GetContainerStatusRequest) + * @see ContainerManagementProtocol#getContainerStatus(GetContainerStatusRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationRequest.java index efa2b6ad77c..8ccbc8a9102 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationRequest.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.util.Records; @@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.util.Records; * *

Currently, this is empty.

* - * @see ClientRMProtocol#getNewApplication(GetNewApplicationRequest) + * @see ApplicationClientProtocol#getNewApplication(GetNewApplicationRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationResponse.java index e79c5b6810e..ee095b2dd53 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationResponse.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.util.Records; @@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.util.Records; *

The response sent by the ResourceManager to the client for * a request to get a new {@link ApplicationId} for submitting applications.

* - * @see ClientRMProtocol#getNewApplication(GetNewApplicationRequest) + * @see ApplicationClientProtocol#getNewApplication(GetNewApplicationRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java index 8cdf3c1f56a..33d4aac3f0f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java @@ -20,14 +20,14 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; /** *

The request sent by clients to get queue information * from the ResourceManager.

* - * @see ClientRMProtocol#getQueueInfo(GetQueueInfoRequest) + * @see ApplicationClientProtocol#getQueueInfo(GetQueueInfoRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java index 5137fdd6e1e..bef729a4f0a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.records.QueueInfo; import org.apache.hadoop.yarn.util.Records; @@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.util.Records; * .

* * @see QueueInfo - * @see ClientRMProtocol#getQueueInfo(GetQueueInfoRequest) + * @see ApplicationClientProtocol#getQueueInfo(GetQueueInfoRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueUserAclsInfoRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueUserAclsInfoRequest.java index d251ed0fe5f..e52033ce4bc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueUserAclsInfoRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueUserAclsInfoRequest.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records; * *

Currently, this is empty.

* - * @see ClientRMProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest) + * @see ApplicationClientProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueUserAclsInfoResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueUserAclsInfoResponse.java index 8db9f808ab0..dfb7a15eed9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueUserAclsInfoResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueUserAclsInfoResponse.java @@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; import org.apache.hadoop.yarn.util.Records; @@ -38,7 +38,7 @@ import org.apache.hadoop.yarn.util.Records; * * @see QueueACL * @see QueueUserACLInfo - * @see ClientRMProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest) + * @see ApplicationClientProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationRequest.java index ca54f30e18f..2630424380e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationRequest.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.util.Records; @@ -33,7 +33,7 @@ import org.apache.hadoop.yarn.util.Records; *

The request includes the {@link ApplicationId} of the application to be * aborted.

* - * @see ClientRMProtocol#forceKillApplication(KillApplicationRequest) + * @see ApplicationClientProtocol#forceKillApplication(KillApplicationRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java index 4bfae078a6d..9fa536d6b4e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records; * *

Currently it's empty.

* - * @see ClientRMProtocol#forceKillApplication(KillApplicationRequest) + * @see ApplicationClientProtocol#forceKillApplication(KillApplicationRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java index 123c598cf82..e5da5b29fc7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.util.Records; @@ -42,7 +42,7 @@ import org.apache.hadoop.yarn.util.Records; * *

* - * @see AMRMProtocol#registerApplicationMaster(RegisterApplicationMasterRequest) + * @see ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java index 58157105a82..15504d2307b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java @@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.util.Records; @@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.util.Records; * *

* - * @see AMRMProtocol#registerApplicationMaster(RegisterApplicationMasterRequest) + * @see ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java index 3341aac81f3..45c2f08cb24 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.util.Records; @@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.util.Records; * necessary binaries/jar/shared-objects etc. via the * {@link ContainerLaunchContext}.

* - * @see ContainerManager#startContainer(StartContainerRequest) + * @see ContainerManagementProtocol#startContainer(StartContainerRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerResponse.java index 5fe8963c6cb..4b00691db03 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerResponse.java @@ -23,7 +23,7 @@ import java.util.Map; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.util.Records; * ApplicationMaster when asked to start an * allocated container.

* - * @see ContainerManager#startContainer(StartContainerRequest) + * @see ContainerManagementProtocol#startContainer(StartContainerRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerRequest.java index 331ead55bf6..7b8368ea2e4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerRequest.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.Records; @@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.util.Records; *

The request sent by the ApplicationMaster to the * NodeManager to stop a container.

* - * @see ContainerManager#stopContainer(StopContainerRequest) + * @see ContainerManagementProtocol#stopContainer(StopContainerRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerResponse.java index ad8e6baa459..4bb7660770f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerResponse.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.util.Records; * *

Currently, this is empty.

* - * @see ContainerManager#stopContainer(StopContainerRequest) + * @see ContainerManagementProtocol#stopContainer(StopContainerRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SubmitApplicationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SubmitApplicationRequest.java index 94a7cf330f7..c93b89d6425 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SubmitApplicationRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SubmitApplicationRequest.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.Resource; @@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.util.Records; * {@link ContainerLaunchContext} for launching the * ApplicationMaster etc. * - * @see ClientRMProtocol#submitApplication(SubmitApplicationRequest) + * @see ApplicationClientProtocol#submitApplication(SubmitApplicationRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SubmitApplicationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SubmitApplicationResponse.java index c0d3b3bffb0..16c4bfad9bd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SubmitApplicationResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SubmitApplicationResponse.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records; * *

Currently, this is empty.

* - * @see ClientRMProtocol#submitApplication(SubmitApplicationRequest) + * @see ApplicationClientProtocol#submitApplication(SubmitApplicationRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java index 48f52bc93fc..500eea8521e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.util.Records; * *

* - * @see ClientRMProtocol#getApplicationReport(org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest) + * @see ApplicationClientProtocol#getApplicationReport(org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java index b4f6c044f7d..3d669b08c56 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.util.Records; *

* * @see ContainerLaunchContext - * @see ClientRMProtocol#submitApplication(org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest) + * @see ApplicationClientProtocol#submitApplication(org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java index 2a2050a754f..fb205f83344 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java @@ -22,8 +22,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.AMRMProtocol; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -58,9 +58,9 @@ import org.apache.hadoop.yarn.util.Records; * resource-negotiation and then talks to the NodManager to * start/stop containers.

* - * @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) - * @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) - * @see ContainerManager#stopContainer(org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest) + * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) + * @see ContainerManagementProtocol#stopContainer(org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest) */ @Public @Stable @@ -153,8 +153,8 @@ public abstract class Container implements Comparable { * are transparently handled by the framework - the allocated * Container includes the ContainerToken.

* - * @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) - * @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) + * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) * * @return ContainerToken for the container */ diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java index 59e7614bd1c..da2a164dfb2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java @@ -24,7 +24,7 @@ import java.util.Map; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.util.Records; * *

* - * @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) + * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java index 610e557b147..f2a3effa0e6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.records; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.util.Records; * @see LocalResourceVisibility * @see ContainerLaunchContext * @see ApplicationSubmissionContext - * @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) + * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java index 2529f12b384..30099716eab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.records; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; /** *

LocalResourceType specifies the type @@ -41,7 +41,7 @@ import org.apache.hadoop.yarn.api.ContainerManager; * @see LocalResource * @see ContainerLaunchContext * @see ApplicationSubmissionContext - * @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) + * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java index 232ab0a3531..32087eeeb49 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.records; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; /** *

LocalResourceVisibility specifies the visibility @@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.ContainerManager; * @see LocalResource * @see ContainerLaunchContext * @see ApplicationSubmissionContext - * @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) + * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java index afa7a1839c5..496d3307d16 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.util.Records; * *

* - * @see ClientRMProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest) + * @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java index cb8d7edee91..ccd91918cfd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.records; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; /** *

@@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol; *

* * @see QueueInfo - * @see ClientRMProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest) + * @see ApplicationClientProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java index 2d23eb9ce1f..3a58babb9a7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java @@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.util.Records; *

* * @see QueueState - * @see ClientRMProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest) + * @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java index 3d019669357..eb2cbaa5bd7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.records; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; /** *

State of a Queue.

@@ -33,7 +33,7 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol; *

* * @see QueueInfo - * @see ClientRMProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest) + * @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueUserACLInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueUserACLInfo.java index d81808e5a30..21d75fce974 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueUserACLInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueUserACLInfo.java @@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.util.Records; * the given user.

* * @see QueueACL - * @see ClientRMProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest) + * @see ApplicationClientProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java index 19f868f0738..5a85e7ce34a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java @@ -21,7 +21,7 @@ package org.apache.hadoop.yarn.api.records; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.util.Records; * capability to run their component tasks.

* * @see ResourceRequest - * @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceBlacklistRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceBlacklistRequest.java index 60f12c36f05..130d3610353 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceBlacklistRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceBlacklistRequest.java @@ -21,7 +21,7 @@ import java.util.List; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.util.Records; * for the application. * * @see ResourceRequest - * @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java index 7f119fda549..f9e71ebdcbd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java @@ -22,7 +22,7 @@ import java.io.Serializable; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -53,7 +53,7 @@ import org.apache.hadoop.yarn.util.Records; *

* * @see Resource - * @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index 3ef06e0a7f4..d6cb329cfcd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -606,21 +606,21 @@ public class YarnConfiguration extends Configuration { * YARN Service Level Authorization */ public static final String - YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER = + YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL = "security.resourcetracker.protocol.acl"; public static final String - YARN_SECURITY_SERVICE_AUTHORIZATION_CLIENT_RESOURCEMANAGER = - "security.client.resourcemanager.protocol.acl"; + YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONCLIENT_PROTOCOL = + "security.applicationclient.protocol.acl"; public static final String - YARN_SECURITY_SERVICE_AUTHORIZATION_ADMIN = - "security.admin.protocol.acl"; + YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCEMANAGER_ADMINISTRATION_PROTOCOL = + "security.resourcemanager-administration.protocol.acl"; public static final String - YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONMASTER_RESOURCEMANAGER = - "security.applicationmaster.resourcemanager.protocol.acl"; + YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONMASTER_PROTOCOL = + "security.applicationmaster.protocol.acl"; public static final String - YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGER = - "security.containermanager.protocol.acl"; + YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL = + "security.containermanagement.protocol.acl"; public static final String YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER = "security.resourcelocalizer.protocol.acl"; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/client_RM_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/client_RM_protocol.proto rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto index 5aa2380ae5f..1e0cad302df 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/client_RM_protocol.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto @@ -17,14 +17,14 @@ */ option java_package = "org.apache.hadoop.yarn.proto"; -option java_outer_classname = "ClientRMProtocol"; +option java_outer_classname = "ApplicationClientProtocol"; option java_generic_services = true; option java_generate_equals_and_hash = true; import "Security.proto"; import "yarn_service_protos.proto"; -service ClientRMProtocolService { +service ApplicationClientProtocolService { rpc getNewApplication (GetNewApplicationRequestProto) returns (GetNewApplicationResponseProto); rpc getApplicationReport (GetApplicationReportRequestProto) returns (GetApplicationReportResponseProto); rpc submitApplication (SubmitApplicationRequestProto) returns (SubmitApplicationResponseProto); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/AM_RM_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationmaster_protocol.proto similarity index 92% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/AM_RM_protocol.proto rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationmaster_protocol.proto index 744f957cd74..be896e4e40d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/AM_RM_protocol.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationmaster_protocol.proto @@ -17,14 +17,14 @@ */ option java_package = "org.apache.hadoop.yarn.proto"; -option java_outer_classname = "AMRMProtocol"; +option java_outer_classname = "ApplicationMasterProtocol"; option java_generic_services = true; option java_generate_equals_and_hash = true; import "yarn_service_protos.proto"; -service AMRMProtocolService { +service ApplicationMasterProtocolService { rpc registerApplicationMaster (RegisterApplicationMasterRequestProto) returns (RegisterApplicationMasterResponseProto); rpc finishApplicationMaster (FinishApplicationMasterRequestProto) returns (FinishApplicationMasterResponseProto); rpc allocate (AllocateRequestProto) returns (AllocateResponseProto); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/container_manager.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto similarity index 92% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/container_manager.proto rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto index 5c94f1e4255..a17131a4525 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/container_manager.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto @@ -17,13 +17,13 @@ */ option java_package = "org.apache.hadoop.yarn.proto"; -option java_outer_classname = "ContainerManager"; +option java_outer_classname = "ContainerManagementProtocol"; option java_generic_services = true; option java_generate_equals_and_hash = true; import "yarn_service_protos.proto"; -service ContainerManagerService { +service ContainerManagementProtocolService { rpc startContainer(StartContainerRequestProto) returns (StartContainerResponseProto); rpc stopContainer(StopContainerRequestProto) returns (StopContainerResponseProto); rpc getContainerStatus(GetContainerStatusRequestProto) returns (GetContainerStatusResponseProto); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/RMAdminProtocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/resourcemanager_administration_protocol.proto similarity index 93% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/RMAdminProtocol.proto rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/resourcemanager_administration_protocol.proto index b2aa46f1ef7..e1c4458553b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/RMAdminProtocol.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/resourcemanager_administration_protocol.proto @@ -17,13 +17,13 @@ */ option java_package = "org.apache.hadoop.yarn.proto"; -option java_outer_classname = "RMAdminProtocol"; +option java_outer_classname = "ResourceManagerAdministrationProtocol"; option java_generic_services = true; option java_generate_equals_and_hash = true; import "yarn_server_resourcemanager_service_protos.proto"; -service RMAdminProtocolService { +service ResourceManagerAdministrationProtocolService { rpc refreshQueues(RefreshQueuesRequestProto) returns (RefreshQueuesResponseProto); rpc refreshNodes(RefreshNodesRequestProto) returns (RefreshNodesResponseProto); rpc refreshSuperUserGroupsConfiguration(RefreshSuperUserGroupsConfigurationRequestProto) returns (RefreshSuperUserGroupsConfigurationResponseProto); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java index 879214ad775..64b4d6f21c8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java @@ -43,10 +43,10 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; -import org.apache.hadoop.yarn.api.ContainerManager; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; @@ -99,7 +99,7 @@ import org.apache.hadoop.yarn.util.Records; * The ApplicationMaster needs to send a heartbeat to the * ResourceManager at regular intervals to inform the * ResourceManager that it is up and alive. The - * {@link AMRMProtocol#allocate} to the ResourceManager from the + * {@link ApplicationMasterProtocol#allocate} to the ResourceManager from the * ApplicationMaster acts as a heartbeat. * *

@@ -118,15 +118,15 @@ import org.apache.hadoop.yarn.util.Records; * up the necessary launch context via {@link ContainerLaunchContext} to specify * the allocated container id, local resources required by the executable, the * environment to be setup for the executable, commands to execute, etc. and - * submit a {@link StartContainerRequest} to the {@link ContainerManager} to + * submit a {@link StartContainerRequest} to the {@link ContainerManagementProtocol} to * launch and execute the defined commands on the given allocated container. *

* *

* The ApplicationMaster can monitor the launched container by * either querying the ResourceManager using - * {@link AMRMProtocol#allocate} to get updates on completed containers or via - * the {@link ContainerManager} by querying for the status of the allocated + * {@link ApplicationMasterProtocol#allocate} to get updates on completed containers or via + * the {@link ContainerManagementProtocol} by querying for the status of the allocated * container's {@link ContainerId}. * *

@@ -707,7 +707,7 @@ public class ApplicationMaster { } /** - * Thread to connect to the {@link ContainerManager} and launch the container + * Thread to connect to the {@link ContainerManagementProtocol} and launch the container * that will execute the shell command. */ private class LaunchContainerRunnable implements Runnable { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java index cec24c29ea8..1fae8fdfe95 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java @@ -41,7 +41,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -75,7 +75,7 @@ import org.apache.hadoop.yarn.util.Records; *

This client is meant to act as an example on how to write yarn-based applications.

* *

To submit an application, a client first needs to connect to the ResourceManager - * aka ApplicationsManager or ASM via the {@link ClientRMProtocol}. The {@link ClientRMProtocol} + * aka ApplicationsManager or ASM via the {@link ApplicationClientProtocol}. The {@link ApplicationClientProtocol} * provides a way for the client to get access to cluster information and to request for a * new {@link ApplicationId}.

* diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java index f76ea5c6e9f..08c4b6d6f7c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java @@ -43,7 +43,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; @@ -85,7 +85,7 @@ public class AMRMClientImpl private int lastResponseId = 0; private ConcurrentHashMap nmTokens; - protected AMRMProtocol rmClient; + protected ApplicationMasterProtocol rmClient; protected final ApplicationAttemptId appAttemptId; protected Resource clusterAvailableResources; protected int clusterNodeCount; @@ -185,10 +185,10 @@ public class AMRMClientImpl } // CurrentUser should already have AMToken loaded. - rmClient = currentUser.doAs(new PrivilegedAction() { + rmClient = currentUser.doAs(new PrivilegedAction() { @Override - public AMRMProtocol run() { - return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class, rmAddress, + public ApplicationMasterProtocol run() { + return (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class, rmAddress, conf); } }); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java index b1be764ccda..65bdcec14bb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java @@ -32,7 +32,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; @@ -160,7 +160,7 @@ public class NMClientImpl extends AbstractService implements NMClient { private ContainerId containerId; private NodeId nodeId; private Token containerToken; - private ContainerManager containerManager; + private ContainerManagementProtocol containerManager; public NMCommunicator(ContainerId containerId, NodeId nodeId, Token containerToken) { @@ -186,10 +186,10 @@ public class NMClientImpl extends AbstractService implements NMClient { currentUser.addToken(token); containerManager = currentUser - .doAs(new PrivilegedAction() { + .doAs(new PrivilegedAction() { @Override - public ContainerManager run() { - return (ContainerManager) rpc.getProxy(ContainerManager.class, + public ContainerManagementProtocol run() { + return (ContainerManagementProtocol) rpc.getProxy(ContainerManagementProtocol.class, containerAddress, getConfig()); } }); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java index 48be1a329c4..0717fa4a690 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java @@ -30,7 +30,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; @@ -68,7 +68,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient { private static final Log LOG = LogFactory.getLog(YarnClientImpl.class); - protected ClientRMProtocol rmClient; + protected ApplicationClientProtocol rmClient; protected InetSocketAddress rmAddress; protected long statePollIntervalMillis; @@ -107,8 +107,8 @@ public class YarnClientImpl extends AbstractService implements YarnClient { protected void serviceStart() throws Exception { YarnRPC rpc = YarnRPC.create(getConfig()); - this.rmClient = (ClientRMProtocol) rpc.getProxy( - ClientRMProtocol.class, rmAddress, getConfig()); + this.rmClient = (ApplicationClientProtocol) rpc.getProxy( + ApplicationClientProtocol.class, rmAddress, getConfig()); if (LOG.isDebugEnabled()) { LOG.debug("Connecting to ResourceManager at " + rmAddress); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java index 6a8c218c291..060ff4bc1f4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java @@ -29,7 +29,7 @@ import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.apache.hadoop.yarn.api.RMAdminProtocol; +import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocol; import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsRequest; import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesRequest; import org.apache.hadoop.yarn.api.protocolrecords.RefreshQueuesRequest; @@ -165,7 +165,7 @@ public class RMAdminCLI extends Configured implements Tool { return UserGroupInformation.getCurrentUser(); } - private RMAdminProtocol createAdminProtocol() throws IOException { + private ResourceManagerAdministrationProtocol createAdminProtocol() throws IOException { // Get the current configuration final YarnConfiguration conf = new YarnConfiguration(getConf()); @@ -176,11 +176,11 @@ public class RMAdminCLI extends Configured implements Tool { YarnConfiguration.DEFAULT_RM_ADMIN_PORT); final YarnRPC rpc = YarnRPC.create(conf); - RMAdminProtocol adminProtocol = - getUGI(conf).doAs(new PrivilegedAction() { + ResourceManagerAdministrationProtocol adminProtocol = + getUGI(conf).doAs(new PrivilegedAction() { @Override - public RMAdminProtocol run() { - return (RMAdminProtocol) rpc.getProxy(RMAdminProtocol.class, + public ResourceManagerAdministrationProtocol run() { + return (ResourceManagerAdministrationProtocol) rpc.getProxy(ResourceManagerAdministrationProtocol.class, addr, conf); } }); @@ -190,7 +190,7 @@ public class RMAdminCLI extends Configured implements Tool { private int refreshQueues() throws IOException, YarnException { // Refresh the queue properties - RMAdminProtocol adminProtocol = createAdminProtocol(); + ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol(); RefreshQueuesRequest request = recordFactory.newRecordInstance(RefreshQueuesRequest.class); adminProtocol.refreshQueues(request); @@ -199,7 +199,7 @@ public class RMAdminCLI extends Configured implements Tool { private int refreshNodes() throws IOException, YarnException { // Refresh the nodes - RMAdminProtocol adminProtocol = createAdminProtocol(); + ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol(); RefreshNodesRequest request = recordFactory.newRecordInstance(RefreshNodesRequest.class); adminProtocol.refreshNodes(request); @@ -209,7 +209,7 @@ public class RMAdminCLI extends Configured implements Tool { private int refreshUserToGroupsMappings() throws IOException, YarnException { // Refresh the user-to-groups mappings - RMAdminProtocol adminProtocol = createAdminProtocol(); + ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol(); RefreshUserToGroupsMappingsRequest request = recordFactory.newRecordInstance(RefreshUserToGroupsMappingsRequest.class); adminProtocol.refreshUserToGroupsMappings(request); @@ -219,7 +219,7 @@ public class RMAdminCLI extends Configured implements Tool { private int refreshSuperUserGroupsConfiguration() throws IOException, YarnException { // Refresh the super-user groups - RMAdminProtocol adminProtocol = createAdminProtocol(); + ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol(); RefreshSuperUserGroupsConfigurationRequest request = recordFactory.newRecordInstance(RefreshSuperUserGroupsConfigurationRequest.class); adminProtocol.refreshSuperUserGroupsConfiguration(request); @@ -228,7 +228,7 @@ public class RMAdminCLI extends Configured implements Tool { private int refreshAdminAcls() throws IOException, YarnException { // Refresh the admin acls - RMAdminProtocol adminProtocol = createAdminProtocol(); + ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol(); RefreshAdminAclsRequest request = recordFactory.newRecordInstance(RefreshAdminAclsRequest.class); adminProtocol.refreshAdminAcls(request); @@ -237,7 +237,7 @@ public class RMAdminCLI extends Configured implements Tool { private int refreshServiceAcls() throws IOException, YarnException { // Refresh the service acls - RMAdminProtocol adminProtocol = createAdminProtocol(); + ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol(); RefreshServiceAclsRequest request = recordFactory.newRecordInstance(RefreshServiceAclsRequest.class); adminProtocol.refreshServiceAcls(request); @@ -246,7 +246,7 @@ public class RMAdminCLI extends Configured implements Tool { private int getGroups(String[] usernames) throws IOException { // Get groups users belongs to - RMAdminProtocol adminProtocol = createAdminProtocol(); + ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol(); if (usernames.length == 0) { usernames = new String[] { UserGroupInformation.getCurrentUser().getUserName() }; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java index 8827e0803db..e9c338efd48 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java @@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.tools.GetGroupsBase; import org.apache.hadoop.tools.GetUserMappingsProtocol; import org.apache.hadoop.util.ToolRunner; -import org.apache.hadoop.yarn.api.RMAdminProtocol; +import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocol; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.ipc.YarnRPC; @@ -63,8 +63,8 @@ public class GetGroupsForTesting extends GetGroupsBase { YarnConfiguration.DEFAULT_RM_ADMIN_PORT); final YarnRPC rpc = YarnRPC.create(conf); - RMAdminProtocol adminProtocol = (RMAdminProtocol) rpc.getProxy( - RMAdminProtocol.class, addr, getConf()); + ResourceManagerAdministrationProtocol adminProtocol = (ResourceManagerAdministrationProtocol) rpc.getProxy( + ResourceManagerAdministrationProtocol.class, addr, getConf()); return adminProtocol; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java index 4de458e39e5..da1d28a2495 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java @@ -35,7 +35,7 @@ import java.util.concurrent.ConcurrentHashMap; import junit.framework.Assert; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; @@ -550,9 +550,9 @@ public class TestAMRMClient { snoopRequest = amClient.ask.iterator().next(); assertTrue(snoopRequest.getNumContainers() == 2); - AMRMProtocol realRM = amClient.rmClient; + ApplicationMasterProtocol realRM = amClient.rmClient; try { - AMRMProtocol mockRM = mock(AMRMProtocol.class); + ApplicationMasterProtocol mockRM = mock(ApplicationMasterProtocol.class); when(mockRM.allocate(any(AllocateRequest.class))).thenAnswer( new Answer() { public AllocateResponse answer(InvocationOnMock invocation) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java index fbc876aa45b..2ee68b425be 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java @@ -30,7 +30,7 @@ import java.util.HashMap; import junit.framework.Assert; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; import org.apache.hadoop.yarn.api.records.ApplicationAccessType; @@ -146,7 +146,7 @@ public class TestYarnClient { @Override public void start() { - rmClient = mock(ClientRMProtocol.class); + rmClient = mock(ApplicationClientProtocol.class); GetApplicationReportResponse mockResponse = mock(GetApplicationReportResponse.class); mockReport = mock(ApplicationReport.class); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java index 83943242fcd..70b4e5bb070 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java @@ -28,8 +28,8 @@ import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto; import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto; import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto; -import org.apache.hadoop.yarn.api.ClientRMProtocol; -import org.apache.hadoop.yarn.api.ClientRMProtocolPB; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocolPB; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; @@ -92,16 +92,16 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestPr import com.google.protobuf.ServiceException; -public class ClientRMProtocolPBClientImpl implements ClientRMProtocol, +public class ApplicationClientProtocolPBClientImpl implements ApplicationClientProtocol, Closeable { - private ClientRMProtocolPB proxy; + private ApplicationClientProtocolPB proxy; - public ClientRMProtocolPBClientImpl(long clientVersion, + public ApplicationClientProtocolPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException { - RPC.setProtocolEngine(conf, ClientRMProtocolPB.class, + RPC.setProtocolEngine(conf, ApplicationClientProtocolPB.class, ProtobufRpcEngine.class); - proxy = RPC.getProxy(ClientRMProtocolPB.class, clientVersion, addr, conf); + proxy = RPC.getProxy(ApplicationClientProtocolPB.class, clientVersion, addr, conf); } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/AMRMProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationMasterProtocolPBClientImpl.java similarity index 88% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/AMRMProtocolPBClientImpl.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationMasterProtocolPBClientImpl.java index 064fe82d177..a131c9730ba 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/AMRMProtocolPBClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationMasterProtocolPBClientImpl.java @@ -25,8 +25,8 @@ import java.net.InetSocketAddress; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.yarn.api.AMRMProtocol; -import org.apache.hadoop.yarn.api.AMRMProtocolPB; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; @@ -47,15 +47,15 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterR import com.google.protobuf.ServiceException; -public class AMRMProtocolPBClientImpl implements AMRMProtocol, Closeable { +public class ApplicationMasterProtocolPBClientImpl implements ApplicationMasterProtocol, Closeable { - private AMRMProtocolPB proxy; + private ApplicationMasterProtocolPB proxy; - public AMRMProtocolPBClientImpl(long clientVersion, InetSocketAddress addr, + public ApplicationMasterProtocolPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException { - RPC.setProtocolEngine(conf, AMRMProtocolPB.class, ProtobufRpcEngine.class); + RPC.setProtocolEngine(conf, ApplicationMasterProtocolPB.class, ProtobufRpcEngine.class); proxy = - (AMRMProtocolPB) RPC.getProxy(AMRMProtocolPB.class, clientVersion, + (ApplicationMasterProtocolPB) RPC.getProxy(ApplicationMasterProtocolPB.class, clientVersion, addr, conf); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java similarity index 90% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java index 89213755e02..f5bac4cbb62 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java @@ -27,8 +27,8 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.ContainerManager; -import org.apache.hadoop.yarn.api.ContainerManagerPB; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; +import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; @@ -50,7 +50,7 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerRequestProto; import com.google.protobuf.ServiceException; -public class ContainerManagerPBClientImpl implements ContainerManager, +public class ContainerManagementProtocolPBClientImpl implements ContainerManagementProtocol, Closeable { // Not a documented config. Only used for tests @@ -62,17 +62,17 @@ public class ContainerManagerPBClientImpl implements ContainerManager, */ static final int DEFAULT_COMMAND_TIMEOUT = 60000; - private ContainerManagerPB proxy; + private ContainerManagementProtocolPB proxy; - public ContainerManagerPBClientImpl(long clientVersion, + public ContainerManagementProtocolPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException { - RPC.setProtocolEngine(conf, ContainerManagerPB.class, + RPC.setProtocolEngine(conf, ContainerManagementProtocolPB.class, ProtobufRpcEngine.class); UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); int expireIntvl = conf.getInt(NM_COMMAND_TIMEOUT, DEFAULT_COMMAND_TIMEOUT); proxy = - (ContainerManagerPB) RPC.getProxy(ContainerManagerPB.class, + (ContainerManagementProtocolPB) RPC.getProxy(ContainerManagementProtocolPB.class, clientVersion, addr, ugi, conf, NetUtils.getDefaultSocketFactory(conf), expireIntvl); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java similarity index 92% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java index 29b51a6e12b..746f79937cd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java @@ -26,8 +26,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.ProtobufHelper; import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.yarn.api.RMAdminProtocol; -import org.apache.hadoop.yarn.api.RMAdminProtocolPB; +import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocol; +import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocolPB; import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsRequest; import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsResponse; import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesRequest; @@ -66,16 +66,16 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Refre import com.google.protobuf.ServiceException; -public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable { +public class ResourceManagerAdministrationProtocolPBClientImpl implements ResourceManagerAdministrationProtocol, Closeable { - private RMAdminProtocolPB proxy; + private ResourceManagerAdministrationProtocolPB proxy; - public RMAdminProtocolPBClientImpl(long clientVersion, InetSocketAddress addr, + public ResourceManagerAdministrationProtocolPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException { - RPC.setProtocolEngine(conf, RMAdminProtocolPB.class, + RPC.setProtocolEngine(conf, ResourceManagerAdministrationProtocolPB.class, ProtobufRpcEngine.class); - proxy = (RMAdminProtocolPB)RPC.getProxy( - RMAdminProtocolPB.class, clientVersion, addr, conf); + proxy = (ResourceManagerAdministrationProtocolPB)RPC.getProxy( + ResourceManagerAdministrationProtocolPB.class, clientVersion, addr, conf); } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java index caad876e803..0f9e4b5b7e2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java @@ -26,8 +26,8 @@ import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequest import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto; import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto; import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenResponseProto; -import org.apache.hadoop.yarn.api.ClientRMProtocol; -import org.apache.hadoop.yarn.api.ClientRMProtocolPB; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocolPB; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; @@ -87,11 +87,11 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationResponseP import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; -public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB { +public class ApplicationClientProtocolPBServiceImpl implements ApplicationClientProtocolPB { - private ClientRMProtocol real; + private ApplicationClientProtocol real; - public ClientRMProtocolPBServiceImpl(ClientRMProtocol impl) { + public ApplicationClientProtocolPBServiceImpl(ApplicationClientProtocol impl) { this.real = impl; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/AMRMProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationMasterProtocolPBServiceImpl.java similarity index 92% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/AMRMProtocolPBServiceImpl.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationMasterProtocolPBServiceImpl.java index dae61336a80..d25c4d10326 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/AMRMProtocolPBServiceImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationMasterProtocolPBServiceImpl.java @@ -20,8 +20,8 @@ package org.apache.hadoop.yarn.api.impl.pb.service; import java.io.IOException; -import org.apache.hadoop.yarn.api.AMRMProtocol; -import org.apache.hadoop.yarn.api.AMRMProtocolPB; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; @@ -42,11 +42,11 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterR import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; -public class AMRMProtocolPBServiceImpl implements AMRMProtocolPB { +public class ApplicationMasterProtocolPBServiceImpl implements ApplicationMasterProtocolPB { - private AMRMProtocol real; + private ApplicationMasterProtocol real; - public AMRMProtocolPBServiceImpl(AMRMProtocol impl) { + public ApplicationMasterProtocolPBServiceImpl(ApplicationMasterProtocol impl) { this.real = impl; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagerPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java similarity index 91% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagerPBServiceImpl.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java index 398fce6ead2..07aa10a1ddb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagerPBServiceImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java @@ -20,8 +20,8 @@ package org.apache.hadoop.yarn.api.impl.pb.service; import java.io.IOException; -import org.apache.hadoop.yarn.api.ContainerManager; -import org.apache.hadoop.yarn.api.ContainerManagerPB; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; +import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse; import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse; @@ -42,11 +42,11 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerResponseProto import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; -public class ContainerManagerPBServiceImpl implements ContainerManagerPB { +public class ContainerManagementProtocolPBServiceImpl implements ContainerManagementProtocolPB { - private ContainerManager real; + private ContainerManagementProtocol real; - public ContainerManagerPBServiceImpl(ContainerManager impl) { + public ContainerManagementProtocolPBServiceImpl(ContainerManagementProtocol impl) { this.real = impl; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java similarity index 94% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java index 5450e713096..39f4c7a8d5a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java @@ -20,8 +20,8 @@ package org.apache.hadoop.yarn.api.impl.pb.service; import java.io.IOException; -import org.apache.hadoop.yarn.api.RMAdminProtocol; -import org.apache.hadoop.yarn.api.RMAdminProtocolPB; +import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocol; +import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocolPB; import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsResponse; import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesResponse; import org.apache.hadoop.yarn.api.protocolrecords.RefreshQueuesResponse; @@ -46,11 +46,11 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.*; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; -public class RMAdminProtocolPBServiceImpl implements RMAdminProtocolPB { +public class ResourceManagerAdministrationProtocolPBServiceImpl implements ResourceManagerAdministrationProtocolPB { - private RMAdminProtocol real; + private ResourceManagerAdministrationProtocol real; - public RMAdminProtocolPBServiceImpl(RMAdminProtocol impl) { + public ResourceManagerAdministrationProtocolPBServiceImpl(ResourceManagerAdministrationProtocol impl) { this.real = impl; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerManagerSecurityInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerManagerSecurityInfo.java index c7112e3c01f..6c15003d6ba 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerManagerSecurityInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerManagerSecurityInfo.java @@ -26,7 +26,7 @@ import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenInfo; import org.apache.hadoop.security.token.TokenSelector; -import org.apache.hadoop.yarn.api.ContainerManagerPB; +import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB; public class ContainerManagerSecurityInfo extends SecurityInfo { @@ -38,7 +38,7 @@ public class ContainerManagerSecurityInfo extends SecurityInfo { @Override public TokenInfo getTokenInfo(Class protocol, Configuration conf) { if (!protocol - .equals(ContainerManagerPB.class)) { + .equals(ContainerManagementProtocolPB.class)) { return null; } return new TokenInfo() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/SchedulerSecurityInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/SchedulerSecurityInfo.java index 583e2f46e12..db51c8c0344 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/SchedulerSecurityInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/SchedulerSecurityInfo.java @@ -26,7 +26,7 @@ import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenInfo; import org.apache.hadoop.security.token.TokenSelector; -import org.apache.hadoop.yarn.api.AMRMProtocolPB; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB; public class SchedulerSecurityInfo extends SecurityInfo { @@ -37,7 +37,7 @@ public class SchedulerSecurityInfo extends SecurityInfo { @Override public TokenInfo getTokenInfo(Class protocol, Configuration conf) { - if (!protocol.equals(AMRMProtocolPB.class)) { + if (!protocol.equals(ApplicationMasterProtocolPB.class)) { return null; } return new TokenInfo() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java index 1e738824397..8efb186a478 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java @@ -24,14 +24,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.KerberosInfo; import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.token.TokenInfo; -import org.apache.hadoop.yarn.api.RMAdminProtocolPB; +import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocolPB; import org.apache.hadoop.yarn.conf.YarnConfiguration; public class AdminSecurityInfo extends SecurityInfo { @Override public KerberosInfo getKerberosInfo(Class protocol, Configuration conf) { - if (!protocol.equals(RMAdminProtocolPB.class)) { + if (!protocol.equals(ResourceManagerAdministrationProtocolPB.class)) { return null; } return new KerberosInfo() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientRMSecurityInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientRMSecurityInfo.java index 2dcefd9c00b..340da16e10b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientRMSecurityInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientRMSecurityInfo.java @@ -26,7 +26,7 @@ import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenInfo; import org.apache.hadoop.security.token.TokenSelector; -import org.apache.hadoop.yarn.api.ClientRMProtocolPB; +import org.apache.hadoop.yarn.api.ApplicationClientProtocolPB; import org.apache.hadoop.yarn.conf.YarnConfiguration; public class ClientRMSecurityInfo extends SecurityInfo { @@ -34,7 +34,7 @@ public class ClientRMSecurityInfo extends SecurityInfo { @Override public KerberosInfo getKerberosInfo(Class protocol, Configuration conf) { if (!protocol - .equals(ClientRMProtocolPB.class)) { + .equals(ApplicationClientProtocolPB.class)) { return null; } return new KerberosInfo() { @@ -59,7 +59,7 @@ public class ClientRMSecurityInfo extends SecurityInfo { @Override public TokenInfo getTokenInfo(Class protocol, Configuration conf) { if (!protocol - .equals(ClientRMProtocolPB.class)) { + .equals(ApplicationClientProtocolPB.class)) { return null; } return new TokenInfo() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java index 7ec3b9dc2a6..094c133860f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java @@ -34,7 +34,7 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenRenewer; import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier; import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -97,7 +97,7 @@ public class RMDelegationTokenIdentifier extends AbstractDelegationTokenIdentifi @Override public long renew(Token token, Configuration conf) throws IOException, InterruptedException { - final ClientRMProtocol rmClient = getRmClient(token, conf); + final ApplicationClientProtocol rmClient = getRmClient(token, conf); if (rmClient != null) { try { RenewDelegationTokenRequest request = @@ -119,7 +119,7 @@ public class RMDelegationTokenIdentifier extends AbstractDelegationTokenIdentifi @Override public void cancel(Token token, Configuration conf) throws IOException, InterruptedException { - final ClientRMProtocol rmClient = getRmClient(token, conf); + final ApplicationClientProtocol rmClient = getRmClient(token, conf); if (rmClient != null) { try { CancelDelegationTokenRequest request = @@ -137,7 +137,7 @@ public class RMDelegationTokenIdentifier extends AbstractDelegationTokenIdentifi } } - private static ClientRMProtocol getRmClient(Token token, + private static ApplicationClientProtocol getRmClient(Token token, Configuration conf) { InetSocketAddress addr = SecurityUtil.getTokenServiceAddr(token); if (localSecretManager != null) { @@ -152,7 +152,7 @@ public class RMDelegationTokenIdentifier extends AbstractDelegationTokenIdentifi } } final YarnRPC rpc = YarnRPC.create(conf); - return (ClientRMProtocol)rpc.getProxy(ClientRMProtocol.class, addr, conf); + return (ApplicationClientProtocol)rpc.getProxy(ApplicationClientProtocol.class, addr, conf); } // get renewer so we can always renew our own tokens diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java index 5d0964bef67..0c633f169b7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java @@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.StringUtils; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; @@ -79,13 +79,13 @@ public class TestContainerLaunchRPC { YarnRPC rpc = YarnRPC.create(conf); String bindAddr = "localhost:0"; InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); - Server server = rpc.getServer(ContainerManager.class, + Server server = rpc.getServer(ContainerManagementProtocol.class, new DummyContainerManager(), addr, conf, null, 1); server.start(); try { - ContainerManager proxy = (ContainerManager) rpc.getProxy( - ContainerManager.class, + ContainerManagementProtocol proxy = (ContainerManagementProtocol) rpc.getProxy( + ContainerManagementProtocol.class, server.getListenerAddress(), conf); ContainerLaunchContext containerLaunchContext = recordFactory .newRecordInstance(ContainerLaunchContext.class); @@ -124,7 +124,7 @@ public class TestContainerLaunchRPC { Assert.fail("timeout exception should have occurred!"); } - public class DummyContainerManager implements ContainerManager { + public class DummyContainerManager implements ContainerManagementProtocol { private ContainerStatus status = null; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java index b5a5141cee5..68aeb563fcc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java @@ -30,9 +30,9 @@ import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.yarn.api.ClientRMProtocol; -import org.apache.hadoop.yarn.api.ContainerManager; -import org.apache.hadoop.yarn.api.ContainerManagerPB; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; +import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; @@ -74,13 +74,13 @@ public class TestRPC { YarnRPC rpc = YarnRPC.create(conf); String bindAddr = "localhost:0"; InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); - Server server = rpc.getServer(ContainerManager.class, + Server server = rpc.getServer(ContainerManagementProtocol.class, new DummyContainerManager(), addr, conf, null, 1); server.start(); // Any unrelated protocol would do - ClientRMProtocol proxy = (ClientRMProtocol) rpc.getProxy( - ClientRMProtocol.class, NetUtils.getConnectAddress(server), conf); + ApplicationClientProtocol proxy = (ApplicationClientProtocol) rpc.getProxy( + ApplicationClientProtocol.class, NetUtils.getConnectAddress(server), conf); try { proxy.getNewApplication(Records @@ -89,8 +89,8 @@ public class TestRPC { } catch (YarnException e) { Assert.assertTrue(e.getMessage().matches( "Unknown method getNewApplication called on.*" - + "org.apache.hadoop.yarn.proto.ClientRMProtocol" - + "\\$ClientRMProtocolService\\$BlockingInterface protocol.")); + + "org.apache.hadoop.yarn.proto.ApplicationClientProtocol" + + "\\$ApplicationClientProtocolService\\$BlockingInterface protocol.")); } catch (Exception e) { e.printStackTrace(); } @@ -107,12 +107,12 @@ public class TestRPC { YarnRPC rpc = YarnRPC.create(conf); String bindAddr = "localhost:0"; InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); - Server server = rpc.getServer(ContainerManager.class, + Server server = rpc.getServer(ContainerManagementProtocol.class, new DummyContainerManager(), addr, conf, null, 1); server.start(); - RPC.setProtocolEngine(conf, ContainerManagerPB.class, ProtobufRpcEngine.class); - ContainerManager proxy = (ContainerManager) - rpc.getProxy(ContainerManager.class, + RPC.setProtocolEngine(conf, ContainerManagementProtocolPB.class, ProtobufRpcEngine.class); + ContainerManagementProtocol proxy = (ContainerManagementProtocol) + rpc.getProxy(ContainerManagementProtocol.class, NetUtils.getConnectAddress(server), conf); ContainerLaunchContext containerLaunchContext = recordFactory.newRecordInstance(ContainerLaunchContext.class); @@ -162,7 +162,7 @@ public class TestRPC { Assert.assertEquals(ContainerState.RUNNING, status.getState()); } - public class DummyContainerManager implements ContainerManager { + public class DummyContainerManager implements ContainerManagementProtocol { private ContainerStatus status = null; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPCFactories.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPCFactories.java index b6aa65e3944..a3c2cce4436 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPCFactories.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPCFactories.java @@ -26,7 +26,7 @@ import junit.framework.Assert; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; @@ -55,12 +55,12 @@ public class TestRPCFactories { private void testPbServerFactory() { InetSocketAddress addr = new InetSocketAddress(0); Configuration conf = new Configuration(); - AMRMProtocol instance = new AMRMProtocolTestImpl(); + ApplicationMasterProtocol instance = new AMRMProtocolTestImpl(); Server server = null; try { server = RpcServerFactoryPBImpl.get().getServer( - AMRMProtocol.class, instance, addr, conf, null, 1); + ApplicationMasterProtocol.class, instance, addr, conf, null, 1); server.start(); } catch (YarnRuntimeException e) { e.printStackTrace(); @@ -77,19 +77,19 @@ public class TestRPCFactories { InetSocketAddress addr = new InetSocketAddress(0); System.err.println(addr.getHostName() + addr.getPort()); Configuration conf = new Configuration(); - AMRMProtocol instance = new AMRMProtocolTestImpl(); + ApplicationMasterProtocol instance = new AMRMProtocolTestImpl(); Server server = null; try { server = RpcServerFactoryPBImpl.get().getServer( - AMRMProtocol.class, instance, addr, conf, null, 1); + ApplicationMasterProtocol.class, instance, addr, conf, null, 1); server.start(); System.err.println(server.getListenerAddress()); System.err.println(NetUtils.getConnectAddress(server)); - AMRMProtocol amrmClient = null; + ApplicationMasterProtocol amrmClient = null; try { - amrmClient = (AMRMProtocol) RpcClientFactoryPBImpl.get().getClient(AMRMProtocol.class, 1, NetUtils.getConnectAddress(server), conf); + amrmClient = (ApplicationMasterProtocol) RpcClientFactoryPBImpl.get().getClient(ApplicationMasterProtocol.class, 1, NetUtils.getConnectAddress(server), conf); } catch (YarnRuntimeException e) { e.printStackTrace(); Assert.fail("Failed to create client"); @@ -105,7 +105,7 @@ public class TestRPCFactories { } } - public class AMRMProtocolTestImpl implements AMRMProtocol { + public class AMRMProtocolTestImpl implements ApplicationMasterProtocol { @Override public RegisterApplicationMasterResponse registerApplicationMaster( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java index a8152042e3b..bc5825a9b95 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java @@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.util.Records; @@ -41,7 +41,7 @@ import org.apache.hadoop.yarn.util.Records; *

* * @see NodeReport - * @see ClientRMProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest) + * @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest) */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java index 24b3e98c0a1..f66be98a58f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager; import java.util.concurrent.ConcurrentMap; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; @@ -60,5 +60,5 @@ public interface Context { NodeHealthStatus getNodeHealthStatus(); - ContainerManager getContainerManager(); + ContainerManagementProtocol getContainerManager(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java index b3af5c2390b..64f6de9f207 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java @@ -36,7 +36,7 @@ import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; @@ -307,7 +307,7 @@ public class NodeManager extends CompositeService private final NMContainerTokenSecretManager containerTokenSecretManager; private final NMTokenSecretManagerInNM nmTokenSecretManager; - private ContainerManager containerManager; + private ContainerManagementProtocol containerManager; private WebServer webServer; private final NodeHealthStatus nodeHealthStatus = RecordFactoryProvider .getRecordFactory(null).newRecordInstance(NodeHealthStatus.class); @@ -360,11 +360,11 @@ public class NodeManager extends CompositeService } @Override - public ContainerManager getContainerManager() { + public ContainerManagementProtocol getContainerManager() { return this.containerManager; } - public void setContainerManager(ContainerManager containerManager) { + public void setContainerManager(ContainerManagementProtocol containerManager) { this.containerManager = containerManager; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java index 8a34ad5d65a..071ea5a4733 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java @@ -41,7 +41,7 @@ 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.util.StringUtils; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; @@ -107,7 +107,7 @@ import org.apache.hadoop.yarn.service.ServiceStateChangeListener; import com.google.common.annotations.VisibleForTesting; public class ContainerManagerImpl extends CompositeService implements - ServiceStateChangeListener, ContainerManager, + ServiceStateChangeListener, ContainerManagementProtocol, EventHandler { private static final Log LOG = LogFactory.getLog(ContainerManagerImpl.class); @@ -233,7 +233,7 @@ public class ContainerManagerImpl extends CompositeService implements YarnConfiguration.DEFAULT_NM_PORT); server = - rpc.getServer(ContainerManager.class, this, initialAddress, conf, + rpc.getServer(ContainerManagementProtocol.class, this, initialAddress, conf, this.context.getContainerTokenSecretManager(), conf.getInt(YarnConfiguration.NM_CONTAINER_MGR_THREAD_COUNT, YarnConfiguration.DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT)); diff --git a/hadoop-yarn-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-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/authorize/NMPolicyProvider.java index 11c470edf84..89e3d78935a 100644 --- a/hadoop-yarn-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-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/authorize/NMPolicyProvider.java @@ -21,7 +21,7 @@ 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.api.ContainerManagerPB; +import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB; @@ -35,8 +35,8 @@ public class NMPolicyProvider extends PolicyProvider { private static final Service[] nodeManagerServices = new Service[] { new Service( - YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGER, - ContainerManagerPB.class), + YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL, + ContainerManagementProtocolPB.class), new Service(YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER, LocalizationProtocolPB.class) }; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java index 3cf4601d92e..cda47b22770 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java @@ -39,7 +39,7 @@ import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; @@ -102,7 +102,7 @@ public class TestNodeManagerReboot { nm = new MyNodeManager(); nm.start(); - final ContainerManager containerManager = nm.getContainerManager(); + final ContainerManagementProtocol containerManager = nm.getContainerManager(); // create files under fileCache createFiles(nmLocalDir.getAbsolutePath(), ContainerLocalizer.FILECACHE, 100); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java index 2bdbbedd2ef..312f7cce1cf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java @@ -39,7 +39,7 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; @@ -192,15 +192,15 @@ public class TestNodeManagerShutdown { UserGroupInformation currentUser = UserGroupInformation .createRemoteUser(cId.toString()); - ContainerManager containerManager = - currentUser.doAs(new PrivilegedAction() { + ContainerManagementProtocol containerManager = + currentUser.doAs(new PrivilegedAction() { @Override - public ContainerManager run() { + public ContainerManagementProtocol run() { Configuration conf = new Configuration(); YarnRPC rpc = YarnRPC.create(conf); InetSocketAddress containerManagerBindAddress = NetUtils.createSocketAddrForHost("127.0.0.1", 12345); - return (ContainerManager) rpc.getProxy(ContainerManager.class, + return (ContainerManagementProtocol) rpc.getProxy(ContainerManagementProtocol.class, containerManagerBindAddress, conf); } }); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java index e1806c1f3c8..62ef72fb81d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java @@ -30,7 +30,7 @@ import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -210,13 +210,13 @@ public abstract class BaseContainerManagerTest { new Path(localDir.getAbsolutePath()), new Path[] {}); } - public static void waitForContainerState(ContainerManager containerManager, + public static void waitForContainerState(ContainerManagementProtocol containerManager, ContainerId containerID, ContainerState finalState) throws InterruptedException, YarnException, IOException { waitForContainerState(containerManager, containerID, finalState, 20); } - public static void waitForContainerState(ContainerManager containerManager, + public static void waitForContainerState(ContainerManagementProtocol containerManager, ContainerId containerID, ContainerState finalState, int timeOutMax) throws InterruptedException, YarnException, IOException { GetContainerStatusRequest request = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java index a4e57573f2e..de842a10334 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java @@ -32,7 +32,7 @@ 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.api.RMAdminProtocol; +import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocol; import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsRequest; import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsResponse; import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesRequest; @@ -56,7 +56,7 @@ 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 { +public class AdminService extends AbstractService implements ResourceManagerAdministrationProtocol { private static final Log LOG = LogFactory.getLog(AdminService.class); @@ -108,7 +108,7 @@ public class AdminService extends AbstractService implements RMAdminProtocol { Configuration conf = getConfig(); YarnRPC rpc = YarnRPC.create(conf); this.server = - rpc.getServer(RMAdminProtocol.class, this, masterServiceAddress, + rpc.getServer(ResourceManagerAdministrationProtocol.class, this, masterServiceAddress, conf, null, conf.getInt(YarnConfiguration.RM_ADMIN_CLIENT_THREAD_COUNT, YarnConfiguration.DEFAULT_RM_ADMIN_CLIENT_THREAD_COUNT)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java index 3d96ced018b..8fa54389aa6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java @@ -36,7 +36,7 @@ import org.apache.hadoop.ipc.Server; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.PolicyProvider; import org.apache.hadoop.util.StringUtils; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; @@ -83,7 +83,7 @@ import org.apache.hadoop.yarn.service.AbstractService; @SuppressWarnings("unchecked") @Private public class ApplicationMasterService extends AbstractService implements - AMRMProtocol { + ApplicationMasterProtocol { private static final Log LOG = LogFactory.getLog(ApplicationMasterService.class); private final AMLivelinessMonitor amLivelinessMonitor; private YarnScheduler rScheduler; @@ -117,7 +117,7 @@ public class ApplicationMasterService extends AbstractService implements YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT); this.server = - rpc.getServer(AMRMProtocol.class, this, masterServiceAddress, + rpc.getServer(ApplicationMasterProtocol.class, this, masterServiceAddress, conf, this.rmContext.getApplicationTokenSecretManager(), conf.getInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT, YarnConfiguration.DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java index 88bc68b785a..3e24d366062 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java @@ -38,7 +38,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; import org.apache.hadoop.security.authorize.PolicyProvider; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; @@ -98,7 +98,7 @@ import org.apache.hadoop.yarn.util.Records; * interfaces to the resource manager from the client. */ public class ClientRMService extends AbstractService implements - ClientRMProtocol { + ApplicationClientProtocol { private static final ArrayList EMPTY_APPS_REPORT = new ArrayList(); private static final Log LOG = LogFactory.getLog(ClientRMService.class); @@ -138,7 +138,7 @@ public class ClientRMService extends AbstractService implements Configuration conf = getConfig(); YarnRPC rpc = YarnRPC.create(conf); this.server = - rpc.getServer(ClientRMProtocol.class, this, + rpc.getServer(ApplicationClientProtocol.class, this, clientBindAddress, conf, this.rmDTSecretManager, conf.getInt(YarnConfiguration.RM_CLIENT_THREAD_COUNT, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java index 4ec82e476be..60258b172f4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java @@ -38,7 +38,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -68,7 +68,7 @@ public class AMLauncher implements Runnable { private static final Log LOG = LogFactory.getLog(AMLauncher.class); - private ContainerManager containerMgrProxy; + private ContainerManagementProtocol containerMgrProxy; private final RMAppAttempt application; private final Configuration conf; @@ -125,7 +125,7 @@ public class AMLauncher implements Runnable { } // Protected. For tests. - protected ContainerManager getContainerMgrProxy( + protected ContainerManagementProtocol getContainerMgrProxy( final ContainerId containerId) { final NodeId node = masterContainer.getNodeId(); @@ -142,10 +142,10 @@ public class AMLauncher implements Runnable { .getContainerToken(), containerManagerBindAddress); currentUser.addToken(token); } - return currentUser.doAs(new PrivilegedAction() { + return currentUser.doAs(new PrivilegedAction() { @Override - public ContainerManager run() { - return (ContainerManager) rpc.getProxy(ContainerManager.class, + public ContainerManagementProtocol run() { + return (ContainerManagementProtocol) rpc.getProxy(ContainerManagementProtocol.class, containerManagerBindAddress, conf); } }); diff --git a/hadoop-yarn-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-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/authorize/RMPolicyProvider.java index 95711083b6d..bd7573b7a1d 100644 --- a/hadoop-yarn-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-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/authorize/RMPolicyProvider.java @@ -21,10 +21,10 @@ 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.api.AMRMProtocolPB; -import org.apache.hadoop.yarn.api.ClientRMProtocolPB; -import org.apache.hadoop.yarn.api.ContainerManagerPB; -import org.apache.hadoop.yarn.api.RMAdminProtocolPB; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB; +import org.apache.hadoop.yarn.api.ApplicationClientProtocolPB; +import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB; +import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocolPB; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.api.ResourceTrackerPB; @@ -38,20 +38,20 @@ public class RMPolicyProvider extends PolicyProvider { private static final Service[] resourceManagerServices = new Service[] { new Service( - YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER, + YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL, ResourceTrackerPB.class), new Service( - YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CLIENT_RESOURCEMANAGER, - ClientRMProtocolPB.class), + YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONCLIENT_PROTOCOL, + ApplicationClientProtocolPB.class), new Service( - YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONMASTER_RESOURCEMANAGER, - AMRMProtocolPB.class), + YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONMASTER_PROTOCOL, + ApplicationMasterProtocolPB.class), new Service( - YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_ADMIN, - RMAdminProtocolPB.class), + YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCEMANAGER_ADMINISTRATION_PROTOCOL, + ResourceManagerAdministrationProtocolPB.class), new Service( - YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGER, - ContainerManagerPB.class), + YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL, + ContainerManagementProtocolPB.class), }; @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java index a110597b815..52102ff4b11 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java @@ -23,7 +23,7 @@ import java.util.List; import junit.framework.Assert; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; @@ -44,19 +44,19 @@ public class MockAM { private volatile int responseId = 0; private final ApplicationAttemptId attemptId; private final RMContext context; - private AMRMProtocol amRMProtocol; + private ApplicationMasterProtocol amRMProtocol; private final List requests = new ArrayList(); private final List releases = new ArrayList(); - MockAM(RMContext context, AMRMProtocol amRMProtocol, + MockAM(RMContext context, ApplicationMasterProtocol amRMProtocol, ApplicationAttemptId attemptId) { this.context = context; this.amRMProtocol = amRMProtocol; this.attemptId = attemptId; } - void setAMRMProtocol(AMRMProtocol amRMProtocol) { + void setAMRMProtocol(ApplicationMasterProtocol amRMProtocol) { this.amRMProtocol = amRMProtocol; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java index c9dd8a28945..b062eb9436c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java @@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; @@ -123,7 +123,7 @@ public class MockRM extends ResourceManager { // get new application id public GetNewApplicationResponse getNewAppId() throws Exception { - ClientRMProtocol client = getClientRMService(); + ApplicationClientProtocol client = getClientRMService(); return client.getNewApplication(Records .newRecord(GetNewApplicationRequest.class)); } @@ -164,7 +164,7 @@ public class MockRM extends ResourceManager { public RMApp submitApp(int masterMemory, String name, String user, Map acls, boolean unmanaged, String queue, int maxAppAttempts, Credentials ts, String appType) throws Exception { - ClientRMProtocol client = getClientRMService(); + ApplicationClientProtocol client = getClientRMService(); GetNewApplicationResponse resp = client.getNewApplication(Records .newRecord(GetNewApplicationRequest.class)); ApplicationId appId = resp.getApplicationId(); @@ -201,7 +201,7 @@ public class MockRM extends ResourceManager { UserGroupInformation.createUserForTesting(user, new String[] {"someGroup"}); PrivilegedAction action = new PrivilegedAction() { - ClientRMProtocol client; + ApplicationClientProtocol client; SubmitApplicationRequest req; @Override public SubmitApplicationResponse run() { @@ -215,7 +215,7 @@ public class MockRM extends ResourceManager { return null; } PrivilegedAction setClientReq( - ClientRMProtocol client, SubmitApplicationRequest req) { + ApplicationClientProtocol client, SubmitApplicationRequest req) { this.client = client; this.req = req; return this; @@ -261,7 +261,7 @@ public class MockRM extends ResourceManager { } public void killApp(ApplicationId appId) throws Exception { - ClientRMProtocol client = getClientRMService(); + ApplicationClientProtocol client = getClientRMService(); KillApplicationRequest req = Records .newRecord(KillApplicationRequest.class); req.setApplicationId(appId); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMWithCustomAMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMWithCustomAMLauncher.java index 9a4526adc8f..bf6b2646598 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMWithCustomAMLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMWithCustomAMLauncher.java @@ -19,7 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; @@ -28,14 +28,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; public class MockRMWithCustomAMLauncher extends MockRM { - private final ContainerManager containerManager; + private final ContainerManagementProtocol containerManager; - public MockRMWithCustomAMLauncher(ContainerManager containerManager) { + public MockRMWithCustomAMLauncher(ContainerManagementProtocol containerManager) { this(new Configuration(), containerManager); } public MockRMWithCustomAMLauncher(Configuration conf, - ContainerManager containerManager) { + ContainerManagementProtocol containerManager) { super(conf); this.containerManager = containerManager; } @@ -48,7 +48,7 @@ public class MockRMWithCustomAMLauncher extends MockRM { AMLauncherEventType event) { return new AMLauncher(context, application, event, getConfig()) { @Override - protected ContainerManager getContainerMgrProxy( + protected ContainerManagementProtocol getContainerMgrProxy( ContainerId containerId) { return containerManager; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java index ab3756bb343..89522356285 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java @@ -30,7 +30,7 @@ import junit.framework.Assert; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; @@ -60,7 +60,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaS import org.apache.hadoop.yarn.server.utils.BuilderUtils; @Private -public class NodeManager implements ContainerManager { +public class NodeManager implements ContainerManagementProtocol { private static final Log LOG = LogFactory.getLog(NodeManager.class); private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java index 270aa780a12..c70f8097a24 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java @@ -32,8 +32,8 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.io.DataInputByteBuffer; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.AMRMProtocol; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; @@ -67,7 +67,7 @@ public class TestAMAuthorization { UserGroupInformation.setConfiguration(confWithSecurityEnabled); } - public static final class MyContainerManager implements ContainerManager { + public static final class MyContainerManager implements ContainerManagementProtocol { public ByteBuffer amTokens; @@ -99,7 +99,7 @@ public class TestAMAuthorization { public static class MockRMWithAMS extends MockRMWithCustomAMLauncher { - public MockRMWithAMS(Configuration conf, ContainerManager containerManager) { + public MockRMWithAMS(Configuration conf, ContainerManagementProtocol containerManager) { super(conf, containerManager); } @@ -154,11 +154,11 @@ public class TestAMAuthorization { credentials.readTokenStorageStream(buf); currentUser.addCredentials(credentials); - AMRMProtocol client = currentUser - .doAs(new PrivilegedAction() { + ApplicationMasterProtocol client = currentUser + .doAs(new PrivilegedAction() { @Override - public AMRMProtocol run() { - return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class, rm + public ApplicationMasterProtocol run() { + return (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class, rm .getApplicationMasterService().getBindAddress(), conf); } }); @@ -208,11 +208,11 @@ public class TestAMAuthorization { .createRemoteUser(applicationAttemptId.toString()); // First try contacting NM without tokens - AMRMProtocol client = currentUser - .doAs(new PrivilegedAction() { + ApplicationMasterProtocol client = currentUser + .doAs(new PrivilegedAction() { @Override - public AMRMProtocol run() { - return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class, + public ApplicationMasterProtocol run() { + return (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class, serviceAddr, conf); } }); @@ -240,10 +240,10 @@ public class TestAMAuthorization { // Create a client to the RM. client = currentUser - .doAs(new PrivilegedAction() { + .doAs(new PrivilegedAction() { @Override - public AMRMProtocol run() { - return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class, + public ApplicationMasterProtocol run() { + return (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class, serviceAddr, conf); } }); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.java index ef36858e9ed..8ba1586e3b7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.java @@ -32,7 +32,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AccessControlList; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; @@ -78,7 +78,7 @@ public class TestApplicationACLs { YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS, YarnConfiguration.DEFAULT_RM_PORT); - private static ClientRMProtocol rmClient; + private static ApplicationClientProtocol rmClient; private static RecordFactory recordFactory = RecordFactoryProvider .getRecordFactory(conf); @@ -121,10 +121,10 @@ public class TestApplicationACLs { UserGroupInformation owner = UserGroupInformation .createRemoteUser(APP_OWNER); - rmClient = owner.doAs(new PrivilegedExceptionAction() { + rmClient = owner.doAs(new PrivilegedExceptionAction() { @Override - public ClientRMProtocol run() throws Exception { - return (ClientRMProtocol) rpc.getProxy(ClientRMProtocol.class, + public ApplicationClientProtocol run() throws Exception { + return (ApplicationClientProtocol) rpc.getProxy(ApplicationClientProtocol.class, rmAddress, conf); } }); @@ -178,15 +178,15 @@ public class TestApplicationACLs { return applicationId; } - private ClientRMProtocol getRMClientForUser(String user) + private ApplicationClientProtocol getRMClientForUser(String user) throws IOException, InterruptedException { UserGroupInformation userUGI = UserGroupInformation .createRemoteUser(user); - ClientRMProtocol userClient = userUGI - .doAs(new PrivilegedExceptionAction() { + ApplicationClientProtocol userClient = userUGI + .doAs(new PrivilegedExceptionAction() { @Override - public ClientRMProtocol run() throws Exception { - return (ClientRMProtocol) rpc.getProxy(ClientRMProtocol.class, + public ApplicationClientProtocol run() throws Exception { + return (ApplicationClientProtocol) rpc.getProxy(ApplicationClientProtocol.class, rmAddress, conf); } }); @@ -237,7 +237,7 @@ public class TestApplicationACLs { .newRecordInstance(KillApplicationRequest.class); finishAppRequest.setApplicationId(applicationId); - ClientRMProtocol superUserClient = getRMClientForUser(SUPER_USER); + ApplicationClientProtocol superUserClient = getRMClientForUser(SUPER_USER); // View as the superUser superUserClient.getApplicationReport(appReportRequest); @@ -268,7 +268,7 @@ public class TestApplicationACLs { .newRecordInstance(KillApplicationRequest.class); finishAppRequest.setApplicationId(applicationId); - ClientRMProtocol friendClient = getRMClientForUser(FRIEND); + ApplicationClientProtocol friendClient = getRMClientForUser(FRIEND); // View as the friend friendClient.getApplicationReport(appReportRequest); @@ -299,7 +299,7 @@ public class TestApplicationACLs { .newRecordInstance(KillApplicationRequest.class); finishAppRequest.setApplicationId(applicationId); - ClientRMProtocol enemyRmClient = getRMClientForUser(ENEMY); + ApplicationClientProtocol enemyRmClient = getRMClientForUser(ENEMY); // View as the enemy ApplicationReport appReport = enemyRmClient.getApplicationReport( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java index 9231442a3f6..792d510b75c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java @@ -24,7 +24,7 @@ import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; @@ -55,7 +55,7 @@ public class TestApplicationMasterLauncher { .getLog(TestApplicationMasterLauncher.class); private static final class MyContainerManagerImpl implements - ContainerManager { + ContainerManagementProtocol { boolean launched = false; boolean cleanedup = false; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java index 75b0a9e5f4b..1c185ac05d8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java @@ -40,7 +40,7 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.yarn.MockApps; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; @@ -124,9 +124,9 @@ public class TestClientRMService { YarnRPC rpc = YarnRPC.create(conf); InetSocketAddress rmAddress = rm.getClientRMService().getBindAddress(); LOG.info("Connecting to ResourceManager at " + rmAddress); - ClientRMProtocol client = - (ClientRMProtocol) rpc - .getProxy(ClientRMProtocol.class, rmAddress, conf); + ApplicationClientProtocol client = + (ApplicationClientProtocol) rpc + .getProxy(ApplicationClientProtocol.class, rmAddress, conf); // Make call GetClusterNodesRequest request = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java index 6bfe12cbe62..e1b7f08a925 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java @@ -52,7 +52,7 @@ import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; @@ -108,7 +108,7 @@ public class TestClientRMTokens { clientRMService.init(conf); clientRMService.start(); - ClientRMProtocol clientRMWithDT = null; + ApplicationClientProtocol clientRMWithDT = null; try { // Create a user for the renewr and fake the authentication-method @@ -351,7 +351,7 @@ public class TestClientRMTokens { // the kerberos based rpc. private org.apache.hadoop.yarn.api.records.Token getDelegationToken( final UserGroupInformation loggedInUser, - final ClientRMProtocol clientRMService, final String renewerString) + final ApplicationClientProtocol clientRMService, final String renewerString) throws IOException, InterruptedException { org.apache.hadoop.yarn.api.records.Token token = loggedInUser .doAs(new PrivilegedExceptionAction() { @@ -369,7 +369,7 @@ public class TestClientRMTokens { } private long renewDelegationToken(final UserGroupInformation loggedInUser, - final ClientRMProtocol clientRMService, + final ApplicationClientProtocol clientRMService, final org.apache.hadoop.yarn.api.records.Token dToken) throws IOException, InterruptedException { long nextExpTime = loggedInUser.doAs(new PrivilegedExceptionAction() { @@ -386,7 +386,7 @@ public class TestClientRMTokens { } private void cancelDelegationToken(final UserGroupInformation loggedInUser, - final ClientRMProtocol clientRMService, + final ApplicationClientProtocol clientRMService, final org.apache.hadoop.yarn.api.records.Token dToken) throws IOException, InterruptedException { loggedInUser.doAs(new PrivilegedExceptionAction() { @@ -401,7 +401,7 @@ public class TestClientRMTokens { }); } - private ClientRMProtocol getClientRMProtocolWithDT( + private ApplicationClientProtocol getClientRMProtocolWithDT( org.apache.hadoop.yarn.api.records.Token token, final InetSocketAddress rmAddress, String user, final Configuration conf) { // Maybe consider converting to Hadoop token, serialize de-serialize etc @@ -412,11 +412,11 @@ public class TestClientRMTokens { ugi.addToken(ProtoUtils.convertFromProtoFormat(token, rmAddress)); final YarnRPC rpc = YarnRPC.create(conf); - ClientRMProtocol clientRMWithDT = ugi - .doAs(new PrivilegedAction() { + ApplicationClientProtocol clientRMWithDT = ugi + .doAs(new PrivilegedAction() { @Override - public ClientRMProtocol run() { - return (ClientRMProtocol) rpc.getProxy(ClientRMProtocol.class, + public ApplicationClientProtocol run() { + return (ApplicationClientProtocol) rpc.getProxy(ApplicationClientProtocol.class, rmAddress, conf); } }); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java index 22279484076..3fa1fda0153 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java @@ -31,7 +31,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; import org.apache.hadoop.yarn.api.records.ApplicationAccessType; @@ -295,11 +295,11 @@ public class TestSchedulerUtils { UserGroupInformation currentUser = UserGroupInformation.createRemoteUser(applicationAttemptId.toString()); - AMRMProtocol client = currentUser - .doAs(new PrivilegedAction() { + ApplicationMasterProtocol client = currentUser + .doAs(new PrivilegedAction() { @Override - public AMRMProtocol run() { - return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class, rm + public ApplicationMasterProtocol run() { + return (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class, rm .getApplicationMasterService().getBindAddress(), conf); } }); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java index b74839b98e3..3e60462b45e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java @@ -29,7 +29,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.io.DataInputByteBuffer; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; @@ -75,7 +75,7 @@ public class TestApplicationTokens { final Configuration conf = rm.getConfig(); final YarnRPC rpc = YarnRPC.create(conf); - AMRMProtocol rmClient = null; + ApplicationMasterProtocol rmClient = null; try { MockNM nm1 = rm.registerNode("localhost:1234", 5120); @@ -167,7 +167,7 @@ public class TestApplicationTokens { final Configuration conf = rm.getConfig(); final YarnRPC rpc = YarnRPC.create(conf); - AMRMProtocol rmClient = null; + ApplicationMasterProtocol rmClient = null; try { MockNM nm1 = rm.registerNode("localhost:1234", 5120); @@ -235,13 +235,13 @@ public class TestApplicationTokens { } } - private AMRMProtocol createRMClient(final MockRM rm, + private ApplicationMasterProtocol createRMClient(final MockRM rm, final Configuration conf, final YarnRPC rpc, UserGroupInformation currentUser) { - return currentUser.doAs(new PrivilegedAction() { + return currentUser.doAs(new PrivilegedAction() { @Override - public AMRMProtocol run() { - return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class, rm + public ApplicationMasterProtocol run() { + return (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class, rm .getApplicationMasterService().getBindAddress(), conf); } }); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java index fd99f5f4d86..c8376c73fcc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java @@ -45,7 +45,7 @@ import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenInfo; import org.apache.hadoop.security.token.TokenSelector; import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; @@ -149,7 +149,7 @@ public class TestClientTokens { } } - private static class CustomNM implements ContainerManager { + private static class CustomNM implements ContainerManagementProtocol { public String clientTokensSecret; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java index 985ccd69b52..2706340d703 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java @@ -44,8 +44,8 @@ import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Shell; -import org.apache.hadoop.yarn.api.AMRMProtocol; -import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; @@ -141,7 +141,7 @@ public class TestContainerManagerSecurity { ApplicationId appID = resourceManager.getClientRMService() .getNewApplication(Records.newRecord(GetNewApplicationRequest.class)) .getApplicationId(); - AMRMProtocol scheduler = submitAndRegisterApplication(resourceManager, + ApplicationMasterProtocol scheduler = submitAndRegisterApplication(resourceManager, yarnRPC, appID); // Now request a container. @@ -162,8 +162,8 @@ public class TestContainerManagerSecurity { authenticatedUser.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - ContainerManager client = (ContainerManager) yarnRPC.getProxy( - ContainerManager.class, NetUtils + ContainerManagementProtocol client = (ContainerManagementProtocol) yarnRPC.getProxy( + ContainerManagementProtocol.class, NetUtils .createSocketAddr(allocatedContainer.getNodeId().toString()), conf); LOG.info("Going to make a legal stopContainer() request"); @@ -203,7 +203,7 @@ public class TestContainerManagerSecurity { ApplicationId appID = resourceManager.getClientRMService() .getNewApplication(Records.newRecord(GetNewApplicationRequest.class)) .getApplicationId(); - AMRMProtocol scheduler = submitAndRegisterApplication(resourceManager, + ApplicationMasterProtocol scheduler = submitAndRegisterApplication(resourceManager, yarnRPC, appID); // Now request a container. @@ -270,8 +270,8 @@ public class TestContainerManagerSecurity { maliceUser.doAs(new PrivilegedAction() { @Override public Void run() { - ContainerManager client = (ContainerManager) yarnRPC.getProxy( - ContainerManager.class, NetUtils + ContainerManagementProtocol client = (ContainerManagementProtocol) yarnRPC.getProxy( + ContainerManagementProtocol.class, NetUtils .createSocketAddr(allocatedContainer.getNodeId().toString()), conf); @@ -318,7 +318,7 @@ public class TestContainerManagerSecurity { final ApplicationId appID = resourceManager.getClientRMService() .getNewApplication(Records.newRecord(GetNewApplicationRequest.class)) .getApplicationId(); - AMRMProtocol scheduler = submitAndRegisterApplication(resourceManager, + ApplicationMasterProtocol scheduler = submitAndRegisterApplication(resourceManager, yarnRPC, appID); // Now request a container. @@ -358,8 +358,8 @@ public class TestContainerManagerSecurity { unauthorizedUser.doAs(new PrivilegedAction() { @Override public Void run() { - ContainerManager client = (ContainerManager) yarnRPC.getProxy( - ContainerManager.class, NetUtils + ContainerManagementProtocol client = (ContainerManagementProtocol) yarnRPC.getProxy( + ContainerManagementProtocol.class, NetUtils .createSocketAddr(allocatedContainer.getNodeId().toString()), conf); @@ -403,7 +403,7 @@ public class TestContainerManagerSecurity { resourceManager.getClientRMService().forceKillApplication(request); } - private AMRMProtocol submitAndRegisterApplication( + private ApplicationMasterProtocol submitAndRegisterApplication( ResourceManager resourceManager, final YarnRPC yarnRPC, ApplicationId appID) throws IOException, UnsupportedFileSystemException, YarnException, @@ -472,11 +472,11 @@ public class TestContainerManagerSecurity { currentUser.addToken(appToken); } - AMRMProtocol scheduler = currentUser - .doAs(new PrivilegedAction() { + ApplicationMasterProtocol scheduler = currentUser + .doAs(new PrivilegedAction() { @Override - public AMRMProtocol run() { - return (AMRMProtocol) yarnRPC.getProxy(AMRMProtocol.class, + public ApplicationMasterProtocol run() { + return (ApplicationMasterProtocol) yarnRPC.getProxy(ApplicationMasterProtocol.class, schedulerAddr, conf); } }); @@ -490,7 +490,7 @@ public class TestContainerManagerSecurity { return scheduler; } - private Container requestAndGetContainer(AMRMProtocol scheduler, + private Container requestAndGetContainer(ApplicationMasterProtocol scheduler, ApplicationId appID) throws YarnException, InterruptedException, IOException { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/AppReportFetcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/AppReportFetcher.java index a5aad90d63d..077783facfc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/AppReportFetcher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/AppReportFetcher.java @@ -24,7 +24,7 @@ import java.net.InetSocketAddress; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -41,7 +41,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC; public class AppReportFetcher { private static final Log LOG = LogFactory.getLog(AppReportFetcher.class); private final Configuration conf; - private final ClientRMProtocol applicationsManager; + private final ApplicationClientProtocol applicationsManager; private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); /** @@ -57,7 +57,7 @@ public class AppReportFetcher { YarnConfiguration.DEFAULT_RM_PORT); LOG.info("Connecting to ResourceManager at " + rmAddress); applicationsManager = - (ClientRMProtocol) rpc.getProxy(ClientRMProtocol.class, + (ApplicationClientProtocol) rpc.getProxy(ApplicationClientProtocol.class, rmAddress, this.conf); LOG.info("Connected to ResourceManager at " + rmAddress); } @@ -69,7 +69,7 @@ public class AppReportFetcher { * @param conf the configuration to use * @param applicationsManager what to use to get the RM reports. */ - public AppReportFetcher(Configuration conf, ClientRMProtocol applicationsManager) { + public AppReportFetcher(Configuration conf, ApplicationClientProtocol applicationsManager) { this.conf = conf; this.applicationsManager = applicationsManager; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm index d081b50db46..698ffea3942 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm @@ -32,10 +32,10 @@ Hadoop MapReduce Next Generation - Writing YARN Applications The general concept is that an 'Application Submission Client' submits an 'Application' to the YARN Resource Manager. The client communicates with the - ResourceManager using the 'ClientRMProtocol' to first acquire a new - 'ApplicationId' if needed via ClientRMProtocol#getNewApplication and then - submit the 'Application' to be run via ClientRMProtocol#submitApplication. As - part of the ClientRMProtocol#submitApplication call, the client needs to + ResourceManager using the 'ApplicationClientProtocol' to first acquire a new + 'ApplicationId' if needed via ApplicationClientProtocol#getNewApplication and then + submit the 'Application' to be run via ApplicationClientProtocol#submitApplication. As + part of the ApplicationClientProtocol#submitApplication call, the client needs to provide sufficient information to the ResourceManager to 'launch' the application's first container i.e. the ApplicationMaster. You need to provide information such as the details about the local @@ -47,35 +47,35 @@ Hadoop MapReduce Next Generation - Writing YARN Applications The YARN ResourceManager will then launch the ApplicationMaster (as specified) on an allocated container. The ApplicationMaster is then expected to - communicate with the ResourceManager using the 'AMRMProtocol'. Firstly, the + communicate with the ResourceManager using the 'ApplicationMasterProtocol'. Firstly, the ApplicationMaster needs to register itself with the ResourceManager. To complete the task assigned to it, the ApplicationMaster can then request for - and receive containers via AMRMProtocol#allocate. After a container is + and receive containers via ApplicationMasterProtocol#allocate. After a container is allocated to it, the ApplicationMaster communicates with the NodeManager using ContainerManager#startContainer to launch the container for its task. As part of launching this container, the ApplicationMaster has to specify the ContainerLaunchContext which, similar to the ApplicationSubmissionContext, has the launch information such as command line specification, environment, etc. Once the task is completed, the ApplicationMaster has to signal the - ResourceManager of its completion via the AMRMProtocol#finishApplicationMaster. + ResourceManager of its completion via the ApplicationMasterProtocol#finishApplicationMaster. Meanwhile, the client can monitor the application's status by querying the ResourceManager or by directly querying the ApplicationMaster if it supports such a service. If needed, it can also kill the application via - ClientRMProtocol#forceKillApplication. + ApplicationClientProtocol#forceKillApplication. * Interfaces The interfaces you'd most like be concerned with are: - * ClientRMProtocol - Client\<--\>ResourceManager\ + * ApplicationClientProtocol - Client\<--\>ResourceManager\ The protocol for a client that wishes to communicate with the ResourceManager to launch a new application (i.e. the ApplicationMaster), check on the status of the application or kill the application. For example, a job-client (a job launching program from the gateway) would use this protocol. - * AMRMProtocol - ApplicationMaster\<--\>ResourceManager\ + * ApplicationMasterProtocol - ApplicationMaster\<--\>ResourceManager\ The protocol used by the ApplicationMaster to register/unregister itself to/from the ResourceManager as well as to request for resources from the Scheduler to complete its tasks. @@ -93,7 +93,7 @@ Hadoop MapReduce Next Generation - Writing YARN Applications interface of the ResourceManager. +---+ - ClientRMProtocol applicationsManager; + ApplicationClientProtocol applicationsManager; YarnConfiguration yarnConf = new YarnConfiguration(conf); InetSocketAddress rmAddress = NetUtils.createSocketAddr(yarnConf.get( @@ -104,8 +104,8 @@ Hadoop MapReduce Next Generation - Writing YARN Applications appsManagerServerConf.setClass( YarnConfiguration.YARN_SECURITY_INFO, ClientRMSecurityInfo.class, SecurityInfo.class); - applicationsManager = ((ClientRMProtocol) rpc.getProxy( - ClientRMProtocol.class, rmAddress, appsManagerServerConf)); + applicationsManager = ((ApplicationClientProtocol) rpc.getProxy( + ApplicationClientProtocol.class, rmAddress, appsManagerServerConf)); +---+ * Once a handle is obtained to the ASM, the client needs to request the @@ -256,7 +256,7 @@ Hadoop MapReduce Next Generation - Writing YARN Applications * There are multiple ways a client can track progress of the actual task. * It can communicate with the ResourceManager and request for a report of - the application via ClientRMProtocol#getApplicationReport. + the application via ApplicationClientProtocol#getApplicationReport. +-----+ GetApplicationReportRequest reportRequest = @@ -299,7 +299,7 @@ Hadoop MapReduce Next Generation - Writing YARN Applications * In certain situations, if the application is taking too long or due to other factors, the client may wish to kill the application. The - ClientRMProtocol supports the forceKillApplication call that allows a + ApplicationClientProtocol supports the forceKillApplication call that allows a client to send a kill signal to the ApplicationMaster via the ResourceManager. An ApplicationMaster if so designed may also support an abort call via its rpc layer that a client may be able to leverage. @@ -350,7 +350,7 @@ Hadoop MapReduce Next Generation - Writing YARN Applications * After an ApplicationMaster has initialized itself completely, it needs to register with the ResourceManager via - AMRMProtocol#registerApplicationMaster. The ApplicationMaster always + ApplicationMasterProtocol#registerApplicationMaster. The ApplicationMaster always communicate via the Scheduler interface of the ResourceManager. +---+ @@ -361,8 +361,8 @@ Hadoop MapReduce Next Generation - Writing YARN Applications YarnConfiguration.RM_SCHEDULER_ADDRESS, YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS)); LOG.info("Connecting to ResourceManager at " + rmAddress); - AMRMProtocol resourceManager = - (AMRMProtocol) rpc.getProxy(AMRMProtocol.class, rmAddress, conf); + ApplicationMasterProtocol resourceManager = + (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class, rmAddress, conf); // Register the AM with the RM // Set the required info into the registration request: @@ -391,7 +391,7 @@ Hadoop MapReduce Next Generation - Writing YARN Applications timeout expiry interval at the ResourceManager is defined by a config setting accessible via YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS with the default being defined by YarnConfiguration.DEFAULT_RM_AM_EXPIRY_INTERVAL_MS. - The AMRMProtocol#allocate calls to the ResourceManager count as heartbeats + The ApplicationMasterProtocol#allocate calls to the ResourceManager count as heartbeats as it also supports sending progress update information. Therefore, an allocate call with no containers requested and progress information updated if any is a valid way for making heartbeat calls to the ResourceManager. @@ -682,7 +682,7 @@ Hadoop MapReduce Next Generation - Writing YARN Applications +---+ * The ApplicationMaster, as mentioned previously, will get updates of - completed containers as part of the response from the AMRMProtocol#allocate + completed containers as part of the response from the ApplicationMasterProtocol#allocate calls. It can also monitor its launched containers pro-actively by querying the ContainerManager for the status.