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
This commit is contained in:
Arun Murthy 2013-06-17 02:29:56 +00:00
parent c71d86cd7c
commit 68807a65c2
128 changed files with 504 additions and 498 deletions

View File

@ -140,7 +140,7 @@
<property> <property>
<name>security.resourcetracker.protocol.acl</name> <name>security.resourcetracker.protocol.acl</name>
<value>*</value> <value>*</value>
<description>ACL for ResourceTracker protocol, used by the <description>ACL for ResourceTrackerProtocol, used by the
ResourceManager and NodeManager to communicate with each other. ResourceManager and NodeManager to communicate with each other.
The ACL is a comma-separated list of user and group names. The user and 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". group list is separated by a blank. For e.g. "alice,bob users,wheel".
@ -148,18 +148,18 @@
</property> </property>
<property> <property>
<name>security.admin.protocol.acl</name> <name>security.resourcemanager-administration.protocol.acl</name>
<value>*</value> <value>*</value>
<description>ACL for RMAdminProtocol, for admin commands. <description>ACL for ResourceManagerAdministrationProtocol, for admin commands.
The ACL is a comma-separated list of user and group names. The user and 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". group list is separated by a blank. For e.g. "alice,bob users,wheel".
A special value of "*" means all users are allowed.</description> A special value of "*" means all users are allowed.</description>
</property> </property>
<property> <property>
<name>security.client.resourcemanager.protocol.acl</name> <name>security.applicationclient.protocol.acl</name>
<value>*</value> <value>*</value>
<description>ACL for ClientRMProtocol, used by the ResourceManager <description>ACL for ApplicationClientProtocol, used by the ResourceManager
and applications submission clients to communicate with each other. and applications submission clients to communicate with each other.
The ACL is a comma-separated list of user and group names. The user and 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". group list is separated by a blank. For e.g. "alice,bob users,wheel".
@ -167,9 +167,9 @@
</property> </property>
<property> <property>
<name>security.applicationmaster.resourcemanager.protocol.acl</name> <name>security.applicationmaster.protocol.acl</name>
<value>*</value> <value>*</value>
<description>ACL for AMRMProtocol, used by the ResourceManager <description>ACL for ApplicationMasterProtocol, used by the ResourceManager
and ApplicationMasters to communicate with each other. and ApplicationMasters to communicate with each other.
The ACL is a comma-separated list of user and group names. The user and 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". group list is separated by a blank. For e.g. "alice,bob users,wheel".
@ -177,9 +177,9 @@
</property> </property>
<property> <property>
<name>security.containermanager.protocol.acl</name> <name>security.containermanagement.protocol.acl</name>
<value>*</value> <value>*</value>
<description>ACL for ContainerManager protocol, used by the NodeManager <description>ACL for ContainerManagementProtocol protocol, used by the NodeManager
and ApplicationMasters to communicate with each other. and ApplicationMasters to communicate with each other.
The ACL is a comma-separated list of user and group names. The user and 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". group list is separated by a blank. For e.g. "alice,bob users,wheel".

View File

@ -48,7 +48,7 @@ import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.StringUtils; 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.StartContainerRequest;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
@ -140,7 +140,7 @@ public class ContainerLauncherImpl extends AbstractService implements
return; return;
} }
ContainerManager proxy = null; ContainerManagementProtocol proxy = null;
try { try {
proxy = getCMProxy(containerID, containerMgrAddress, proxy = getCMProxy(containerID, containerMgrAddress,
@ -198,7 +198,7 @@ public class ContainerLauncherImpl extends AbstractService implements
} else if (!isCompletelyDone()) { } else if (!isCompletelyDone()) {
LOG.info("KILLING " + taskAttemptID); LOG.info("KILLING " + taskAttemptID);
ContainerManager proxy = null; ContainerManagementProtocol proxy = null;
try { try {
proxy = getCMProxy(this.containerID, this.containerMgrAddress, proxy = getCMProxy(this.containerID, this.containerMgrAddress,
this.containerToken); this.containerToken);
@ -348,7 +348,7 @@ public class ContainerLauncherImpl extends AbstractService implements
return new EventProcessor(event); return new EventProcessor(event);
} }
protected ContainerManager getCMProxy(ContainerId containerID, protected ContainerManagementProtocol getCMProxy(ContainerId containerID,
final String containerManagerBindAddr, final String containerManagerBindAddr,
org.apache.hadoop.yarn.api.records.Token containerToken) org.apache.hadoop.yarn.api.records.Token containerToken)
throws IOException { throws IOException {
@ -364,11 +364,11 @@ public class ContainerLauncherImpl extends AbstractService implements
ProtoUtils.convertFromProtoFormat(containerToken, cmAddr); ProtoUtils.convertFromProtoFormat(containerToken, cmAddr);
user.addToken(token); user.addToken(token);
ContainerManager proxy = user ContainerManagementProtocol proxy = user
.doAs(new PrivilegedAction<ContainerManager>() { .doAs(new PrivilegedAction<ContainerManagementProtocol>() {
@Override @Override
public ContainerManager run() { public ContainerManagementProtocol run() {
return (ContainerManager) rpc.getProxy(ContainerManager.class, return (ContainerManagementProtocol) rpc.getProxy(ContainerManagementProtocol.class,
cmAddr, getConfig()); cmAddr, getConfig());
} }
}); });

View File

@ -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.app.job.impl.JobImpl;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils; import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
import org.apache.hadoop.security.UserGroupInformation; 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.FinishApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
@ -69,7 +69,7 @@ public abstract class RMCommunicator extends AbstractService
protected Thread allocatorThread; protected Thread allocatorThread;
@SuppressWarnings("rawtypes") @SuppressWarnings("rawtypes")
protected EventHandler eventHandler; protected EventHandler eventHandler;
protected AMRMProtocol scheduler; protected ApplicationMasterProtocol scheduler;
private final ClientService clientService; private final ClientService clientService;
protected int lastResponseID; protected int lastResponseID;
private Resource maxContainerCapability; private Resource maxContainerCapability;
@ -254,7 +254,7 @@ public abstract class RMCommunicator extends AbstractService
allocatorThread.start(); allocatorThread.start();
} }
protected AMRMProtocol createSchedulerProxy() { protected ApplicationMasterProtocol createSchedulerProxy() {
final Configuration conf = getConfig(); final Configuration conf = getConfig();
final YarnRPC rpc = YarnRPC.create(conf); final YarnRPC rpc = YarnRPC.create(conf);
final InetSocketAddress serviceAddr = conf.getSocketAddr( final InetSocketAddress serviceAddr = conf.getSocketAddr(
@ -270,10 +270,10 @@ public abstract class RMCommunicator extends AbstractService
} }
// CurrentUser should already have AMToken loaded. // CurrentUser should already have AMToken loaded.
return currentUser.doAs(new PrivilegedAction<AMRMProtocol>() { return currentUser.doAs(new PrivilegedAction<ApplicationMasterProtocol>() {
@Override @Override
public AMRMProtocol run() { public ApplicationMasterProtocol run() {
return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class, return (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class,
serviceAddr, conf); serviceAddr, conf);
} }
}); });

View File

@ -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.ContainerAllocator;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent; import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator; 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.AllocateRequest;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
@ -194,8 +194,8 @@ public class MRAppBenchmark {
ClientService clientService, AppContext context) { ClientService clientService, AppContext context) {
return new RMContainerAllocator(clientService, context) { return new RMContainerAllocator(clientService, context) {
@Override @Override
protected AMRMProtocol createSchedulerProxy() { protected ApplicationMasterProtocol createSchedulerProxy() {
return new AMRMProtocol() { return new ApplicationMasterProtocol() {
@Override @Override
public RegisterApplicationMasterResponse public RegisterApplicationMasterResponse

View File

@ -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.ContainerLauncherEvent;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl; import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl;
import org.apache.hadoop.net.NetUtils; 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.ContainerId;
import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.Token;
import org.junit.Test; import org.junit.Test;
@ -225,7 +225,7 @@ public class TestFail {
} }
@Override @Override
protected ContainerManager getCMProxy(ContainerId contianerID, protected ContainerManagementProtocol getCMProxy(ContainerId contianerID,
String containerManagerBindAddr, Token containerToken) String containerManagerBindAddr, Token containerToken)
throws IOException { throws IOException {
try { try {

View File

@ -74,7 +74,7 @@ import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.yarn.Clock; import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.ClusterInfo; import org.apache.hadoop.yarn.ClusterInfo;
import org.apache.hadoop.yarn.SystemClock; 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.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
@ -1387,7 +1387,7 @@ public class TestRMContainerAllocator {
} }
@Override @Override
protected AMRMProtocol createSchedulerProxy() { protected ApplicationMasterProtocol createSchedulerProxy() {
return this.rm.getApplicationMasterService(); return this.rm.getApplicationMasterService();
} }
@ -1600,8 +1600,8 @@ public class TestRMContainerAllocator {
protected void register() { protected void register() {
} }
@Override @Override
protected AMRMProtocol createSchedulerProxy() { protected ApplicationMasterProtocol createSchedulerProxy() {
return mock(AMRMProtocol.class); return mock(ApplicationMasterProtocol.class);
} }
@Override @Override
protected synchronized void heartbeat() throws Exception { protected synchronized void heartbeat() throws Exception {

View File

@ -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.app.job.impl.TaskAttemptImpl;
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils; import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
import org.apache.hadoop.net.NetUtils; 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.GetContainerStatusRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@ -232,7 +232,7 @@ public class TestContainerLauncher {
YarnRPC rpc = YarnRPC.create(conf); YarnRPC rpc = YarnRPC.create(conf);
String bindAddr = "localhost:0"; String bindAddr = "localhost:0";
InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
server = rpc.getServer(ContainerManager.class, new DummyContainerManager(), server = rpc.getServer(ContainerManagementProtocol.class, new DummyContainerManager(),
addr, conf, null, 1); addr, conf, null, 1);
server.start(); server.start();
@ -345,12 +345,12 @@ public class TestContainerLauncher {
protected ContainerLauncher createContainerLauncher(AppContext context) { protected ContainerLauncher createContainerLauncher(AppContext context) {
return new ContainerLauncherImpl(context) { return new ContainerLauncherImpl(context) {
@Override @Override
protected ContainerManager getCMProxy(ContainerId containerID, protected ContainerManagementProtocol getCMProxy(ContainerId containerID,
String containerManagerBindAddr, Token containerToken) String containerManagerBindAddr, Token containerToken)
throws IOException { throws IOException {
// make proxy connect to our local containerManager server // make proxy connect to our local containerManager server
ContainerManager proxy = (ContainerManager) rpc.getProxy( ContainerManagementProtocol proxy = (ContainerManagementProtocol) rpc.getProxy(
ContainerManager.class, ContainerManagementProtocol.class,
NetUtils.getConnectAddress(server), conf); NetUtils.getConnectAddress(server), conf);
return proxy; return proxy;
} }
@ -359,7 +359,7 @@ public class TestContainerLauncher {
}; };
} }
public class DummyContainerManager implements ContainerManager { public class DummyContainerManager implements ContainerManagementProtocol {
private ContainerStatus status = null; private ContainerStatus status = null;

View File

@ -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.job.event.TaskAttemptEventType;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher.EventType; import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher.EventType;
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils; 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.GetContainerStatusRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@ -147,8 +147,8 @@ public class TestContainerLauncherImpl {
EventHandler mockEventHandler = mock(EventHandler.class); EventHandler mockEventHandler = mock(EventHandler.class);
when(mockContext.getEventHandler()).thenReturn(mockEventHandler); when(mockContext.getEventHandler()).thenReturn(mockEventHandler);
ContainerManager mockCM = mock(ContainerManager.class); ContainerManagementProtocol mockCM = mock(ContainerManagementProtocol.class);
when(mockRpc.getProxy(eq(ContainerManager.class), when(mockRpc.getProxy(eq(ContainerManagementProtocol.class),
any(InetSocketAddress.class), any(Configuration.class))) any(InetSocketAddress.class), any(Configuration.class)))
.thenReturn(mockCM); .thenReturn(mockCM);
@ -213,8 +213,8 @@ public class TestContainerLauncherImpl {
EventHandler mockEventHandler = mock(EventHandler.class); EventHandler mockEventHandler = mock(EventHandler.class);
when(mockContext.getEventHandler()).thenReturn(mockEventHandler); when(mockContext.getEventHandler()).thenReturn(mockEventHandler);
ContainerManager mockCM = mock(ContainerManager.class); ContainerManagementProtocol mockCM = mock(ContainerManagementProtocol.class);
when(mockRpc.getProxy(eq(ContainerManager.class), when(mockRpc.getProxy(eq(ContainerManagementProtocol.class),
any(InetSocketAddress.class), any(Configuration.class))) any(InetSocketAddress.class), any(Configuration.class)))
.thenReturn(mockCM); .thenReturn(mockCM);
@ -279,8 +279,8 @@ public class TestContainerLauncherImpl {
EventHandler mockEventHandler = mock(EventHandler.class); EventHandler mockEventHandler = mock(EventHandler.class);
when(mockContext.getEventHandler()).thenReturn(mockEventHandler); when(mockContext.getEventHandler()).thenReturn(mockEventHandler);
ContainerManager mockCM = mock(ContainerManager.class); ContainerManagementProtocol mockCM = mock(ContainerManagementProtocol.class);
when(mockRpc.getProxy(eq(ContainerManager.class), when(mockRpc.getProxy(eq(ContainerManagementProtocol.class),
any(InetSocketAddress.class), any(Configuration.class))) any(InetSocketAddress.class), any(Configuration.class)))
.thenReturn(mockCM); .thenReturn(mockCM);
@ -338,8 +338,8 @@ public class TestContainerLauncherImpl {
EventHandler mockEventHandler = mock(EventHandler.class); EventHandler mockEventHandler = mock(EventHandler.class);
when(mockContext.getEventHandler()).thenReturn(mockEventHandler); when(mockContext.getEventHandler()).thenReturn(mockEventHandler);
ContainerManager mockCM = new ContainerManagerForTest(startLaunchBarrier, completeLaunchBarrier); ContainerManagementProtocol mockCM = new ContainerManagerForTest(startLaunchBarrier, completeLaunchBarrier);
when(mockRpc.getProxy(eq(ContainerManager.class), when(mockRpc.getProxy(eq(ContainerManagementProtocol.class),
any(InetSocketAddress.class), any(Configuration.class))) any(InetSocketAddress.class), any(Configuration.class)))
.thenReturn(mockCM); .thenReturn(mockCM);
@ -417,7 +417,7 @@ public class TestContainerLauncherImpl {
currentTime + 10000L, 123, currentTime)); currentTime + 10000L, 123, currentTime));
} }
private static class ContainerManagerForTest implements ContainerManager { private static class ContainerManagerForTest implements ContainerManagementProtocol {
private CyclicBarrier startLaunchBarrier; private CyclicBarrier startLaunchBarrier;
private CyclicBarrier completeLaunchBarrier; private CyclicBarrier completeLaunchBarrier;

View File

@ -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.client.ClientService;
import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.yarn.ClusterInfo; 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.protocolrecords.AllocateRequest;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -94,8 +94,8 @@ public class TestLocalContainerAllocator {
} }
@Override @Override
protected AMRMProtocol createSchedulerProxy() { protected ApplicationMasterProtocol createSchedulerProxy() {
AMRMProtocol scheduler = mock(AMRMProtocol.class); ApplicationMasterProtocol scheduler = mock(ApplicationMasterProtocol.class);
try { try {
when(scheduler.allocate(isA(AllocateRequest.class))) when(scheduler.allocate(isA(AllocateRequest.class)))
.thenThrow(RPCUtil.getRemoteException(new IOException("forcefail"))); .thenThrow(RPCUtil.getRemoteException(new IOException("forcefail")));

View File

@ -258,7 +258,7 @@
<property><!--Loaded from job.xml--><name>mapreduce.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property> <property><!--Loaded from job.xml--><name>mapreduce.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property>
<property><!--Loaded from job.xml--><name>mapreduce.tasktracker.http.address</name><value>0.0.0.0:50060</value></property> <property><!--Loaded from job.xml--><name>mapreduce.tasktracker.http.address</name><value>0.0.0.0:50060</value></property>
<property><!--Loaded from job.xml--><name>mapreduce.jobhistory.webapp.address</name><value>0.0.0.0:19888</value></property> <property><!--Loaded from job.xml--><name>mapreduce.jobhistory.webapp.address</name><value>0.0.0.0:19888</value></property>
<property><!--Loaded from Unknown--><name>rpc.engine.org.apache.hadoop.yarn.proto.AMRMProtocol$AMRMProtocolService$BlockingInterface</name><value>org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine</value></property> <property><!--Loaded from Unknown--><name>rpc.engine.org.apache.hadoop.yarn.proto.ApplicationMasterProtocol$ApplicationMasterProtocolService$BlockingInterface</name><value>org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine</value></property>
<property><!--Loaded from job.xml--><name>yarn.ipc.rpc.class</name><value>org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC</value></property> <property><!--Loaded from job.xml--><name>yarn.ipc.rpc.class</name><value>org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC</value></property>
<property><!--Loaded from job.xml--><name>mapreduce.job.name</name><value>Sleep job</value></property> <property><!--Loaded from job.xml--><name>mapreduce.job.name</name><value>Sleep job</value></property>
<property><!--Loaded from job.xml--><name>kfs.blocksize</name><value>67108864</value></property> <property><!--Loaded from job.xml--><name>kfs.blocksize</name><value>67108864</value></property>

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.v2.util.MRApps; import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token; 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.protocolrecords.GetNewApplicationResponse;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
@ -54,7 +54,7 @@ public class ResourceMgrDelegate extends YarnClientImpl {
private ApplicationId applicationId; 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. * @param conf the configuration object.
*/ */
public ResourceMgrDelegate(YarnConfiguration conf) { public ResourceMgrDelegate(YarnConfiguration conf) {

View File

@ -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.mapreduce.v2.jobhistory.JHAdminConfig;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.net.NetUtils; 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.CancelDelegationTokenRequest;
import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest;
@ -223,7 +223,7 @@ public class TestClientRedirect {
Assert.assertEquals(1, counters.countCounters()); Assert.assertEquals(1, counters.countCounters());
} }
class RMService extends AbstractService implements ClientRMProtocol { class RMService extends AbstractService implements ApplicationClientProtocol {
private String clientServiceBindAddress; private String clientServiceBindAddress;
InetSocketAddress clientBindAddress; InetSocketAddress clientBindAddress;
private Server server; private Server server;
@ -250,7 +250,7 @@ public class TestClientRedirect {
// Kerberos if security is enabled, so no secretManager. // Kerberos if security is enabled, so no secretManager.
YarnRPC rpc = YarnRPC.create(getConfig()); YarnRPC rpc = YarnRPC.create(getConfig());
Configuration clientServerConf = new Configuration(getConfig()); Configuration clientServerConf = new Configuration(getConfig());
this.server = rpc.getServer(ClientRMProtocol.class, this, this.server = rpc.getServer(ApplicationClientProtocol.class, this,
clientBindAddress, clientServerConf, null, 1); clientBindAddress, clientServerConf, null, 1);
this.server.start(); this.server.start();
super.serviceStart(); super.serviceStart();

View File

@ -26,7 +26,7 @@ import junit.framework.Assert;
import org.apache.hadoop.mapreduce.JobStatus; import org.apache.hadoop.mapreduce.JobStatus;
import org.apache.hadoop.mapreduce.JobStatus.State; 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.GetAllApplicationsRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse;
import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
@ -52,7 +52,7 @@ public class TestResourceMgrDelegate {
*/ */
@Test @Test
public void testGetRootQueues() throws IOException, InterruptedException { 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); GetQueueInfoResponse response = Mockito.mock(GetQueueInfoResponse.class);
org.apache.hadoop.yarn.api.records.QueueInfo queueInfo = org.apache.hadoop.yarn.api.records.QueueInfo queueInfo =
Mockito.mock(org.apache.hadoop.yarn.api.records.QueueInfo.class); Mockito.mock(org.apache.hadoop.yarn.api.records.QueueInfo.class);
@ -90,7 +90,7 @@ public class TestResourceMgrDelegate {
@Test @Test
public void tesAllJobs() throws Exception { public void tesAllJobs() throws Exception {
final ClientRMProtocol applicationsManager = Mockito.mock(ClientRMProtocol.class); final ApplicationClientProtocol applicationsManager = Mockito.mock(ApplicationClientProtocol.class);
GetAllApplicationsResponse allApplicationsResponse = Records GetAllApplicationsResponse allApplicationsResponse = Records
.newRecord(GetAllApplicationsResponse.class); .newRecord(GetAllApplicationsResponse.class);
List<ApplicationReport> applications = new ArrayList<ApplicationReport>(); List<ApplicationReport> applications = new ArrayList<ApplicationReport>();

View File

@ -59,7 +59,7 @@ import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token; 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.GetAllApplicationsRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
@ -197,7 +197,7 @@ public class TestYARNRunner extends TestCase {
@Test(timeout=20000) @Test(timeout=20000)
public void testResourceMgrDelegate() throws Exception { public void testResourceMgrDelegate() throws Exception {
/* we not want a mock of resource mgr delegate */ /* 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) { ResourceMgrDelegate delegate = new ResourceMgrDelegate(conf) {
@Override @Override
protected void serviceStart() { protected void serviceStart() {

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.mapred.ResourceMgrDelegate;
import org.apache.hadoop.mapred.YARNRunner; import org.apache.hadoop.mapred.YARNRunner;
import org.apache.hadoop.mapreduce.protocol.ClientProtocol; import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
import org.apache.hadoop.security.token.Token; 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.GetDelegationTokenRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
@ -104,7 +104,7 @@ public class TestYarnClientProtocolProvider extends TestCase {
rmDTToken.setPassword(ByteBuffer.wrap("testcluster".getBytes())); rmDTToken.setPassword(ByteBuffer.wrap("testcluster".getBytes()));
rmDTToken.setService("0.0.0.0:8032"); rmDTToken.setService("0.0.0.0:8032");
getDTResponse.setRMDelegationToken(rmDTToken); getDTResponse.setRMDelegationToken(rmDTToken);
final ClientRMProtocol cRMProtocol = mock(ClientRMProtocol.class); final ApplicationClientProtocol cRMProtocol = mock(ApplicationClientProtocol.class);
when(cRMProtocol.getDelegationToken(any( when(cRMProtocol.getDelegationToken(any(
GetDelegationTokenRequest.class))).thenReturn(getDTResponse); GetDelegationTokenRequest.class))).thenReturn(getDTResponse);
ResourceMgrDelegate rmgrDelegate = new ResourceMgrDelegate( ResourceMgrDelegate rmgrDelegate = new ResourceMgrDelegate(

View File

@ -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 YARN-823. Moved RMAdmin from yarn.client to yarn.client.cli and renamed it to
be RMAdminCLI. (Jian He via vinodkv) be RMAdminCLI. (Jian He via vinodkv)
YARN-387. Renamed YARN protocols for consistency.
ClientRMProtocol -> ApplicationClientProtocol
AMRMProtocol -> ApplicationMasterProtocol
ContainerManager -> ContainerManagementProtocol
(vinodkv via acmurthy)
NEW FEATURES NEW FEATURES
YARN-482. FS: Extend SchedulingMode to intermediate queues. YARN-482. FS: Extend SchedulingMode to intermediate queues.

View File

@ -54,11 +54,11 @@
<includes> <includes>
<include>yarn_protos.proto</include> <include>yarn_protos.proto</include>
<include>yarn_service_protos.proto</include> <include>yarn_service_protos.proto</include>
<include>AM_RM_protocol.proto</include> <include>applicationmaster_protocol.proto</include>
<include>client_RM_protocol.proto</include> <include>applicationclient_protocol.proto</include>
<include>container_manager.proto</include> <include>containermanagement_protocol.proto</include>
<include>yarn_server_resourcemanager_service_protos.proto</include> <include>yarn_server_resourcemanager_service_protos.proto</include>
<include>RMAdminProtocol.proto</include> <include>resourcemanager_administration_protocol.proto</include>
</includes> </includes>
</source> </source>
<output>${project.build.directory}/generated-sources/java</output> <output>${project.build.directory}/generated-sources/java</output>

View File

@ -63,7 +63,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
*/ */
@Public @Public
@Stable @Stable
public interface ClientRMProtocol { public interface ApplicationClientProtocol {
/** /**
* <p>The interface used by clients to obtain a new {@link ApplicationId} for * <p>The interface used by clients to obtain a new {@link ApplicationId} for
* submitting new applications.</p> * submitting new applications.</p>

View File

@ -18,10 +18,10 @@
package org.apache.hadoop.yarn.api; package org.apache.hadoop.yarn.api;
import org.apache.hadoop.ipc.ProtocolInfo; 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) protocolVersion = 1)
public interface AMRMProtocolPB extends AMRMProtocolService.BlockingInterface { public interface ApplicationClientProtocolPB extends ApplicationClientProtocolService.BlockingInterface {
} }

View File

@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
*/ */
@Public @Public
@Stable @Stable
public interface AMRMProtocol { public interface ApplicationMasterProtocol {
/** /**
* <p>The interface used by a new <code>ApplicationMaster</code> to register * <p>The interface used by a new <code>ApplicationMaster</code> to register

View File

@ -18,11 +18,10 @@
package org.apache.hadoop.yarn.api; package org.apache.hadoop.yarn.api;
import org.apache.hadoop.ipc.ProtocolInfo; import org.apache.hadoop.ipc.ProtocolInfo;
import org.apache.hadoop.yarn.proto.RMAdminProtocol.RMAdminProtocolService; import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService;
@ProtocolInfo( @ProtocolInfo(protocolName = "org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB",
protocolName = "org.apache.hadoop.yarn.api.RMAdminProtocolPB",
protocolVersion = 1) protocolVersion = 1)
public interface RMAdminProtocolPB extends RMAdminProtocolService.BlockingInterface { public interface ApplicationMasterProtocolPB extends ApplicationMasterProtocolService.BlockingInterface {
} }

View File

@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
*/ */
@Public @Public
@Stable @Stable
public interface ContainerManager { public interface ContainerManagementProtocol {
/** /**
* <p>The <code>ApplicationMaster</code> requests a <code>NodeManager</code> * <p>The <code>ApplicationMaster</code> requests a <code>NodeManager</code>
* to <em>start</em> a {@link Container} allocated to it using this interface. * to <em>start</em> a {@link Container} allocated to it using this interface.

View File

@ -18,11 +18,11 @@
package org.apache.hadoop.yarn.api; package org.apache.hadoop.yarn.api;
import org.apache.hadoop.ipc.ProtocolInfo; import org.apache.hadoop.ipc.ProtocolInfo;
import org.apache.hadoop.yarn.proto.ContainerManager.ContainerManagerService; import org.apache.hadoop.yarn.proto.ContainerManagementProtocol.ContainerManagementProtocolService;
@ProtocolInfo( @ProtocolInfo(
protocolName = "org.apache.hadoop.yarn.api.ContainerManagerPB", protocolName = "org.apache.hadoop.yarn.api.ContainerManagementProtocolPB",
protocolVersion = 1) protocolVersion = 1)
public interface ContainerManagerPB extends ContainerManagerService.BlockingInterface { public interface ContainerManagementProtocolPB extends ContainerManagementProtocolService.BlockingInterface {
} }

View File

@ -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.RefreshUserToGroupsMappingsRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsResponse; import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
public interface RMAdminProtocol extends GetUserMappingsProtocol { public interface ResourceManagerAdministrationProtocol extends GetUserMappingsProtocol {
public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request) public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
throws YarnException, IOException; throws YarnException, IOException;

View File

@ -18,10 +18,11 @@
package org.apache.hadoop.yarn.api; package org.apache.hadoop.yarn.api;
import org.apache.hadoop.ipc.ProtocolInfo; 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) protocolVersion = 1)
public interface ClientRMProtocolPB extends ClientRMProtocolService.BlockingInterface { public interface ResourceManagerAdministrationProtocolPB extends ResourceManagerAdministrationProtocolService.BlockingInterface {
} }

View File

@ -22,7 +22,7 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
@ -53,7 +53,7 @@ import org.apache.hadoop.yarn.util.Records;
* </ul> * </ul>
* </p> * </p>
* *
* @see AMRMProtocol#allocate(AllocateRequest) * @see ApplicationMasterProtocol#allocate(AllocateRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Evolving;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.AMCommand;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
@ -57,7 +57,7 @@ import org.apache.hadoop.yarn.util.Records;
* </ul> * </ul>
* </p> * </p>
* *
* @see AMRMProtocol#allocate(AllocateRequest) * @see ApplicationMasterProtocol#allocate(AllocateRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.util.Records;
* </ul> * </ul>
* </p> * </p>
* *
* @see AMRMProtocol#finishApplicationMaster(FinishApplicationMasterRequest) * @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest)
*/ */
public abstract class FinishApplicationMasterRequest { public abstract class FinishApplicationMasterRequest {

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records;
* *
* <p>Currently, this is empty.</p> * <p>Currently, this is empty.</p>
* *
* @see AMRMProtocol#finishApplicationMaster(FinishApplicationMasterRequest) * @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records;
* *
* <p>Currently, this is empty.</p> * <p>Currently, this is empty.</p>
* *
* @see ClientRMProtocol#getAllApplications(GetAllApplicationsRequest) * @see ApplicationClientProtocol#getAllApplications(GetAllApplicationsRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.api.records.ApplicationReport;
import org.apache.hadoop.yarn.util.Records; 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.</p> * is running, RPC port, tracking URL, diagnostics, start time etc.</p>
* *
* @see ApplicationReport * @see ApplicationReport
* @see ClientRMProtocol#getAllApplications(GetAllApplicationsRequest) * @see ApplicationClientProtocol#getAllApplications(GetAllApplicationsRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.util.Records;
* <p>The request should include the {@link ApplicationId} of the * <p>The request should include the {@link ApplicationId} of the
* application.</p> * application.</p>
* *
* @see ClientRMProtocol#getApplicationReport(GetApplicationReportRequest) * @see ApplicationClientProtocol#getApplicationReport(GetApplicationReportRequest)
* @see ApplicationReport * @see ApplicationReport
*/ */
@Public @Public

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.api.records.ApplicationReport;
import org.apache.hadoop.yarn.util.Records; 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 <code>ApplicationMaster</code> is * as user, queue, name, host on which the <code>ApplicationMaster</code> is
* running, RPC port, tracking URL, diagnostics, start time etc.</p> * running, RPC port, tracking URL, diagnostics, start time etc.</p>
* *
* @see ClientRMProtocol#getApplicationReport(GetApplicationReportRequest) * @see ApplicationClientProtocol#getApplicationReport(GetApplicationReportRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records;
* *
* <p>Currently, this is empty.</p> * <p>Currently, this is empty.</p>
* *
* @see ClientRMProtocol#getClusterMetrics(GetClusterMetricsRequest) * @see ApplicationClientProtocol#getClusterMetrics(GetClusterMetricsRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.api.records.YarnClusterMetrics;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.util.Records;
* requesting cluster metrics.<p> * requesting cluster metrics.<p>
* *
* @see YarnClusterMetrics * @see YarnClusterMetrics
* @see ClientRMProtocol#getClusterMetrics(GetClusterMetricsRequest) * @see ApplicationClientProtocol#getClusterMetrics(GetClusterMetricsRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records;
* *
* <p>Currently, this is empty.</p> * <p>Currently, this is empty.</p>
* *
* @see ClientRMProtocol#getClusterNodes(GetClusterNodesRequest) * @see ApplicationClientProtocol#getClusterNodes(GetClusterNodesRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.api.records.NodeReport;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.util.Records;
* status etc. * status etc.
* *
* @see NodeReport * @see NodeReport
* @see ClientRMProtocol#getClusterNodes(GetClusterNodesRequest) * @see ApplicationClientProtocol#getClusterNodes(GetClusterNodesRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records;
* <p>The request sent by the <code>ApplicationMaster</code> to the * <p>The request sent by the <code>ApplicationMaster</code> to the
* <code>NodeManager</code> to get {@link ContainerStatus} of a container.</p> * <code>NodeManager</code> to get {@link ContainerStatus} of a container.</p>
* *
* @see ContainerManager#getContainerStatus(GetContainerStatusRequest) * @see ContainerManagementProtocol#getContainerStatus(GetContainerStatusRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.api.records.ContainerStatus;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.util.Records;
* <code>ApplicationMaster</code> when asked to obtain the <em>status</em> * <code>ApplicationMaster</code> when asked to obtain the <em>status</em>
* of a container.</p> * of a container.</p>
* *
* @see ContainerManager#getContainerStatus(GetContainerStatusRequest) * @see ContainerManagementProtocol#getContainerStatus(GetContainerStatusRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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.ApplicationId;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.util.Records;
* *
* <p>Currently, this is empty.</p> * <p>Currently, this is empty.</p>
* *
* @see ClientRMProtocol#getNewApplication(GetNewApplicationRequest) * @see ApplicationClientProtocol#getNewApplication(GetNewApplicationRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.ApplicationId;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.util.Records;
* <p>The response sent by the <code>ResourceManager</code> to the client for * <p>The response sent by the <code>ResourceManager</code> to the client for
* a request to get a new {@link ApplicationId} for submitting applications.</p> * a request to get a new {@link ApplicationId} for submitting applications.</p>
* *
* @see ClientRMProtocol#getNewApplication(GetNewApplicationRequest) * @see ApplicationClientProtocol#getNewApplication(GetNewApplicationRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,14 +20,14 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
* <p>The request sent by clients to get <em>queue information</em> * <p>The request sent by clients to get <em>queue information</em>
* from the <code>ResourceManager</code>.</p> * from the <code>ResourceManager</code>.</p>
* *
* @see ClientRMProtocol#getQueueInfo(GetQueueInfoRequest) * @see ApplicationClientProtocol#getQueueInfo(GetQueueInfoRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.api.records.QueueInfo;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.util.Records;
* .</p> * .</p>
* *
* @see QueueInfo * @see QueueInfo
* @see ClientRMProtocol#getQueueInfo(GetQueueInfoRequest) * @see ApplicationClientProtocol#getQueueInfo(GetQueueInfoRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records;
* *
* <p>Currently, this is empty.</p> * <p>Currently, this is empty.</p>
* *
* @see ClientRMProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest) * @see ApplicationClientProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.QueueACL;
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -38,7 +38,7 @@ import org.apache.hadoop.yarn.util.Records;
* *
* @see QueueACL * @see QueueACL
* @see QueueUserACLInfo * @see QueueUserACLInfo
* @see ClientRMProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest) * @see ApplicationClientProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.ApplicationId;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -33,7 +33,7 @@ import org.apache.hadoop.yarn.util.Records;
* <p>The request includes the {@link ApplicationId} of the application to be * <p>The request includes the {@link ApplicationId} of the application to be
* aborted.</p> * aborted.</p>
* *
* @see ClientRMProtocol#forceKillApplication(KillApplicationRequest) * @see ApplicationClientProtocol#forceKillApplication(KillApplicationRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records;
* *
* <p>Currently it's empty.</p> * <p>Currently it's empty.</p>
* *
* @see ClientRMProtocol#forceKillApplication(KillApplicationRequest) * @see ApplicationClientProtocol#forceKillApplication(KillApplicationRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -42,7 +42,7 @@ import org.apache.hadoop.yarn.util.Records;
* </ul> * </ul>
* </p> * </p>
* *
* @see AMRMProtocol#registerApplicationMaster(RegisterApplicationMasterRequest) * @see ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.util.Records;
* </ul> * </ul>
* </p> * </p>
* *
* @see AMRMProtocol#registerApplicationMaster(RegisterApplicationMasterRequest) * @see ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.util.Records; 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 * necessary binaries/jar/shared-objects etc. via the
* {@link ContainerLaunchContext}.</p> * {@link ContainerLaunchContext}.</p>
* *
* @see ContainerManager#startContainer(StartContainerRequest) * @see ContainerManagementProtocol#startContainer(StartContainerRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -23,7 +23,7 @@ import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.util.Records;
* <code>ApplicationMaster</code> when asked to <em>start</em> an * <code>ApplicationMaster</code> when asked to <em>start</em> an
* allocated container.</p> * allocated container.</p>
* *
* @see ContainerManager#startContainer(StartContainerRequest) * @see ContainerManagementProtocol#startContainer(StartContainerRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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.ContainerId;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.util.Records;
* <p>The request sent by the <code>ApplicationMaster</code> to the * <p>The request sent by the <code>ApplicationMaster</code> to the
* <code>NodeManager</code> to <em>stop</em> a container.</p> * <code>NodeManager</code> to <em>stop</em> a container.</p>
* *
* @see ContainerManager#stopContainer(StopContainerRequest) * @see ContainerManagementProtocol#stopContainer(StopContainerRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.util.Records;
* *
* <p>Currently, this is empty.</p> * <p>Currently, this is empty.</p>
* *
* @see ContainerManager#stopContainer(StopContainerRequest) * @see ContainerManagementProtocol#stopContainer(StopContainerRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.util.Records;
* {@link ContainerLaunchContext} for launching the * {@link ContainerLaunchContext} for launching the
* <code>ApplicationMaster</code> etc. * <code>ApplicationMaster</code> etc.
* *
* @see ClientRMProtocol#submitApplication(SubmitApplicationRequest) * @see ApplicationClientProtocol#submitApplication(SubmitApplicationRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records;
* *
* <p>Currently, this is empty.</p> * <p>Currently, this is empty.</p>
* *
* @see ClientRMProtocol#submitApplication(SubmitApplicationRequest) * @see ApplicationClientProtocol#submitApplication(SubmitApplicationRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.util.Records;
* </ul> * </ul>
* </p> * </p>
* *
* @see ClientRMProtocol#getApplicationReport(org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest) * @see ApplicationClientProtocol#getApplicationReport(org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.util.Records;
* </p> * </p>
* *
* @see ContainerLaunchContext * @see ContainerLaunchContext
* @see ClientRMProtocol#submitApplication(org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest) * @see ApplicationClientProtocol#submitApplication(org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -22,8 +22,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.ContainerManager; import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
import org.apache.hadoop.yarn.util.Records; 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 <code>NodManager</code> to * resource-negotiation and then talks to the <code>NodManager</code> to
* start/stop containers.</p> * start/stop containers.</p>
* *
* @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
* @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
* @see ContainerManager#stopContainer(org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest) * @see ContainerManagementProtocol#stopContainer(org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest)
*/ */
@Public @Public
@Stable @Stable
@ -153,8 +153,8 @@ public abstract class Container implements Comparable<Container> {
* are transparently handled by the framework - the allocated * are transparently handled by the framework - the allocated
* <code>Container</code> includes the <code>ContainerToken</code>.</p> * <code>Container</code> includes the <code>ContainerToken</code>.</p>
* *
* @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
* @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
* *
* @return <code>ContainerToken</code> for the container * @return <code>ContainerToken</code> for the container
*/ */

View File

@ -24,7 +24,7 @@ import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.util.Records;
* </ul> * </ul>
* </p> * </p>
* *
* @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.util.Records;
* @see LocalResourceVisibility * @see LocalResourceVisibility
* @see ContainerLaunchContext * @see ContainerLaunchContext
* @see ApplicationSubmissionContext * @see ApplicationSubmissionContext
* @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.yarn.api.ContainerManager; import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
/** /**
* <p><code>LocalResourceType</code> specifies the <em>type</em> * <p><code>LocalResourceType</code> specifies the <em>type</em>
@ -41,7 +41,7 @@ import org.apache.hadoop.yarn.api.ContainerManager;
* @see LocalResource * @see LocalResource
* @see ContainerLaunchContext * @see ContainerLaunchContext
* @see ApplicationSubmissionContext * @see ApplicationSubmissionContext
* @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.yarn.api.ContainerManager; import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
/** /**
* <p><code>LocalResourceVisibility</code> specifies the <em>visibility</em> * <p><code>LocalResourceVisibility</code> specifies the <em>visibility</em>
@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.ContainerManager;
* @see LocalResource * @see LocalResource
* @see ContainerLaunchContext * @see ContainerLaunchContext
* @see ApplicationSubmissionContext * @see ApplicationSubmissionContext
* @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.util.Records;
* </ul> * </ul>
* </p> * </p>
* *
* @see ClientRMProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest) * @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.yarn.api.ClientRMProtocol; import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
/** /**
* <p> * <p>
@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
* </p> * </p>
* *
* @see QueueInfo * @see QueueInfo
* @see ClientRMProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest) * @see ApplicationClientProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.util.Records;
* </p> * </p>
* *
* @see QueueState * @see QueueState
* @see ClientRMProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest) * @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.yarn.api.ClientRMProtocol; import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
/** /**
* <p>State of a Queue.</p> * <p>State of a Queue.</p>
@ -33,7 +33,7 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
* </p> * </p>
* *
* @see QueueInfo * @see QueueInfo
* @see ClientRMProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest) * @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.util.Records;
* the given user.</p> * the given user.</p>
* *
* @see QueueACL * @see QueueACL
* @see ClientRMProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest) * @see ApplicationClientProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -21,7 +21,7 @@ package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Evolving;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.util.Records;
* capability to run their component tasks.</p> * capability to run their component tasks.</p>
* *
* @see ResourceRequest * @see ResourceRequest
* @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -21,7 +21,7 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.util.Records;
* for the application. * for the application.
* *
* @see ResourceRequest * @see ResourceRequest
* @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -22,7 +22,7 @@ import java.io.Serializable;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; 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; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -53,7 +53,7 @@ import org.apache.hadoop.yarn.util.Records;
* </p> * </p>
* *
* @see Resource * @see Resource
* @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
*/ */
@Public @Public
@Stable @Stable

View File

@ -606,21 +606,21 @@ public class YarnConfiguration extends Configuration {
* YARN Service Level Authorization * YARN Service Level Authorization
*/ */
public static final String public static final String
YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER = YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL =
"security.resourcetracker.protocol.acl"; "security.resourcetracker.protocol.acl";
public static final String public static final String
YARN_SECURITY_SERVICE_AUTHORIZATION_CLIENT_RESOURCEMANAGER = YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONCLIENT_PROTOCOL =
"security.client.resourcemanager.protocol.acl"; "security.applicationclient.protocol.acl";
public static final String public static final String
YARN_SECURITY_SERVICE_AUTHORIZATION_ADMIN = YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCEMANAGER_ADMINISTRATION_PROTOCOL =
"security.admin.protocol.acl"; "security.resourcemanager-administration.protocol.acl";
public static final String public static final String
YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONMASTER_RESOURCEMANAGER = YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONMASTER_PROTOCOL =
"security.applicationmaster.resourcemanager.protocol.acl"; "security.applicationmaster.protocol.acl";
public static final String public static final String
YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGER = YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL =
"security.containermanager.protocol.acl"; "security.containermanagement.protocol.acl";
public static final String public static final String
YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER = YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER =
"security.resourcelocalizer.protocol.acl"; "security.resourcelocalizer.protocol.acl";

View File

@ -17,14 +17,14 @@
*/ */
option java_package = "org.apache.hadoop.yarn.proto"; 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_generic_services = true;
option java_generate_equals_and_hash = true; option java_generate_equals_and_hash = true;
import "Security.proto"; import "Security.proto";
import "yarn_service_protos.proto"; import "yarn_service_protos.proto";
service ClientRMProtocolService { service ApplicationClientProtocolService {
rpc getNewApplication (GetNewApplicationRequestProto) returns (GetNewApplicationResponseProto); rpc getNewApplication (GetNewApplicationRequestProto) returns (GetNewApplicationResponseProto);
rpc getApplicationReport (GetApplicationReportRequestProto) returns (GetApplicationReportResponseProto); rpc getApplicationReport (GetApplicationReportRequestProto) returns (GetApplicationReportResponseProto);
rpc submitApplication (SubmitApplicationRequestProto) returns (SubmitApplicationResponseProto); rpc submitApplication (SubmitApplicationRequestProto) returns (SubmitApplicationResponseProto);

View File

@ -17,14 +17,14 @@
*/ */
option java_package = "org.apache.hadoop.yarn.proto"; 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_generic_services = true;
option java_generate_equals_and_hash = true; option java_generate_equals_and_hash = true;
import "yarn_service_protos.proto"; import "yarn_service_protos.proto";
service AMRMProtocolService { service ApplicationMasterProtocolService {
rpc registerApplicationMaster (RegisterApplicationMasterRequestProto) returns (RegisterApplicationMasterResponseProto); rpc registerApplicationMaster (RegisterApplicationMasterRequestProto) returns (RegisterApplicationMasterResponseProto);
rpc finishApplicationMaster (FinishApplicationMasterRequestProto) returns (FinishApplicationMasterResponseProto); rpc finishApplicationMaster (FinishApplicationMasterRequestProto) returns (FinishApplicationMasterResponseProto);
rpc allocate (AllocateRequestProto) returns (AllocateResponseProto); rpc allocate (AllocateRequestProto) returns (AllocateResponseProto);

View File

@ -17,13 +17,13 @@
*/ */
option java_package = "org.apache.hadoop.yarn.proto"; 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_generic_services = true;
option java_generate_equals_and_hash = true; option java_generate_equals_and_hash = true;
import "yarn_service_protos.proto"; import "yarn_service_protos.proto";
service ContainerManagerService { service ContainerManagementProtocolService {
rpc startContainer(StartContainerRequestProto) returns (StartContainerResponseProto); rpc startContainer(StartContainerRequestProto) returns (StartContainerResponseProto);
rpc stopContainer(StopContainerRequestProto) returns (StopContainerResponseProto); rpc stopContainer(StopContainerRequestProto) returns (StopContainerResponseProto);
rpc getContainerStatus(GetContainerStatusRequestProto) returns (GetContainerStatusResponseProto); rpc getContainerStatus(GetContainerStatusRequestProto) returns (GetContainerStatusResponseProto);

View File

@ -17,13 +17,13 @@
*/ */
option java_package = "org.apache.hadoop.yarn.proto"; 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_generic_services = true;
option java_generate_equals_and_hash = true; option java_generate_equals_and_hash = true;
import "yarn_server_resourcemanager_service_protos.proto"; import "yarn_server_resourcemanager_service_protos.proto";
service RMAdminProtocolService { service ResourceManagerAdministrationProtocolService {
rpc refreshQueues(RefreshQueuesRequestProto) returns (RefreshQueuesResponseProto); rpc refreshQueues(RefreshQueuesRequestProto) returns (RefreshQueuesResponseProto);
rpc refreshNodes(RefreshNodesRequestProto) returns (RefreshNodesResponseProto); rpc refreshNodes(RefreshNodesRequestProto) returns (RefreshNodesResponseProto);
rpc refreshSuperUserGroupsConfiguration(RefreshSuperUserGroupsConfigurationRequestProto) returns (RefreshSuperUserGroupsConfigurationResponseProto); rpc refreshSuperUserGroupsConfiguration(RefreshSuperUserGroupsConfigurationRequestProto) returns (RefreshSuperUserGroupsConfigurationResponseProto);

View File

@ -43,10 +43,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; 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;
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; 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.AllocateRequest;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
@ -99,7 +99,7 @@ import org.apache.hadoop.yarn.util.Records;
* The <code>ApplicationMaster</code> needs to send a heartbeat to the * The <code>ApplicationMaster</code> needs to send a heartbeat to the
* <code>ResourceManager</code> at regular intervals to inform the * <code>ResourceManager</code> at regular intervals to inform the
* <code>ResourceManager</code> that it is up and alive. The * <code>ResourceManager</code> that it is up and alive. The
* {@link AMRMProtocol#allocate} to the <code>ResourceManager</code> from the * {@link ApplicationMasterProtocol#allocate} to the <code>ResourceManager</code> from the
* <code>ApplicationMaster</code> acts as a heartbeat. * <code>ApplicationMaster</code> acts as a heartbeat.
* *
* <p> * <p>
@ -118,15 +118,15 @@ import org.apache.hadoop.yarn.util.Records;
* up the necessary launch context via {@link ContainerLaunchContext} to specify * up the necessary launch context via {@link ContainerLaunchContext} to specify
* the allocated container id, local resources required by the executable, the * the allocated container id, local resources required by the executable, the
* environment to be setup for the executable, commands to execute, etc. and * 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. * launch and execute the defined commands on the given allocated container.
* </p> * </p>
* *
* <p> * <p>
* The <code>ApplicationMaster</code> can monitor the launched container by * The <code>ApplicationMaster</code> can monitor the launched container by
* either querying the <code>ResourceManager</code> using * either querying the <code>ResourceManager</code> using
* {@link AMRMProtocol#allocate} to get updates on completed containers or via * {@link ApplicationMasterProtocol#allocate} to get updates on completed containers or via
* the {@link ContainerManager} by querying for the status of the allocated * the {@link ContainerManagementProtocol} by querying for the status of the allocated
* container's {@link ContainerId}. * container's {@link ContainerId}.
* *
* <p> * <p>
@ -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. * that will execute the shell command.
*/ */
private class LaunchContainerRunnable implements Runnable { private class LaunchContainerRunnable implements Runnable {

View File

@ -41,7 +41,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; 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.GetNewApplicationResponse;
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -75,7 +75,7 @@ import org.apache.hadoop.yarn.util.Records;
* <p>This client is meant to act as an example on how to write yarn-based applications. </p> * <p>This client is meant to act as an example on how to write yarn-based applications. </p>
* *
* <p> To submit an application, a client first needs to connect to the <code>ResourceManager</code> * <p> To submit an application, a client first needs to connect to the <code>ResourceManager</code>
* 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 * provides a way for the client to get access to cluster information and to request for a
* new {@link ApplicationId}. <p> * new {@link ApplicationId}. <p>
* *

View File

@ -43,7 +43,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.UserGroupInformation; 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.AllocateRequest;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
@ -85,7 +85,7 @@ public class AMRMClientImpl<T extends ContainerRequest>
private int lastResponseId = 0; private int lastResponseId = 0;
private ConcurrentHashMap<String, Token> nmTokens; private ConcurrentHashMap<String, Token> nmTokens;
protected AMRMProtocol rmClient; protected ApplicationMasterProtocol rmClient;
protected final ApplicationAttemptId appAttemptId; protected final ApplicationAttemptId appAttemptId;
protected Resource clusterAvailableResources; protected Resource clusterAvailableResources;
protected int clusterNodeCount; protected int clusterNodeCount;
@ -185,10 +185,10 @@ public class AMRMClientImpl<T extends ContainerRequest>
} }
// CurrentUser should already have AMToken loaded. // CurrentUser should already have AMToken loaded.
rmClient = currentUser.doAs(new PrivilegedAction<AMRMProtocol>() { rmClient = currentUser.doAs(new PrivilegedAction<ApplicationMasterProtocol>() {
@Override @Override
public AMRMProtocol run() { public ApplicationMasterProtocol run() {
return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class, rmAddress, return (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class, rmAddress,
conf); conf);
} }
}); });

View File

@ -32,7 +32,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation; 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.GetContainerStatusRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@ -160,7 +160,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
private ContainerId containerId; private ContainerId containerId;
private NodeId nodeId; private NodeId nodeId;
private Token containerToken; private Token containerToken;
private ContainerManager containerManager; private ContainerManagementProtocol containerManager;
public NMCommunicator(ContainerId containerId, NodeId nodeId, public NMCommunicator(ContainerId containerId, NodeId nodeId,
Token containerToken) { Token containerToken) {
@ -186,10 +186,10 @@ public class NMClientImpl extends AbstractService implements NMClient {
currentUser.addToken(token); currentUser.addToken(token);
containerManager = currentUser containerManager = currentUser
.doAs(new PrivilegedAction<ContainerManager>() { .doAs(new PrivilegedAction<ContainerManagementProtocol>() {
@Override @Override
public ContainerManager run() { public ContainerManagementProtocol run() {
return (ContainerManager) rpc.getProxy(ContainerManager.class, return (ContainerManagementProtocol) rpc.getProxy(ContainerManagementProtocol.class,
containerAddress, getConfig()); containerAddress, getConfig());
} }
}); });

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RPC; 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.GetAllApplicationsRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; 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); private static final Log LOG = LogFactory.getLog(YarnClientImpl.class);
protected ClientRMProtocol rmClient; protected ApplicationClientProtocol rmClient;
protected InetSocketAddress rmAddress; protected InetSocketAddress rmAddress;
protected long statePollIntervalMillis; protected long statePollIntervalMillis;
@ -107,8 +107,8 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
protected void serviceStart() throws Exception { protected void serviceStart() throws Exception {
YarnRPC rpc = YarnRPC.create(getConfig()); YarnRPC rpc = YarnRPC.create(getConfig());
this.rmClient = (ClientRMProtocol) rpc.getProxy( this.rmClient = (ApplicationClientProtocol) rpc.getProxy(
ClientRMProtocol.class, rmAddress, getConfig()); ApplicationClientProtocol.class, rmAddress, getConfig());
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Connecting to ResourceManager at " + rmAddress); LOG.debug("Connecting to ResourceManager at " + rmAddress);
} }

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner; 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.RefreshAdminAclsRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesRequest; import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RefreshQueuesRequest; import org.apache.hadoop.yarn.api.protocolrecords.RefreshQueuesRequest;
@ -165,7 +165,7 @@ public class RMAdminCLI extends Configured implements Tool {
return UserGroupInformation.getCurrentUser(); return UserGroupInformation.getCurrentUser();
} }
private RMAdminProtocol createAdminProtocol() throws IOException { private ResourceManagerAdministrationProtocol createAdminProtocol() throws IOException {
// Get the current configuration // Get the current configuration
final YarnConfiguration conf = new YarnConfiguration(getConf()); final YarnConfiguration conf = new YarnConfiguration(getConf());
@ -176,11 +176,11 @@ public class RMAdminCLI extends Configured implements Tool {
YarnConfiguration.DEFAULT_RM_ADMIN_PORT); YarnConfiguration.DEFAULT_RM_ADMIN_PORT);
final YarnRPC rpc = YarnRPC.create(conf); final YarnRPC rpc = YarnRPC.create(conf);
RMAdminProtocol adminProtocol = ResourceManagerAdministrationProtocol adminProtocol =
getUGI(conf).doAs(new PrivilegedAction<RMAdminProtocol>() { getUGI(conf).doAs(new PrivilegedAction<ResourceManagerAdministrationProtocol>() {
@Override @Override
public RMAdminProtocol run() { public ResourceManagerAdministrationProtocol run() {
return (RMAdminProtocol) rpc.getProxy(RMAdminProtocol.class, return (ResourceManagerAdministrationProtocol) rpc.getProxy(ResourceManagerAdministrationProtocol.class,
addr, conf); addr, conf);
} }
}); });
@ -190,7 +190,7 @@ public class RMAdminCLI extends Configured implements Tool {
private int refreshQueues() throws IOException, YarnException { private int refreshQueues() throws IOException, YarnException {
// Refresh the queue properties // Refresh the queue properties
RMAdminProtocol adminProtocol = createAdminProtocol(); ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol();
RefreshQueuesRequest request = RefreshQueuesRequest request =
recordFactory.newRecordInstance(RefreshQueuesRequest.class); recordFactory.newRecordInstance(RefreshQueuesRequest.class);
adminProtocol.refreshQueues(request); adminProtocol.refreshQueues(request);
@ -199,7 +199,7 @@ public class RMAdminCLI extends Configured implements Tool {
private int refreshNodes() throws IOException, YarnException { private int refreshNodes() throws IOException, YarnException {
// Refresh the nodes // Refresh the nodes
RMAdminProtocol adminProtocol = createAdminProtocol(); ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol();
RefreshNodesRequest request = RefreshNodesRequest request =
recordFactory.newRecordInstance(RefreshNodesRequest.class); recordFactory.newRecordInstance(RefreshNodesRequest.class);
adminProtocol.refreshNodes(request); adminProtocol.refreshNodes(request);
@ -209,7 +209,7 @@ public class RMAdminCLI extends Configured implements Tool {
private int refreshUserToGroupsMappings() throws IOException, private int refreshUserToGroupsMappings() throws IOException,
YarnException { YarnException {
// Refresh the user-to-groups mappings // Refresh the user-to-groups mappings
RMAdminProtocol adminProtocol = createAdminProtocol(); ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol();
RefreshUserToGroupsMappingsRequest request = RefreshUserToGroupsMappingsRequest request =
recordFactory.newRecordInstance(RefreshUserToGroupsMappingsRequest.class); recordFactory.newRecordInstance(RefreshUserToGroupsMappingsRequest.class);
adminProtocol.refreshUserToGroupsMappings(request); adminProtocol.refreshUserToGroupsMappings(request);
@ -219,7 +219,7 @@ public class RMAdminCLI extends Configured implements Tool {
private int refreshSuperUserGroupsConfiguration() throws IOException, private int refreshSuperUserGroupsConfiguration() throws IOException,
YarnException { YarnException {
// Refresh the super-user groups // Refresh the super-user groups
RMAdminProtocol adminProtocol = createAdminProtocol(); ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol();
RefreshSuperUserGroupsConfigurationRequest request = RefreshSuperUserGroupsConfigurationRequest request =
recordFactory.newRecordInstance(RefreshSuperUserGroupsConfigurationRequest.class); recordFactory.newRecordInstance(RefreshSuperUserGroupsConfigurationRequest.class);
adminProtocol.refreshSuperUserGroupsConfiguration(request); adminProtocol.refreshSuperUserGroupsConfiguration(request);
@ -228,7 +228,7 @@ public class RMAdminCLI extends Configured implements Tool {
private int refreshAdminAcls() throws IOException, YarnException { private int refreshAdminAcls() throws IOException, YarnException {
// Refresh the admin acls // Refresh the admin acls
RMAdminProtocol adminProtocol = createAdminProtocol(); ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol();
RefreshAdminAclsRequest request = RefreshAdminAclsRequest request =
recordFactory.newRecordInstance(RefreshAdminAclsRequest.class); recordFactory.newRecordInstance(RefreshAdminAclsRequest.class);
adminProtocol.refreshAdminAcls(request); adminProtocol.refreshAdminAcls(request);
@ -237,7 +237,7 @@ public class RMAdminCLI extends Configured implements Tool {
private int refreshServiceAcls() throws IOException, YarnException { private int refreshServiceAcls() throws IOException, YarnException {
// Refresh the service acls // Refresh the service acls
RMAdminProtocol adminProtocol = createAdminProtocol(); ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol();
RefreshServiceAclsRequest request = RefreshServiceAclsRequest request =
recordFactory.newRecordInstance(RefreshServiceAclsRequest.class); recordFactory.newRecordInstance(RefreshServiceAclsRequest.class);
adminProtocol.refreshServiceAcls(request); adminProtocol.refreshServiceAcls(request);
@ -246,7 +246,7 @@ public class RMAdminCLI extends Configured implements Tool {
private int getGroups(String[] usernames) throws IOException { private int getGroups(String[] usernames) throws IOException {
// Get groups users belongs to // Get groups users belongs to
RMAdminProtocol adminProtocol = createAdminProtocol(); ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol();
if (usernames.length == 0) { if (usernames.length == 0) {
usernames = new String[] { UserGroupInformation.getCurrentUser().getUserName() }; usernames = new String[] { UserGroupInformation.getCurrentUser().getUserName() };

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.tools.GetGroupsBase; import org.apache.hadoop.tools.GetGroupsBase;
import org.apache.hadoop.tools.GetUserMappingsProtocol; import org.apache.hadoop.tools.GetUserMappingsProtocol;
import org.apache.hadoop.util.ToolRunner; 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.conf.YarnConfiguration;
import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.ipc.YarnRPC;
@ -63,8 +63,8 @@ public class GetGroupsForTesting extends GetGroupsBase {
YarnConfiguration.DEFAULT_RM_ADMIN_PORT); YarnConfiguration.DEFAULT_RM_ADMIN_PORT);
final YarnRPC rpc = YarnRPC.create(conf); final YarnRPC rpc = YarnRPC.create(conf);
RMAdminProtocol adminProtocol = (RMAdminProtocol) rpc.getProxy( ResourceManagerAdministrationProtocol adminProtocol = (ResourceManagerAdministrationProtocol) rpc.getProxy(
RMAdminProtocol.class, addr, getConf()); ResourceManagerAdministrationProtocol.class, addr, getConf());
return adminProtocol; return adminProtocol;
} }

View File

@ -35,7 +35,7 @@ import java.util.concurrent.ConcurrentHashMap;
import junit.framework.Assert; import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration; 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.AllocateRequest;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
@ -550,9 +550,9 @@ public class TestAMRMClient {
snoopRequest = amClient.ask.iterator().next(); snoopRequest = amClient.ask.iterator().next();
assertTrue(snoopRequest.getNumContainers() == 2); assertTrue(snoopRequest.getNumContainers() == 2);
AMRMProtocol realRM = amClient.rmClient; ApplicationMasterProtocol realRM = amClient.rmClient;
try { try {
AMRMProtocol mockRM = mock(AMRMProtocol.class); ApplicationMasterProtocol mockRM = mock(ApplicationMasterProtocol.class);
when(mockRM.allocate(any(AllocateRequest.class))).thenAnswer( when(mockRM.allocate(any(AllocateRequest.class))).thenAnswer(
new Answer<AllocateResponse>() { new Answer<AllocateResponse>() {
public AllocateResponse answer(InvocationOnMock invocation) public AllocateResponse answer(InvocationOnMock invocation)

View File

@ -30,7 +30,7 @@ import java.util.HashMap;
import junit.framework.Assert; import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration; 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.GetApplicationReportRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
@ -146,7 +146,7 @@ public class TestYarnClient {
@Override @Override
public void start() { public void start() {
rmClient = mock(ClientRMProtocol.class); rmClient = mock(ApplicationClientProtocol.class);
GetApplicationReportResponse mockResponse = GetApplicationReportResponse mockResponse =
mock(GetApplicationReportResponse.class); mock(GetApplicationReportResponse.class);
mockReport = mock(ApplicationReport.class); mockReport = mock(ApplicationReport.class);

View File

@ -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.CancelDelegationTokenRequestProto;
import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto; import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto; import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
import org.apache.hadoop.yarn.api.ClientRMProtocol; import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
import org.apache.hadoop.yarn.api.ClientRMProtocolPB; import org.apache.hadoop.yarn.api.ApplicationClientProtocolPB;
import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; 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; import com.google.protobuf.ServiceException;
public class ClientRMProtocolPBClientImpl implements ClientRMProtocol, public class ApplicationClientProtocolPBClientImpl implements ApplicationClientProtocol,
Closeable { Closeable {
private ClientRMProtocolPB proxy; private ApplicationClientProtocolPB proxy;
public ClientRMProtocolPBClientImpl(long clientVersion, public ApplicationClientProtocolPBClientImpl(long clientVersion,
InetSocketAddress addr, Configuration conf) throws IOException { InetSocketAddress addr, Configuration conf) throws IOException {
RPC.setProtocolEngine(conf, ClientRMProtocolPB.class, RPC.setProtocolEngine(conf, ApplicationClientProtocolPB.class,
ProtobufRpcEngine.class); ProtobufRpcEngine.class);
proxy = RPC.getProxy(ClientRMProtocolPB.class, clientVersion, addr, conf); proxy = RPC.getProxy(ApplicationClientProtocolPB.class, clientVersion, addr, conf);
} }
@Override @Override

View File

@ -25,8 +25,8 @@ import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.yarn.api.AMRMProtocol; import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
import org.apache.hadoop.yarn.api.AMRMProtocolPB; import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; 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; 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 { Configuration conf) throws IOException {
RPC.setProtocolEngine(conf, AMRMProtocolPB.class, ProtobufRpcEngine.class); RPC.setProtocolEngine(conf, ApplicationMasterProtocolPB.class, ProtobufRpcEngine.class);
proxy = proxy =
(AMRMProtocolPB) RPC.getProxy(AMRMProtocolPB.class, clientVersion, (ApplicationMasterProtocolPB) RPC.getProxy(ApplicationMasterProtocolPB.class, clientVersion,
addr, conf); addr, conf);
} }

View File

@ -27,8 +27,8 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation; 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.ContainerManagerPB; import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; 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; import com.google.protobuf.ServiceException;
public class ContainerManagerPBClientImpl implements ContainerManager, public class ContainerManagementProtocolPBClientImpl implements ContainerManagementProtocol,
Closeable { Closeable {
// Not a documented config. Only used for tests // Not a documented config. Only used for tests
@ -62,17 +62,17 @@ public class ContainerManagerPBClientImpl implements ContainerManager,
*/ */
static final int DEFAULT_COMMAND_TIMEOUT = 60000; 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 { InetSocketAddress addr, Configuration conf) throws IOException {
RPC.setProtocolEngine(conf, ContainerManagerPB.class, RPC.setProtocolEngine(conf, ContainerManagementProtocolPB.class,
ProtobufRpcEngine.class); ProtobufRpcEngine.class);
UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
int expireIntvl = conf.getInt(NM_COMMAND_TIMEOUT, DEFAULT_COMMAND_TIMEOUT); int expireIntvl = conf.getInt(NM_COMMAND_TIMEOUT, DEFAULT_COMMAND_TIMEOUT);
proxy = proxy =
(ContainerManagerPB) RPC.getProxy(ContainerManagerPB.class, (ContainerManagementProtocolPB) RPC.getProxy(ContainerManagementProtocolPB.class,
clientVersion, addr, ugi, conf, clientVersion, addr, ugi, conf,
NetUtils.getDefaultSocketFactory(conf), expireIntvl); NetUtils.getDefaultSocketFactory(conf), expireIntvl);
} }

View File

@ -26,8 +26,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.ProtobufHelper; import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.yarn.api.RMAdminProtocol; import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocol;
import org.apache.hadoop.yarn.api.RMAdminProtocolPB; import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocolPB;
import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsRequest; import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsResponse; import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsResponse;
import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesRequest; 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; 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 { Configuration conf) throws IOException {
RPC.setProtocolEngine(conf, RMAdminProtocolPB.class, RPC.setProtocolEngine(conf, ResourceManagerAdministrationProtocolPB.class,
ProtobufRpcEngine.class); ProtobufRpcEngine.class);
proxy = (RMAdminProtocolPB)RPC.getProxy( proxy = (ResourceManagerAdministrationProtocolPB)RPC.getProxy(
RMAdminProtocolPB.class, clientVersion, addr, conf); ResourceManagerAdministrationProtocolPB.class, clientVersion, addr, conf);
} }
@Override @Override

View File

@ -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.GetDelegationTokenResponseProto;
import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto; import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenResponseProto; import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenResponseProto;
import org.apache.hadoop.yarn.api.ClientRMProtocol; import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
import org.apache.hadoop.yarn.api.ClientRMProtocolPB; import org.apache.hadoop.yarn.api.ApplicationClientProtocolPB;
import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; 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.RpcController;
import com.google.protobuf.ServiceException; 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; this.real = impl;
} }

View File

@ -20,8 +20,8 @@ package org.apache.hadoop.yarn.api.impl.pb.service;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.yarn.api.AMRMProtocol; import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
import org.apache.hadoop.yarn.api.AMRMProtocolPB; import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; 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.RpcController;
import com.google.protobuf.ServiceException; 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; this.real = impl;
} }

View File

@ -20,8 +20,8 @@ package org.apache.hadoop.yarn.api.impl.pb.service;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.yarn.api.ContainerManager; import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
import org.apache.hadoop.yarn.api.ContainerManagerPB; import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse; 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.RpcController;
import com.google.protobuf.ServiceException; 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; this.real = impl;
} }

View File

@ -20,8 +20,8 @@ package org.apache.hadoop.yarn.api.impl.pb.service;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.yarn.api.RMAdminProtocol; import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocol;
import org.apache.hadoop.yarn.api.RMAdminProtocolPB; import org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocolPB;
import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsResponse; import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsResponse;
import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesResponse; import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesResponse;
import org.apache.hadoop.yarn.api.protocolrecords.RefreshQueuesResponse; 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.RpcController;
import com.google.protobuf.ServiceException; 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; this.real = impl;
} }

View File

@ -26,7 +26,7 @@ import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.token.TokenInfo; import org.apache.hadoop.security.token.TokenInfo;
import org.apache.hadoop.security.token.TokenSelector; 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 { public class ContainerManagerSecurityInfo extends SecurityInfo {
@ -38,7 +38,7 @@ public class ContainerManagerSecurityInfo extends SecurityInfo {
@Override @Override
public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) { public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
if (!protocol if (!protocol
.equals(ContainerManagerPB.class)) { .equals(ContainerManagementProtocolPB.class)) {
return null; return null;
} }
return new TokenInfo() { return new TokenInfo() {

View File

@ -26,7 +26,7 @@ import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.token.TokenInfo; import org.apache.hadoop.security.token.TokenInfo;
import org.apache.hadoop.security.token.TokenSelector; 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 { public class SchedulerSecurityInfo extends SecurityInfo {
@ -37,7 +37,7 @@ public class SchedulerSecurityInfo extends SecurityInfo {
@Override @Override
public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) { public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
if (!protocol.equals(AMRMProtocolPB.class)) { if (!protocol.equals(ApplicationMasterProtocolPB.class)) {
return null; return null;
} }
return new TokenInfo() { return new TokenInfo() {

View File

@ -24,14 +24,14 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.KerberosInfo; import org.apache.hadoop.security.KerberosInfo;
import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.security.token.TokenInfo; 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; import org.apache.hadoop.yarn.conf.YarnConfiguration;
public class AdminSecurityInfo extends SecurityInfo { public class AdminSecurityInfo extends SecurityInfo {
@Override @Override
public KerberosInfo getKerberosInfo(Class<?> protocol, Configuration conf) { public KerberosInfo getKerberosInfo(Class<?> protocol, Configuration conf) {
if (!protocol.equals(RMAdminProtocolPB.class)) { if (!protocol.equals(ResourceManagerAdministrationProtocolPB.class)) {
return null; return null;
} }
return new KerberosInfo() { return new KerberosInfo() {

View File

@ -26,7 +26,7 @@ import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.token.TokenInfo; import org.apache.hadoop.security.token.TokenInfo;
import org.apache.hadoop.security.token.TokenSelector; 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; import org.apache.hadoop.yarn.conf.YarnConfiguration;
public class ClientRMSecurityInfo extends SecurityInfo { public class ClientRMSecurityInfo extends SecurityInfo {
@ -34,7 +34,7 @@ public class ClientRMSecurityInfo extends SecurityInfo {
@Override @Override
public KerberosInfo getKerberosInfo(Class<?> protocol, Configuration conf) { public KerberosInfo getKerberosInfo(Class<?> protocol, Configuration conf) {
if (!protocol if (!protocol
.equals(ClientRMProtocolPB.class)) { .equals(ApplicationClientProtocolPB.class)) {
return null; return null;
} }
return new KerberosInfo() { return new KerberosInfo() {
@ -59,7 +59,7 @@ public class ClientRMSecurityInfo extends SecurityInfo {
@Override @Override
public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) { public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
if (!protocol if (!protocol
.equals(ClientRMProtocolPB.class)) { .equals(ApplicationClientProtocolPB.class)) {
return null; return null;
} }
return new TokenInfo() { return new TokenInfo() {

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenRenewer; import org.apache.hadoop.security.token.TokenRenewer;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier; import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager; 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.CancelDelegationTokenRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest; import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
@ -97,7 +97,7 @@ public class RMDelegationTokenIdentifier extends AbstractDelegationTokenIdentifi
@Override @Override
public long renew(Token<?> token, Configuration conf) throws IOException, public long renew(Token<?> token, Configuration conf) throws IOException,
InterruptedException { InterruptedException {
final ClientRMProtocol rmClient = getRmClient(token, conf); final ApplicationClientProtocol rmClient = getRmClient(token, conf);
if (rmClient != null) { if (rmClient != null) {
try { try {
RenewDelegationTokenRequest request = RenewDelegationTokenRequest request =
@ -119,7 +119,7 @@ public class RMDelegationTokenIdentifier extends AbstractDelegationTokenIdentifi
@Override @Override
public void cancel(Token<?> token, Configuration conf) throws IOException, public void cancel(Token<?> token, Configuration conf) throws IOException,
InterruptedException { InterruptedException {
final ClientRMProtocol rmClient = getRmClient(token, conf); final ApplicationClientProtocol rmClient = getRmClient(token, conf);
if (rmClient != null) { if (rmClient != null) {
try { try {
CancelDelegationTokenRequest request = 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) { Configuration conf) {
InetSocketAddress addr = SecurityUtil.getTokenServiceAddr(token); InetSocketAddress addr = SecurityUtil.getTokenServiceAddr(token);
if (localSecretManager != null) { if (localSecretManager != null) {
@ -152,7 +152,7 @@ public class RMDelegationTokenIdentifier extends AbstractDelegationTokenIdentifi
} }
} }
final YarnRPC rpc = YarnRPC.create(conf); 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 // get renewer so we can always renew our own tokens

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.Server; import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.util.StringUtils; 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.GetContainerStatusRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@ -79,13 +79,13 @@ public class TestContainerLaunchRPC {
YarnRPC rpc = YarnRPC.create(conf); YarnRPC rpc = YarnRPC.create(conf);
String bindAddr = "localhost:0"; String bindAddr = "localhost:0";
InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
Server server = rpc.getServer(ContainerManager.class, Server server = rpc.getServer(ContainerManagementProtocol.class,
new DummyContainerManager(), addr, conf, null, 1); new DummyContainerManager(), addr, conf, null, 1);
server.start(); server.start();
try { try {
ContainerManager proxy = (ContainerManager) rpc.getProxy( ContainerManagementProtocol proxy = (ContainerManagementProtocol) rpc.getProxy(
ContainerManager.class, ContainerManagementProtocol.class,
server.getListenerAddress(), conf); server.getListenerAddress(), conf);
ContainerLaunchContext containerLaunchContext = recordFactory ContainerLaunchContext containerLaunchContext = recordFactory
.newRecordInstance(ContainerLaunchContext.class); .newRecordInstance(ContainerLaunchContext.class);
@ -124,7 +124,7 @@ public class TestContainerLaunchRPC {
Assert.fail("timeout exception should have occurred!"); Assert.fail("timeout exception should have occurred!");
} }
public class DummyContainerManager implements ContainerManager { public class DummyContainerManager implements ContainerManagementProtocol {
private ContainerStatus status = null; private ContainerStatus status = null;

View File

@ -30,9 +30,9 @@ import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server; import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.yarn.api.ClientRMProtocol; import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
import org.apache.hadoop.yarn.api.ContainerManager; import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
import org.apache.hadoop.yarn.api.ContainerManagerPB; import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
@ -74,13 +74,13 @@ public class TestRPC {
YarnRPC rpc = YarnRPC.create(conf); YarnRPC rpc = YarnRPC.create(conf);
String bindAddr = "localhost:0"; String bindAddr = "localhost:0";
InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
Server server = rpc.getServer(ContainerManager.class, Server server = rpc.getServer(ContainerManagementProtocol.class,
new DummyContainerManager(), addr, conf, null, 1); new DummyContainerManager(), addr, conf, null, 1);
server.start(); server.start();
// Any unrelated protocol would do // Any unrelated protocol would do
ClientRMProtocol proxy = (ClientRMProtocol) rpc.getProxy( ApplicationClientProtocol proxy = (ApplicationClientProtocol) rpc.getProxy(
ClientRMProtocol.class, NetUtils.getConnectAddress(server), conf); ApplicationClientProtocol.class, NetUtils.getConnectAddress(server), conf);
try { try {
proxy.getNewApplication(Records proxy.getNewApplication(Records
@ -89,8 +89,8 @@ public class TestRPC {
} catch (YarnException e) { } catch (YarnException e) {
Assert.assertTrue(e.getMessage().matches( Assert.assertTrue(e.getMessage().matches(
"Unknown method getNewApplication called on.*" "Unknown method getNewApplication called on.*"
+ "org.apache.hadoop.yarn.proto.ClientRMProtocol" + "org.apache.hadoop.yarn.proto.ApplicationClientProtocol"
+ "\\$ClientRMProtocolService\\$BlockingInterface protocol.")); + "\\$ApplicationClientProtocolService\\$BlockingInterface protocol."));
} catch (Exception e) { } catch (Exception e) {
e.printStackTrace(); e.printStackTrace();
} }
@ -107,12 +107,12 @@ public class TestRPC {
YarnRPC rpc = YarnRPC.create(conf); YarnRPC rpc = YarnRPC.create(conf);
String bindAddr = "localhost:0"; String bindAddr = "localhost:0";
InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
Server server = rpc.getServer(ContainerManager.class, Server server = rpc.getServer(ContainerManagementProtocol.class,
new DummyContainerManager(), addr, conf, null, 1); new DummyContainerManager(), addr, conf, null, 1);
server.start(); server.start();
RPC.setProtocolEngine(conf, ContainerManagerPB.class, ProtobufRpcEngine.class); RPC.setProtocolEngine(conf, ContainerManagementProtocolPB.class, ProtobufRpcEngine.class);
ContainerManager proxy = (ContainerManager) ContainerManagementProtocol proxy = (ContainerManagementProtocol)
rpc.getProxy(ContainerManager.class, rpc.getProxy(ContainerManagementProtocol.class,
NetUtils.getConnectAddress(server), conf); NetUtils.getConnectAddress(server), conf);
ContainerLaunchContext containerLaunchContext = ContainerLaunchContext containerLaunchContext =
recordFactory.newRecordInstance(ContainerLaunchContext.class); recordFactory.newRecordInstance(ContainerLaunchContext.class);
@ -162,7 +162,7 @@ public class TestRPC {
Assert.assertEquals(ContainerState.RUNNING, status.getState()); Assert.assertEquals(ContainerState.RUNNING, status.getState());
} }
public class DummyContainerManager implements ContainerManager { public class DummyContainerManager implements ContainerManagementProtocol {
private ContainerStatus status = null; private ContainerStatus status = null;

View File

@ -26,7 +26,7 @@ import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.Server; import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NetUtils; 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.AllocateRequest;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
@ -55,12 +55,12 @@ public class TestRPCFactories {
private void testPbServerFactory() { private void testPbServerFactory() {
InetSocketAddress addr = new InetSocketAddress(0); InetSocketAddress addr = new InetSocketAddress(0);
Configuration conf = new Configuration(); Configuration conf = new Configuration();
AMRMProtocol instance = new AMRMProtocolTestImpl(); ApplicationMasterProtocol instance = new AMRMProtocolTestImpl();
Server server = null; Server server = null;
try { try {
server = server =
RpcServerFactoryPBImpl.get().getServer( RpcServerFactoryPBImpl.get().getServer(
AMRMProtocol.class, instance, addr, conf, null, 1); ApplicationMasterProtocol.class, instance, addr, conf, null, 1);
server.start(); server.start();
} catch (YarnRuntimeException e) { } catch (YarnRuntimeException e) {
e.printStackTrace(); e.printStackTrace();
@ -77,19 +77,19 @@ public class TestRPCFactories {
InetSocketAddress addr = new InetSocketAddress(0); InetSocketAddress addr = new InetSocketAddress(0);
System.err.println(addr.getHostName() + addr.getPort()); System.err.println(addr.getHostName() + addr.getPort());
Configuration conf = new Configuration(); Configuration conf = new Configuration();
AMRMProtocol instance = new AMRMProtocolTestImpl(); ApplicationMasterProtocol instance = new AMRMProtocolTestImpl();
Server server = null; Server server = null;
try { try {
server = server =
RpcServerFactoryPBImpl.get().getServer( RpcServerFactoryPBImpl.get().getServer(
AMRMProtocol.class, instance, addr, conf, null, 1); ApplicationMasterProtocol.class, instance, addr, conf, null, 1);
server.start(); server.start();
System.err.println(server.getListenerAddress()); System.err.println(server.getListenerAddress());
System.err.println(NetUtils.getConnectAddress(server)); System.err.println(NetUtils.getConnectAddress(server));
AMRMProtocol amrmClient = null; ApplicationMasterProtocol amrmClient = null;
try { 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) { } catch (YarnRuntimeException e) {
e.printStackTrace(); e.printStackTrace();
Assert.fail("Failed to create client"); Assert.fail("Failed to create client");
@ -105,7 +105,7 @@ public class TestRPCFactories {
} }
} }
public class AMRMProtocolTestImpl implements AMRMProtocol { public class AMRMProtocolTestImpl implements ApplicationMasterProtocol {
@Override @Override
public RegisterApplicationMasterResponse registerApplicationMaster( public RegisterApplicationMasterResponse registerApplicationMaster(

Some files were not shown because too many files have changed in this diff Show More