YARN-926. Modified ContainerManagerProtcol APIs to take in requests for multiple containers. Contributed by Jian He.
MAPREDUCE-5412. Update MR app to use multiple containers API of ContainerManager after YARN-926. Contributed by Jian He. svn merge --ignore-ancestry -c 1506391 ../../trunk/ git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1506392 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
366eda548b
commit
f487f4eb19
|
@ -450,6 +450,9 @@ Release 2.1.0-beta - 2013-07-02
|
|||
MAPREDUCE-5325. MR changes related to YARN-727. ClientRMProtocol.getAllApplications
|
||||
should accept ApplicationType as a parameter. (Xuan Gong via hitesh)
|
||||
|
||||
MAPREDUCE-5412. Update MR app to use multiple containers API of
|
||||
ContainerManager after YARN-926. (Jian He via vinodkv)
|
||||
|
||||
BREAKDOWN OF HADOOP-8562 SUBTASKS
|
||||
|
||||
MAPREDUCE-4739. Some MapReduce tests fail to find winutils.
|
||||
|
|
|
@ -20,7 +20,9 @@ package org.apache.hadoop.mapreduce.v2.app.launcher;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
@ -44,14 +46,15 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
|
|||
import org.apache.hadoop.service.AbstractService;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy;
|
||||
import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
|
@ -139,13 +142,18 @@ public class ContainerLauncherImpl extends AbstractService implements
|
|||
event.getContainerLaunchContext();
|
||||
|
||||
// Now launch the actual container
|
||||
StartContainerRequest startRequest = Records
|
||||
.newRecord(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
startRequest.setContainerToken(event.getContainerToken());
|
||||
StartContainerResponse response =
|
||||
proxy.getContainerManagementProtocol().startContainer(startRequest);
|
||||
|
||||
StartContainerRequest startRequest =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
event.getContainerToken());
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(startRequest);
|
||||
StartContainersRequest requestList = StartContainersRequest.newInstance(list);
|
||||
StartContainersResponse response =
|
||||
proxy.getContainerManagementProtocol().startContainers(requestList);
|
||||
if (response.getFailedRequests() != null
|
||||
&& response.getFailedRequests().containsKey(containerID)) {
|
||||
throw response.getFailedRequests().get(containerID).deSerialize();
|
||||
}
|
||||
ByteBuffer portInfo =
|
||||
response.getAllServicesMetaData().get(
|
||||
ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID);
|
||||
|
@ -192,13 +200,17 @@ public class ContainerLauncherImpl extends AbstractService implements
|
|||
proxy = getCMProxy(this.containerMgrAddress, this.containerID);
|
||||
|
||||
// kill the remote container if already launched
|
||||
StopContainerRequest stopRequest = Records
|
||||
.newRecord(StopContainerRequest.class);
|
||||
stopRequest.setContainerId(this.containerID);
|
||||
proxy.getContainerManagementProtocol().stopContainer(stopRequest);
|
||||
|
||||
List<ContainerId> ids = new ArrayList<ContainerId>();
|
||||
ids.add(this.containerID);
|
||||
StopContainersRequest request = StopContainersRequest.newInstance(ids);
|
||||
StopContainersResponse response =
|
||||
proxy.getContainerManagementProtocol().stopContainers(request);
|
||||
if (response.getFailedRequests() != null
|
||||
&& response.getFailedRequests().containsKey(this.containerID)) {
|
||||
throw response.getFailedRequests().get(this.containerID)
|
||||
.deSerialize();
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
|
||||
// ignore the cleanup failure
|
||||
String message = "cleanup failed for container "
|
||||
+ this.containerID + " : "
|
||||
|
|
|
@ -24,6 +24,8 @@ import java.io.IOException;
|
|||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
@ -52,12 +54,13 @@ import org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl;
|
|||
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -393,18 +396,18 @@ public class TestContainerLauncher {
|
|||
private ContainerStatus status = null;
|
||||
|
||||
@Override
|
||||
public GetContainerStatusResponse getContainerStatus(
|
||||
GetContainerStatusRequest request) throws IOException {
|
||||
GetContainerStatusResponse response = recordFactory
|
||||
.newRecordInstance(GetContainerStatusResponse.class);
|
||||
response.setStatus(status);
|
||||
return response;
|
||||
public GetContainerStatusesResponse getContainerStatuses(
|
||||
GetContainerStatusesRequest request) throws IOException {
|
||||
List<ContainerStatus> statuses = new ArrayList<ContainerStatus>();
|
||||
statuses.add(status);
|
||||
return GetContainerStatusesResponse.newInstance(statuses, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StartContainerResponse startContainer(StartContainerRequest request)
|
||||
public StartContainersResponse startContainers(StartContainersRequest requests)
|
||||
throws IOException {
|
||||
|
||||
StartContainerRequest request = requests.getStartContainerRequests().get(0);
|
||||
ContainerTokenIdentifier containerTokenIdentifier =
|
||||
MRApp.newContainerTokenIdentifier(request.getContainerToken());
|
||||
|
||||
|
@ -412,8 +415,8 @@ public class TestContainerLauncher {
|
|||
Assert.assertEquals(MRApp.NM_HOST + ":" + MRApp.NM_PORT,
|
||||
containerTokenIdentifier.getNmHostAddress());
|
||||
|
||||
StartContainerResponse response = recordFactory
|
||||
.newRecordInstance(StartContainerResponse.class);
|
||||
StartContainersResponse response = recordFactory
|
||||
.newRecordInstance(StartContainersResponse.class);
|
||||
status = recordFactory.newRecordInstance(ContainerStatus.class);
|
||||
try {
|
||||
// make the thread sleep to look like its not going to respond
|
||||
|
@ -429,7 +432,7 @@ public class TestContainerLauncher {
|
|||
}
|
||||
|
||||
@Override
|
||||
public StopContainerResponse stopContainer(StopContainerRequest request)
|
||||
public StopContainersResponse stopContainers(StopContainersRequest request)
|
||||
throws IOException {
|
||||
Exception e = new Exception("Dummy function", new Exception(
|
||||
"Dummy function cause"));
|
||||
|
|
|
@ -45,12 +45,12 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
|
|||
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher.EventType;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -162,8 +162,8 @@ public class TestContainerLauncherImpl {
|
|||
try {
|
||||
ContainerId contId = makeContainerId(0l, 0, 0, 1);
|
||||
TaskAttemptId taskAttemptId = makeTaskAttemptId(0l, 0, 0, TaskType.MAP, 0);
|
||||
StartContainerResponse startResp =
|
||||
recordFactory.newRecordInstance(StartContainerResponse.class);
|
||||
StartContainersResponse startResp =
|
||||
recordFactory.newRecordInstance(StartContainersResponse.class);
|
||||
startResp.setAllServicesMetaData(serviceResponse);
|
||||
|
||||
|
||||
|
@ -176,14 +176,14 @@ public class TestContainerLauncherImpl {
|
|||
.thenReturn(contId);
|
||||
when(mockLaunchEvent.getTaskAttemptID()).thenReturn(taskAttemptId);
|
||||
when(mockLaunchEvent.getContainerMgrAddress()).thenReturn(cmAddress);
|
||||
when(mockCM.startContainer(any(StartContainerRequest.class))).thenReturn(startResp);
|
||||
when(mockCM.startContainers(any(StartContainersRequest.class))).thenReturn(startResp);
|
||||
when(mockLaunchEvent.getContainerToken()).thenReturn(
|
||||
createNewContainerToken(contId, cmAddress));
|
||||
ut.handle(mockLaunchEvent);
|
||||
|
||||
ut.waitForPoolToIdle();
|
||||
|
||||
verify(mockCM).startContainer(any(StartContainerRequest.class));
|
||||
verify(mockCM).startContainers(any(StartContainersRequest.class));
|
||||
|
||||
LOG.info("inserting cleanup event");
|
||||
ContainerLauncherEvent mockCleanupEvent =
|
||||
|
@ -198,7 +198,7 @@ public class TestContainerLauncherImpl {
|
|||
|
||||
ut.waitForPoolToIdle();
|
||||
|
||||
verify(mockCM).stopContainer(any(StopContainerRequest.class));
|
||||
verify(mockCM).stopContainers(any(StopContainersRequest.class));
|
||||
} finally {
|
||||
ut.stop();
|
||||
}
|
||||
|
@ -224,8 +224,8 @@ public class TestContainerLauncherImpl {
|
|||
ContainerId contId = makeContainerId(0l, 0, 0, 1);
|
||||
TaskAttemptId taskAttemptId = makeTaskAttemptId(0l, 0, 0, TaskType.MAP, 0);
|
||||
String cmAddress = "127.0.0.1:8000";
|
||||
StartContainerResponse startResp =
|
||||
recordFactory.newRecordInstance(StartContainerResponse.class);
|
||||
StartContainersResponse startResp =
|
||||
recordFactory.newRecordInstance(StartContainersResponse.class);
|
||||
startResp.setAllServicesMetaData(serviceResponse);
|
||||
|
||||
LOG.info("inserting cleanup event");
|
||||
|
@ -241,7 +241,7 @@ public class TestContainerLauncherImpl {
|
|||
|
||||
ut.waitForPoolToIdle();
|
||||
|
||||
verify(mockCM, never()).stopContainer(any(StopContainerRequest.class));
|
||||
verify(mockCM, never()).stopContainers(any(StopContainersRequest.class));
|
||||
|
||||
LOG.info("inserting launch event");
|
||||
ContainerRemoteLaunchEvent mockLaunchEvent =
|
||||
|
@ -252,14 +252,14 @@ public class TestContainerLauncherImpl {
|
|||
.thenReturn(contId);
|
||||
when(mockLaunchEvent.getTaskAttemptID()).thenReturn(taskAttemptId);
|
||||
when(mockLaunchEvent.getContainerMgrAddress()).thenReturn(cmAddress);
|
||||
when(mockCM.startContainer(any(StartContainerRequest.class))).thenReturn(startResp);
|
||||
when(mockCM.startContainers(any(StartContainersRequest.class))).thenReturn(startResp);
|
||||
when(mockLaunchEvent.getContainerToken()).thenReturn(
|
||||
createNewContainerToken(contId, cmAddress));
|
||||
ut.handle(mockLaunchEvent);
|
||||
|
||||
ut.waitForPoolToIdle();
|
||||
|
||||
verify(mockCM, never()).startContainer(any(StartContainerRequest.class));
|
||||
verify(mockCM, never()).startContainers(any(StartContainersRequest.class));
|
||||
} finally {
|
||||
ut.stop();
|
||||
}
|
||||
|
@ -286,8 +286,8 @@ public class TestContainerLauncherImpl {
|
|||
ContainerId contId = makeContainerId(0l, 0, 0, 1);
|
||||
TaskAttemptId taskAttemptId = makeTaskAttemptId(0l, 0, 0, TaskType.MAP, 0);
|
||||
String cmAddress = "127.0.0.1:8000";
|
||||
StartContainerResponse startResp =
|
||||
recordFactory.newRecordInstance(StartContainerResponse.class);
|
||||
StartContainersResponse startResp =
|
||||
recordFactory.newRecordInstance(StartContainersResponse.class);
|
||||
startResp.setAllServicesMetaData(serviceResponse);
|
||||
|
||||
LOG.info("inserting launch event");
|
||||
|
@ -299,20 +299,20 @@ public class TestContainerLauncherImpl {
|
|||
.thenReturn(contId);
|
||||
when(mockLaunchEvent.getTaskAttemptID()).thenReturn(taskAttemptId);
|
||||
when(mockLaunchEvent.getContainerMgrAddress()).thenReturn(cmAddress);
|
||||
when(mockCM.startContainer(any(StartContainerRequest.class))).thenReturn(startResp);
|
||||
when(mockCM.startContainers(any(StartContainersRequest.class))).thenReturn(startResp);
|
||||
when(mockLaunchEvent.getContainerToken()).thenReturn(
|
||||
createNewContainerToken(contId, cmAddress));
|
||||
ut.handle(mockLaunchEvent);
|
||||
|
||||
ut.waitForPoolToIdle();
|
||||
|
||||
verify(mockCM).startContainer(any(StartContainerRequest.class));
|
||||
verify(mockCM).startContainers(any(StartContainersRequest.class));
|
||||
|
||||
// skip cleanup and make sure stop kills the container
|
||||
|
||||
} finally {
|
||||
ut.stop();
|
||||
verify(mockCM).stopContainer(any(StopContainerRequest.class));
|
||||
verify(mockCM).stopContainers(any(StopContainersRequest.class));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -341,8 +341,8 @@ public class TestContainerLauncherImpl {
|
|||
ContainerId contId = makeContainerId(0l, 0, 0, 1);
|
||||
TaskAttemptId taskAttemptId = makeTaskAttemptId(0l, 0, 0, TaskType.MAP, 0);
|
||||
String cmAddress = "127.0.0.1:8000";
|
||||
StartContainerResponse startResp =
|
||||
recordFactory.newRecordInstance(StartContainerResponse.class);
|
||||
StartContainersResponse startResp =
|
||||
recordFactory.newRecordInstance(StartContainersResponse.class);
|
||||
startResp.setAllServicesMetaData(serviceResponse);
|
||||
|
||||
|
||||
|
@ -415,7 +415,7 @@ public class TestContainerLauncherImpl {
|
|||
this.completeLaunchBarrier = completeLaunchBarrier;
|
||||
}
|
||||
@Override
|
||||
public StartContainerResponse startContainer(StartContainerRequest request)
|
||||
public StartContainersResponse startContainers(StartContainersRequest request)
|
||||
throws IOException {
|
||||
try {
|
||||
startLaunchBarrier.await();
|
||||
|
@ -433,16 +433,14 @@ public class TestContainerLauncherImpl {
|
|||
}
|
||||
|
||||
@Override
|
||||
public StopContainerResponse stopContainer(StopContainerRequest request)
|
||||
public StopContainersResponse stopContainers(StopContainersRequest request)
|
||||
throws IOException {
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetContainerStatusResponse getContainerStatus(
|
||||
GetContainerStatusRequest request) throws IOException {
|
||||
|
||||
public GetContainerStatusesResponse getContainerStatuses(
|
||||
GetContainerStatusesRequest request) throws IOException {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -206,6 +206,9 @@ Release 2.1.0-beta - 2013-07-02
|
|||
YARN-918. Remove ApplicationAttemptId from
|
||||
RegisterApplicationMasterRequestProto. (vinodkv via acmurthy)
|
||||
|
||||
YARN-926. Modified ContainerManagerProtcol APIs to take in requests for
|
||||
multiple containers. (Jian He via vinodkv)
|
||||
|
||||
NEW FEATURES
|
||||
|
||||
YARN-482. FS: Extend SchedulingMode to intermediate queues.
|
||||
|
|
|
@ -131,6 +131,11 @@
|
|||
<Class name="org.apache.hadoop.yarn.server.resourcemanager.resource.Priority$Comparator" />
|
||||
<Bug pattern="SE_COMPARATOR_SHOULD_BE_SERIALIZABLE" />
|
||||
</Match>
|
||||
<!-- Ignore some irrelevant class name warning -->
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.yarn.api.records.SerializedException" />
|
||||
<Bug pattern="NM_CLASS_NOT_EXCEPTION" />
|
||||
</Match>
|
||||
|
||||
<!-- Inconsistent sync warning - only start() is synchronized-->
|
||||
<Match>
|
||||
|
|
|
@ -22,17 +22,17 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.Container;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||
import org.apache.hadoop.yarn.exceptions.InvalidContainerException;
|
||||
import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
|
||||
|
@ -50,10 +50,12 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
|
|||
@Public
|
||||
@Stable
|
||||
public interface ContainerManagementProtocol {
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* The <code>ApplicationMaster</code> requests a <code>NodeManager</code> to
|
||||
* <em>start</em> a {@link Container} allocated to it using this interface.
|
||||
* The <code>ApplicationMaster</code> provides a list of
|
||||
* {@link StartContainerRequest}s to a <code>NodeManager</code> to
|
||||
* <em>start</em> {@link Container}s allocated to it using this interface.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
|
@ -65,82 +67,107 @@ public interface ContainerManagementProtocol {
|
|||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Currently the <code>NodeManager</code> sends an immediate, empty response
|
||||
* via {@link StartContainerResponse} to signify acceptance of the request and
|
||||
* throws an exception in case of errors. The <code>ApplicationMaster</code>
|
||||
* can use {@link #getContainerStatus(GetContainerStatusRequest)} to get
|
||||
* updated status of the to-be-launched or launched container.
|
||||
* The <code>NodeManager</code> sends a response via
|
||||
* {@link StartContainersResponse} which includes a list of
|
||||
* {@link Container}s of successfully launched {@link Container}s, a
|
||||
* containerId-to-exception map for each failed {@link StartContainerRequest} in
|
||||
* which the exception indicates errors from per container and a
|
||||
* allServicesMetaData map between the names of auxiliary services and their
|
||||
* corresponding meta-data. Note: None-container-specific exceptions will
|
||||
* still be thrown by the API method itself.
|
||||
* </p>
|
||||
* <p>
|
||||
* The <code>ApplicationMaster</code> can use
|
||||
* {@link #getContainerStatuses(GetContainerStatusesRequest)} to get updated
|
||||
* statuses of the to-be-launched or launched containers.
|
||||
* </p>
|
||||
*
|
||||
* @param request
|
||||
* request to start a container
|
||||
* @return empty response to indicate acceptance of the request or an
|
||||
* exception
|
||||
* request to start a list of containers
|
||||
* @return response including conatinerIds of all successfully launched
|
||||
* containers, a containerId-to-exception map for failed requests and
|
||||
* a allServicesMetaData map.
|
||||
* @throws YarnException
|
||||
* @throws IOException
|
||||
* @throws NMNotYetReadyException
|
||||
* This exception is thrown when NM starts from scratch but has not
|
||||
* yet connected with RM.
|
||||
* @throws InvalidContainerException
|
||||
* This exception is thrown when NM is rejecting start-container
|
||||
* requests for containers allocated by a previous instance of the
|
||||
* RM
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
StartContainerResponse startContainer(StartContainerRequest request)
|
||||
StartContainersResponse startContainers(StartContainersRequest request)
|
||||
throws YarnException, IOException;
|
||||
|
||||
/**
|
||||
* <p>The <code>ApplicationMaster</code> requests a <code>NodeManager</code>
|
||||
* to <em>stop</em> a {@link Container} allocated to it using this interface.
|
||||
* <p>
|
||||
* The <code>ApplicationMaster</code> requests a <code>NodeManager</code> to
|
||||
* <em>stop</em> a list of {@link Container}s allocated to it using this
|
||||
* interface.
|
||||
* </p>
|
||||
*
|
||||
* <p>The <code>ApplicationMaster</code> sends a
|
||||
* {@link StopContainerRequest} which includes the {@link ContainerId} of the
|
||||
* container to be stopped.</p>
|
||||
* <p>
|
||||
* The <code>ApplicationMaster</code> sends a {@link StopContainersRequest}
|
||||
* which includes the {@link ContainerId}s of the containers to be stopped.
|
||||
* </p>
|
||||
*
|
||||
* <p>Currently the <code>NodeManager</code> sends an immediate, empty
|
||||
* response via {@link StopContainerResponse} to signify acceptance of the
|
||||
* request and throws an exception in case of errors. The
|
||||
* <code>ApplicationMaster</code> can use
|
||||
* {@link #getContainerStatus(GetContainerStatusRequest)} to get updated
|
||||
* status of the container.</p>
|
||||
* <p>
|
||||
* The <code>NodeManager</code> sends a response via
|
||||
* {@link StopContainersResponse} which includes a list of {@link ContainerId}
|
||||
* s of successfully stopped containers, a containerId-to-exception map for
|
||||
* each failed request in which the exception indicates errors from per
|
||||
* container. Note: None-container-specific exceptions will still be thrown by
|
||||
* the API method itself. <code>ApplicationMaster</code> can use
|
||||
* {@link #getContainerStatuses(GetContainerStatusesRequest)} to get updated
|
||||
* statuses of the containers.
|
||||
* </p>
|
||||
*
|
||||
* @param request request to stop a container
|
||||
* @return empty response to indicate acceptance of the request
|
||||
* or an exception
|
||||
* @param request
|
||||
* request to stop a list of containers
|
||||
* @return response which includes a list of containerIds of successfully
|
||||
* stopped containers, a containerId-to-exception map for failed
|
||||
* requests.
|
||||
* @throws YarnException
|
||||
* @throws IOException
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
StopContainerResponse stopContainer(StopContainerRequest request)
|
||||
StopContainersResponse stopContainers(StopContainersRequest request)
|
||||
throws YarnException, IOException;
|
||||
|
||||
/**
|
||||
* <p>The api used by the <code>ApplicationMaster</code> to request for
|
||||
* current status of a <code>Container</code> from the
|
||||
* <code>NodeManager</code>.</p>
|
||||
* <p>
|
||||
* The API used by the <code>ApplicationMaster</code> to request for current
|
||||
* statuses of <code>Container</code>s from the <code>NodeManager</code>.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* The <code>ApplicationMaster</code> sends a
|
||||
* {@link GetContainerStatusesRequest} which includes the {@link ContainerId}s
|
||||
* of all containers whose statuses are needed.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* The <code>NodeManager</code> responds with
|
||||
* {@link GetContainerStatusesResponse} which includes a list of
|
||||
* {@link ContainerStatus} of the successfully queried containers and a
|
||||
* containerId-to-exception map for each failed request in which the exception
|
||||
* indicates errors from per container. Note: None-container-specific
|
||||
* exceptions will still be thrown by the API method itself.
|
||||
* </p>
|
||||
*
|
||||
* @param request
|
||||
* request to get <code>ContainerStatus</code>es of containers with
|
||||
* the specified <code>ContainerId</code>s
|
||||
* @return response containing the list of <code>ContainerStatus</code> of the
|
||||
* successfully queried containers and a containerId-to-exception map
|
||||
* for failed requests.
|
||||
*
|
||||
* <p>The <code>ApplicationMaster</code> sends a
|
||||
* {@link GetContainerStatusRequest} which includes the {@link ContainerId} of
|
||||
* the container whose status is needed.</p>
|
||||
*
|
||||
*<p>The <code>NodeManager</code> responds with
|
||||
*{@link GetContainerStatusResponse} which includes the
|
||||
*{@link ContainerStatus} of the container.</p>
|
||||
*
|
||||
* @param request request to get <code>ContainerStatus</code> of a container
|
||||
* with the specified <code>ContainerId</code>
|
||||
* @return response containing the <code>ContainerStatus</code> of the
|
||||
* container
|
||||
* @throws YarnException
|
||||
* @throws IOException
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
GetContainerStatusResponse getContainerStatus(
|
||||
GetContainerStatusRequest request) throws YarnException,
|
||||
GetContainerStatusesResponse getContainerStatuses(
|
||||
GetContainerStatusesRequest request) throws YarnException,
|
||||
IOException;
|
||||
}
|
||||
|
|
|
@ -1,61 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
/**
|
||||
* <p>The response sent by the <code>NodeManager</code> to the
|
||||
* <code>ApplicationMaster</code> when asked to obtain the <em>status</em>
|
||||
* of a container.</p>
|
||||
*
|
||||
* @see ContainerManagementProtocol#getContainerStatus(GetContainerStatusRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract class GetContainerStatusResponse {
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public static GetContainerStatusResponse newInstance(
|
||||
ContainerStatus containerStatus) {
|
||||
GetContainerStatusResponse response =
|
||||
Records.newRecord(GetContainerStatusResponse.class);
|
||||
response.setStatus(containerStatus);
|
||||
return response;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the <code>ContainerStatus</code> of the container.
|
||||
* @return <code>ContainerStatus</code> of the container
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract ContainerStatus getStatus();
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract void setStatus(ContainerStatus containerStatus);
|
||||
}
|
|
@ -18,6 +18,8 @@
|
|||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
|
@ -26,41 +28,48 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
/**
|
||||
* <p>The request sent by the <code>ApplicationMaster</code> to the
|
||||
* <code>NodeManager</code> to get {@link ContainerStatus} of a container.</p>
|
||||
* <p>
|
||||
* The request sent by the <code>ApplicationMaster</code> to the
|
||||
* <code>NodeManager</code> to get {@link ContainerStatus} of requested
|
||||
* containers.
|
||||
* </p>
|
||||
*
|
||||
* @see ContainerManagementProtocol#getContainerStatus(GetContainerStatusRequest)
|
||||
* @see ContainerManagementProtocol#getContainerStatuses(GetContainerStatusesRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract class GetContainerStatusRequest {
|
||||
public abstract class GetContainerStatusesRequest {
|
||||
|
||||
@Public
|
||||
@Stable
|
||||
public static GetContainerStatusRequest newInstance(ContainerId containerId) {
|
||||
GetContainerStatusRequest request =
|
||||
Records.newRecord(GetContainerStatusRequest.class);
|
||||
request.setContainerId(containerId);
|
||||
public static GetContainerStatusesRequest newInstance(
|
||||
List<ContainerId> containerIds) {
|
||||
GetContainerStatusesRequest request =
|
||||
Records.newRecord(GetContainerStatusesRequest.class);
|
||||
request.setContainerIds(containerIds);
|
||||
return request;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the <code>ContainerId</code> of container for which to obtain the
|
||||
* <code>ContainerStatus</code>.
|
||||
* @return <code>ContainerId</code> of container for which to obtain the
|
||||
* <code>ContainerStatus</code>
|
||||
* Get the list of <code>ContainerId</code>s of containers for which to obtain
|
||||
* the <code>ContainerStatus</code>.
|
||||
*
|
||||
* @return the list of <code>ContainerId</code>s of containers for which to
|
||||
* obtain the <code>ContainerStatus</code>.
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract ContainerId getContainerId();
|
||||
|
||||
public abstract List<ContainerId> getContainerIds();
|
||||
|
||||
/**
|
||||
* Set the <code>ContainerId</code> of container for which to obtain the
|
||||
* <code>ContainerStatus</code>
|
||||
* @param containerId <code>ContainerId</code> of container for which to
|
||||
* obtain the <code>ContainerStatus</code>
|
||||
* Set a list of <code>ContainerId</code>s of containers for which to obtain
|
||||
* the <code>ContainerStatus</code>
|
||||
*
|
||||
* @param containerIds
|
||||
* a list of <code>ContainerId</code>s of containers for which to
|
||||
* obtain the <code>ContainerStatus</code>
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract void setContainerId(ContainerId containerId);
|
||||
public abstract void setContainerIds(List<ContainerId> containerIds);
|
||||
}
|
|
@ -0,0 +1,91 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* The response sent by the <code>NodeManager</code> to the
|
||||
* <code>ApplicationMaster</code> when asked to obtain the
|
||||
* <code>ContainerStatus</code> of requested containers.
|
||||
* </p>
|
||||
*
|
||||
* @see ContainerManagementProtocol#getContainerStatuses(GetContainerStatusesRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract class GetContainerStatusesResponse {
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public static GetContainerStatusesResponse newInstance(
|
||||
List<ContainerStatus> statuses,
|
||||
Map<ContainerId, SerializedException> failedRequests) {
|
||||
GetContainerStatusesResponse response =
|
||||
Records.newRecord(GetContainerStatusesResponse.class);
|
||||
response.setContainerStatuses(statuses);
|
||||
response.setFailedRequests(failedRequests);
|
||||
return response;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the <code>ContainerStatus</code>es of the requested containers.
|
||||
*
|
||||
* @return <code>ContainerStatus</code>es of the requested containers.
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract List<ContainerStatus> getContainerStatuses();
|
||||
|
||||
/**
|
||||
* Set the <code>ContainerStatus</code>es of the requested containers.
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract void setContainerStatuses(List<ContainerStatus> statuses);
|
||||
|
||||
/**
|
||||
* Get the containerId-to-exception map in which the exception indicates error
|
||||
* from per container for failed requests
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract Map<ContainerId, SerializedException> getFailedRequests();
|
||||
|
||||
/**
|
||||
* Set the containerId-to-exception map in which the exception indicates error
|
||||
* from per container for failed requests
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract void setFailedRequests(
|
||||
Map<ContainerId, SerializedException> failedContainers);
|
||||
}
|
|
@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.util.Records;
|
|||
* necessary binaries/jar/shared-objects etc. via the
|
||||
* {@link ContainerLaunchContext}.</p>
|
||||
*
|
||||
* @see ContainerManagementProtocol#startContainer(StartContainerRequest)
|
||||
* @see ContainerManagementProtocol#startContainers(StartContainersRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
|
@ -81,7 +81,7 @@ public abstract class StartContainerRequest {
|
|||
* @return the container token to be used for authorization during starting
|
||||
* container.
|
||||
* @see NMToken
|
||||
* @see ContainerManagementProtocol#startContainer(StartContainerRequest)
|
||||
* @see ContainerManagementProtocol#startContainers(StartContainersRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
|
|
|
@ -0,0 +1,76 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* The request which contains a list of {@link StartContainerRequest} sent by
|
||||
* the <code>ApplicationMaster</code> to the <code>NodeManager</code> to
|
||||
* <em>start</em> containers.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* In each {@link StartContainerRequest}, the <code>ApplicationMaster</code> has
|
||||
* to provide details such as allocated resource capability, security tokens (if
|
||||
* enabled), command to be executed to start the container, environment for the
|
||||
* process, necessary binaries/jar/shared-objects etc. via the
|
||||
* {@link ContainerLaunchContext}.
|
||||
* </p>
|
||||
*
|
||||
* @see ContainerManagementProtocol#startContainers(StartContainersRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract class StartContainersRequest {
|
||||
|
||||
@Public
|
||||
@Stable
|
||||
public static StartContainersRequest newInstance(
|
||||
List<StartContainerRequest> requests) {
|
||||
StartContainersRequest request =
|
||||
Records.newRecord(StartContainersRequest.class);
|
||||
request.setStartContainerRequests(requests);
|
||||
return request;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a list of {@link StartContainerRequest} to start containers.
|
||||
* @return a list of {@link StartContainerRequest} to start containers.
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract List<StartContainerRequest> getStartContainerRequests();
|
||||
|
||||
/**
|
||||
* Set a list of {@link StartContainerRequest} to start containers.
|
||||
* @param request a list of {@link StartContainerRequest} to start containers
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract void setStartContainerRequests(
|
||||
List<StartContainerRequest> request);
|
||||
}
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.yarn.api.protocolrecords;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
|
@ -26,29 +27,71 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
|
|||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
/**
|
||||
* <p>The response sent by the <code>NodeManager</code> to the
|
||||
* <code>ApplicationMaster</code> when asked to <em>start</em> an
|
||||
* allocated container.</p>
|
||||
* <p>
|
||||
* The response sent by the <code>NodeManager</code> to the
|
||||
* <code>ApplicationMaster</code> when asked to <em>start</em> an allocated
|
||||
* container.
|
||||
* </p>
|
||||
*
|
||||
* @see ContainerManagementProtocol#startContainer(StartContainerRequest)
|
||||
* @see ContainerManagementProtocol#startContainers(StartContainersRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract class StartContainerResponse {
|
||||
public abstract class StartContainersResponse {
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public static StartContainerResponse newInstance(
|
||||
Map<String, ByteBuffer> servicesMetaData) {
|
||||
StartContainerResponse response =
|
||||
Records.newRecord(StartContainerResponse.class);
|
||||
public static StartContainersResponse newInstance(
|
||||
Map<String, ByteBuffer> servicesMetaData,
|
||||
List<ContainerId> succeededContainers,
|
||||
Map<ContainerId, SerializedException> failedContainers) {
|
||||
StartContainersResponse response =
|
||||
Records.newRecord(StartContainersResponse.class);
|
||||
response.setAllServicesMetaData(servicesMetaData);
|
||||
response.setSuccessfullyStartedContainers(succeededContainers);
|
||||
response.setFailedRequests(failedContainers);
|
||||
return response;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of <code>ContainerId</code> s of the containers that are
|
||||
* started successfully.
|
||||
*
|
||||
* @return the list of <code>ContainerId</code> s of the containers that are
|
||||
* started successfully.
|
||||
* @see ContainerManagementProtocol#startContainers(StartContainersRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract List<ContainerId> getSuccessfullyStartedContainers();
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract void setSuccessfullyStartedContainers(
|
||||
List<ContainerId> succeededContainers);
|
||||
|
||||
/**
|
||||
* Get the containerId-to-exception map in which the exception indicates error
|
||||
* from per container for failed requests
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract Map<ContainerId, SerializedException> getFailedRequests();
|
||||
|
||||
/**
|
||||
* Set the containerId-to-exception map in which the exception indicates error
|
||||
* from per container for failed requests
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract void setFailedRequests(
|
||||
Map<ContainerId, SerializedException> failedContainers);
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Get the meta-data from all auxiliary services running on the
|
||||
|
@ -76,8 +119,10 @@ public abstract class StartContainerResponse {
|
|||
* Set to the list of auxiliary services which have been started on the
|
||||
* <code>NodeManager</code>. This is done only once when the
|
||||
* <code>NodeManager</code> starts up
|
||||
* @param allServicesMetaData A map from auxiliary service names to the opaque
|
||||
* blob <code>ByteBuffer</code> for that auxiliary service
|
||||
*
|
||||
* @param allServicesMetaData
|
||||
* A map from auxiliary service names to the opaque blob
|
||||
* <code>ByteBuffer</code> for that auxiliary service
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
|
@ -1,47 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
/**
|
||||
* <p>The response sent by the <code>NodeManager</code> to the
|
||||
* <code>ApplicationMaster</code> when asked to <em>stop</em> an
|
||||
* allocated container.</p>
|
||||
*
|
||||
* <p>Currently, this is empty.</p>
|
||||
*
|
||||
* @see ContainerManagementProtocol#stopContainer(StopContainerRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract class StopContainerResponse {
|
||||
@Private
|
||||
@Unstable
|
||||
public static StopContainerResponse newInstance() {
|
||||
StopContainerResponse response =
|
||||
Records.newRecord(StopContainerResponse.class);
|
||||
return response;
|
||||
}
|
||||
}
|
|
@ -18,6 +18,8 @@
|
|||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
|
@ -26,36 +28,36 @@ import org.apache.hadoop.yarn.util.Records;
|
|||
|
||||
/**
|
||||
* <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> containers.</p>
|
||||
*
|
||||
* @see ContainerManagementProtocol#stopContainer(StopContainerRequest)
|
||||
* @see ContainerManagementProtocol#stopContainers(StopContainersRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract class StopContainerRequest {
|
||||
public abstract class StopContainersRequest {
|
||||
|
||||
@Public
|
||||
@Stable
|
||||
public static StopContainerRequest newInstance(ContainerId containerId) {
|
||||
StopContainerRequest request =
|
||||
Records.newRecord(StopContainerRequest.class);
|
||||
request.setContainerId(containerId);
|
||||
public static StopContainersRequest newInstance(List<ContainerId> containerIds) {
|
||||
StopContainersRequest request =
|
||||
Records.newRecord(StopContainersRequest.class);
|
||||
request.setContainerIds(containerIds);
|
||||
return request;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the <code>ContainerId</code> of the container to be stopped.
|
||||
* @return <code>ContainerId</code> of container to be stopped
|
||||
* Get the <code>ContainerId</code>s of the containers to be stopped.
|
||||
* @return <code>ContainerId</code>s of containers to be stopped
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract ContainerId getContainerId();
|
||||
public abstract List<ContainerId> getContainerIds();
|
||||
|
||||
/**
|
||||
* Set the <code>ContainerId</code> of the container to be stopped.
|
||||
* @param containerId <code>ContainerId</code> of the container to be stopped
|
||||
* Set the <code>ContainerId</code>s of the containers to be stopped.
|
||||
* @param containerIds <code>ContainerId</code>s of the containers to be stopped
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract void setContainerId(ContainerId containerId);
|
||||
public abstract void setContainerIds(List<ContainerId> containerIds);
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* The response sent by the <code>NodeManager</code> to the
|
||||
* <code>ApplicationMaster</code> when asked to <em>stop</em> allocated
|
||||
* containers.
|
||||
* </p>
|
||||
*
|
||||
* @see ContainerManagementProtocol#stopContainers(StopContainersRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract class StopContainersResponse {
|
||||
@Private
|
||||
@Unstable
|
||||
public static StopContainersResponse newInstance(
|
||||
List<ContainerId> succeededRequests,
|
||||
Map<ContainerId, SerializedException> failedRequests) {
|
||||
StopContainersResponse response =
|
||||
Records.newRecord(StopContainersResponse.class);
|
||||
response.setFailedRequests(failedRequests);
|
||||
response.setSuccessfullyStoppedContainers(succeededRequests);
|
||||
return response;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of containerIds of successfully stopped containers.
|
||||
*
|
||||
* @return the list of containerIds of successfully stopped containers.
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract List<ContainerId> getSuccessfullyStoppedContainers();
|
||||
|
||||
/**
|
||||
* Set the list of containerIds of successfully stopped containers.
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract void setSuccessfullyStoppedContainers(
|
||||
List<ContainerId> succeededRequests);
|
||||
|
||||
/**
|
||||
* Get the containerId-to-exception map in which the exception indicates error
|
||||
* from per container for failed requests
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public abstract Map<ContainerId, SerializedException> getFailedRequests();
|
||||
|
||||
/**
|
||||
* Set the containerId-to-exception map in which the exception indicates error
|
||||
* from per container for failed requests
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract void setFailedRequests(
|
||||
Map<ContainerId, SerializedException> failedRequests);
|
||||
}
|
|
@ -57,8 +57,8 @@ import org.apache.hadoop.yarn.util.Records;
|
|||
* start/stop containers.</p>
|
||||
*
|
||||
* @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
|
||||
* @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
|
||||
* @see ContainerManagementProtocol#stopContainer(org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest)
|
||||
* @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
|
||||
* @see ContainerManagementProtocol#stopContainers(org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
|
@ -155,7 +155,7 @@ public abstract class Container implements Comparable<Container> {
|
|||
* <code>Container</code> includes the <code>ContainerToken</code>.</p>
|
||||
*
|
||||
* @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
|
||||
* @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
|
||||
* @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
|
||||
*
|
||||
* @return <code>ContainerToken</code> for the container
|
||||
*/
|
||||
|
|
|
@ -49,7 +49,7 @@ import org.apache.hadoop.yarn.util.Records;
|
|||
* </ul>
|
||||
* </p>
|
||||
*
|
||||
* @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
|
||||
* @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
|
@ -132,7 +132,7 @@ public abstract class ContainerLaunchContext {
|
|||
|
||||
/**
|
||||
* <p>
|
||||
* Get application-specific binary <em>service data</em>. This is a map keyed
|
||||
* Set application-specific binary <em>service data</em>. This is a map keyed
|
||||
* by the name of each {@link AuxiliaryService} that is configured on a
|
||||
* NodeManager and value correspond to the application specific data targeted
|
||||
* for the keyed {@link AuxiliaryService}. All pre-existing Map entries are
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.util.Records;
|
|||
* @see LocalResourceVisibility
|
||||
* @see ContainerLaunchContext
|
||||
* @see ApplicationSubmissionContext
|
||||
* @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
|
||||
* @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
|
|
|
@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
|||
* @see LocalResource
|
||||
* @see ContainerLaunchContext
|
||||
* @see ApplicationSubmissionContext
|
||||
* @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
|
||||
* @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
|
|
|
@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
|||
* @see LocalResource
|
||||
* @see ContainerLaunchContext
|
||||
* @see ApplicationSubmissionContext
|
||||
* @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
|
||||
* @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
|
|
|
@ -0,0 +1,93 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.records;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract class SerializedException {
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public static SerializedException newInstance(Throwable e) {
|
||||
SerializedException exception =
|
||||
Records.newRecord(SerializedException.class);
|
||||
exception.init(e);
|
||||
return exception;
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs a new <code>SerializedException</code> with the specified detail
|
||||
* message and cause.
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract void init(String message, Throwable cause);
|
||||
|
||||
/**
|
||||
* Constructs a new <code>SerializedException</code> with the specified detail
|
||||
* message.
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract void init(String message);
|
||||
|
||||
/**
|
||||
* Constructs a new <code>SerializedException</code> with the specified cause.
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract void init(Throwable cause);
|
||||
|
||||
/**
|
||||
* Get the detail message string of this exception.
|
||||
* @return the detail message string of this exception.
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract String getMessage();
|
||||
|
||||
/**
|
||||
* Get the backtrace of this exception.
|
||||
* @return the backtrace of this exception.
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract String getRemoteTrace();
|
||||
|
||||
/**
|
||||
* Get the cause of this exception or null if the cause is nonexistent or
|
||||
* unknown.
|
||||
* @return the cause of this exception.
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract SerializedException getCause();
|
||||
|
||||
/**
|
||||
* Deserialize the exception to a new Throwable.
|
||||
* @return the Throwable form of this serialized exception.
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract Throwable deSerialize();
|
||||
}
|
|
@ -19,12 +19,12 @@
|
|||
package org.apache.hadoop.yarn.exceptions;
|
||||
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
|
||||
/**
|
||||
* This exception is thrown by a NodeManager that is rejecting start-container
|
||||
* requests via
|
||||
* {@link ContainerManagementProtocol#startContainer(StartContainerRequest)}
|
||||
* {@link ContainerManagementProtocol#startContainers(StartContainersRequest)}
|
||||
* for containers allocated by a previous instance of the RM.
|
||||
*/
|
||||
public class InvalidContainerException extends YarnException {
|
||||
|
|
|
@ -19,11 +19,11 @@
|
|||
package org.apache.hadoop.yarn.exceptions;
|
||||
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
|
||||
/**
|
||||
* This exception is thrown on
|
||||
* {@link ContainerManagementProtocol#startContainer(StartContainerRequest)} API
|
||||
* {@link ContainerManagementProtocol#startContainers(StartContainersRequest)} API
|
||||
* when an NM starts from scratch but has not yet connected with RM.
|
||||
*/
|
||||
public class NMNotYetReadyException extends YarnException {
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
|
|||
import org.apache.hadoop.classification.InterfaceStability.Evolving;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
|
||||
|
@ -70,7 +70,7 @@ public class ApplicationInitializationContext {
|
|||
|
||||
/**
|
||||
* Get the data sent to the NodeManager via
|
||||
* {@link ContainerManagementProtocol#startContainer(StartContainerRequest)}
|
||||
* {@link ContainerManagementProtocol#startContainers(StartContainersRequest)}
|
||||
* as part of {@link ContainerLaunchContext#getServiceData()}
|
||||
*
|
||||
* @return the servicesData for this application.
|
||||
|
|
|
@ -24,8 +24,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
|
|||
import org.apache.hadoop.classification.InterfaceStability.Evolving;
|
||||
import org.apache.hadoop.service.AbstractService;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
|
||||
/**
|
||||
|
@ -71,8 +71,8 @@ public abstract class AuxiliaryService extends AbstractService {
|
|||
*
|
||||
* <p>
|
||||
* The information is passed along to applications via
|
||||
* {@link StartContainerResponse#getAllServicesMetaData()} that is returned by
|
||||
* {@link ContainerManagementProtocol#startContainer(StartContainerRequest)}
|
||||
* {@link StartContainersResponse#getAllServicesMetaData()} that is returned by
|
||||
* {@link ContainerManagementProtocol#startContainers(StartContainersRequest)}
|
||||
* </p>
|
||||
*
|
||||
* @return meta-data for this service that should be made available to
|
||||
|
|
|
@ -30,7 +30,7 @@ option java_generate_equals_and_hash = true;
|
|||
import "yarn_service_protos.proto";
|
||||
|
||||
service ContainerManagementProtocolService {
|
||||
rpc startContainer(StartContainerRequestProto) returns (StartContainerResponseProto);
|
||||
rpc stopContainer(StopContainerRequestProto) returns (StopContainerResponseProto);
|
||||
rpc getContainerStatus(GetContainerStatusRequestProto) returns (GetContainerStatusResponseProto);
|
||||
rpc startContainers(StartContainersRequestProto) returns (StartContainersResponseProto);
|
||||
rpc stopContainers(StopContainersRequestProto) returns (StopContainersResponseProto);
|
||||
rpc getContainerStatuses(GetContainerStatusesRequestProto) returns (GetContainerStatusesResponseProto);
|
||||
}
|
||||
|
|
|
@ -182,3 +182,37 @@ message GetContainerStatusRequestProto {
|
|||
message GetContainerStatusResponseProto {
|
||||
optional ContainerStatusProto status = 1;
|
||||
}
|
||||
|
||||
//// bulk API records
|
||||
message StartContainersRequestProto {
|
||||
repeated StartContainerRequestProto start_container_request = 1;
|
||||
}
|
||||
|
||||
message ContainerExceptionMapProto {
|
||||
optional ContainerIdProto container_id = 1;
|
||||
optional SerializedExceptionProto exception = 2;
|
||||
}
|
||||
|
||||
message StartContainersResponseProto {
|
||||
repeated StringBytesMapProto services_meta_data = 1;
|
||||
repeated ContainerIdProto succeeded_requests = 2;
|
||||
repeated ContainerExceptionMapProto failed_requests = 3;
|
||||
}
|
||||
|
||||
message StopContainersRequestProto {
|
||||
repeated ContainerIdProto container_id = 1;
|
||||
}
|
||||
|
||||
message StopContainersResponseProto {
|
||||
repeated ContainerIdProto succeeded_requests = 1;
|
||||
repeated ContainerExceptionMapProto failed_requests = 2;
|
||||
}
|
||||
|
||||
message GetContainerStatusesRequestProto {
|
||||
repeated ContainerIdProto container_id = 1;
|
||||
}
|
||||
|
||||
message GetContainerStatusesResponseProto {
|
||||
repeated ContainerStatusProto status = 1;
|
||||
repeated ContainerExceptionMapProto failed_requests = 2;
|
||||
}
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.client.api.impl;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
@ -30,9 +32,14 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.Container;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
|
@ -181,11 +188,23 @@ public class NMClientImpl extends NMClient {
|
|||
proxy =
|
||||
cmProxy.getProxy(container.getNodeId().toString(),
|
||||
container.getId());
|
||||
allServiceResponse =
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
container.getContainerToken());
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
StartContainersResponse response =
|
||||
proxy
|
||||
.getContainerManagementProtocol().startContainer(
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
container.getContainerToken())).getAllServicesMetaData();
|
||||
.getContainerManagementProtocol().startContainers(allRequests);
|
||||
if (response.getFailedRequests() != null
|
||||
&& response.getFailedRequests().containsKey(container.getId())) {
|
||||
Throwable t =
|
||||
response.getFailedRequests().get(container.getId()).deSerialize();
|
||||
parseAndThrowException(t);
|
||||
}
|
||||
allServiceResponse = response.getAllServicesMetaData();
|
||||
startingContainer.state = ContainerState.RUNNING;
|
||||
} catch (YarnException e) {
|
||||
startingContainer.state = ContainerState.COMPLETE;
|
||||
|
@ -238,11 +257,20 @@ public class NMClientImpl extends NMClient {
|
|||
NodeId nodeId) throws YarnException, IOException {
|
||||
|
||||
ContainerManagementProtocolProxyData proxy = null;
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(containerId);
|
||||
try {
|
||||
proxy = cmProxy.getProxy(nodeId.toString(), containerId);
|
||||
ContainerStatus containerStatus =
|
||||
proxy.getContainerManagementProtocol().getContainerStatus(
|
||||
GetContainerStatusRequest.newInstance(containerId)).getStatus();
|
||||
GetContainerStatusesResponse response =
|
||||
proxy.getContainerManagementProtocol().getContainerStatuses(
|
||||
GetContainerStatusesRequest.newInstance(containerIds));
|
||||
if (response.getFailedRequests() != null
|
||||
&& response.getFailedRequests().containsKey(containerId)) {
|
||||
Throwable t =
|
||||
response.getFailedRequests().get(containerId).deSerialize();
|
||||
parseAndThrowException(t);
|
||||
}
|
||||
ContainerStatus containerStatus = response.getContainerStatuses().get(0);
|
||||
return containerStatus;
|
||||
} finally {
|
||||
if (proxy != null) {
|
||||
|
@ -254,10 +282,19 @@ public class NMClientImpl extends NMClient {
|
|||
private void stopContainerInternal(ContainerId containerId, NodeId nodeId)
|
||||
throws IOException, YarnException {
|
||||
ContainerManagementProtocolProxyData proxy = null;
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(containerId);
|
||||
try {
|
||||
proxy = cmProxy.getProxy(nodeId.toString(), containerId);
|
||||
proxy.getContainerManagementProtocol().stopContainer(
|
||||
StopContainerRequest.newInstance(containerId));
|
||||
StopContainersResponse response =
|
||||
proxy.getContainerManagementProtocol().stopContainers(
|
||||
StopContainersRequest.newInstance(containerIds));
|
||||
if (response.getFailedRequests() != null
|
||||
&& response.getFailedRequests().containsKey(containerId)) {
|
||||
Throwable t = response.getFailedRequests().get(containerId)
|
||||
.deSerialize();
|
||||
parseAndThrowException(t);
|
||||
}
|
||||
} finally {
|
||||
if (proxy != null) {
|
||||
cmProxy.mayBeCloseProxy(proxy);
|
||||
|
@ -285,4 +322,15 @@ public class NMClientImpl extends NMClient {
|
|||
public AtomicBoolean getCleanupRunningContainers() {
|
||||
return cleanupRunningContainers;
|
||||
}
|
||||
|
||||
private void parseAndThrowException(Throwable t) throws YarnException,
|
||||
IOException {
|
||||
if (t instanceof YarnException) {
|
||||
throw (YarnException) t;
|
||||
} else if (t instanceof InvalidToken) {
|
||||
throw (InvalidToken) t;
|
||||
} else {
|
||||
throw (IOException) t;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,24 +30,24 @@ import org.apache.hadoop.net.NetUtils;
|
|||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.ipc.RPCUtil;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
|
@ -87,45 +87,45 @@ public class ContainerManagementProtocolPBClientImpl implements ContainerManagem
|
|||
}
|
||||
|
||||
@Override
|
||||
public GetContainerStatusResponse getContainerStatus(
|
||||
GetContainerStatusRequest request) throws YarnException,
|
||||
IOException {
|
||||
GetContainerStatusRequestProto requestProto =
|
||||
((GetContainerStatusRequestPBImpl) request).getProto();
|
||||
public StartContainersResponse
|
||||
startContainers(StartContainersRequest requests) throws YarnException,
|
||||
IOException {
|
||||
StartContainersRequestProto requestProto =
|
||||
((StartContainersRequestPBImpl) requests).getProto();
|
||||
try {
|
||||
return new GetContainerStatusResponsePBImpl(proxy.getContainerStatus(
|
||||
return new StartContainersResponsePBImpl(proxy.startContainers(null,
|
||||
requestProto));
|
||||
} catch (ServiceException e) {
|
||||
RPCUtil.unwrapAndThrowException(e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public StopContainersResponse stopContainers(StopContainersRequest requests)
|
||||
throws YarnException, IOException {
|
||||
StopContainersRequestProto requestProto =
|
||||
((StopContainersRequestPBImpl) requests).getProto();
|
||||
try {
|
||||
return new StopContainersResponsePBImpl(proxy.stopContainers(null,
|
||||
requestProto));
|
||||
} catch (ServiceException e) {
|
||||
RPCUtil.unwrapAndThrowException(e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetContainerStatusesResponse getContainerStatuses(
|
||||
GetContainerStatusesRequest request) throws YarnException, IOException {
|
||||
GetContainerStatusesRequestProto requestProto =
|
||||
((GetContainerStatusesRequestPBImpl) request).getProto();
|
||||
try {
|
||||
return new GetContainerStatusesResponsePBImpl(proxy.getContainerStatuses(
|
||||
null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
RPCUtil.unwrapAndThrowException(e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public StartContainerResponse startContainer(StartContainerRequest request)
|
||||
throws YarnException, IOException {
|
||||
StartContainerRequestProto requestProto =
|
||||
((StartContainerRequestPBImpl) request).getProto();
|
||||
try {
|
||||
return new StartContainerResponsePBImpl(proxy.startContainer(null,
|
||||
requestProto));
|
||||
} catch (ServiceException e) {
|
||||
RPCUtil.unwrapAndThrowException(e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public StopContainerResponse stopContainer(StopContainerRequest request)
|
||||
throws YarnException, IOException {
|
||||
StopContainerRequestProto requestProto =
|
||||
((StopContainerRequestPBImpl) request).getProto();
|
||||
try {
|
||||
return new StopContainerResponsePBImpl(proxy.stopContainer(null,
|
||||
requestProto));
|
||||
} catch (ServiceException e) {
|
||||
RPCUtil.unwrapAndThrowException(e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,22 +23,22 @@ import java.io.IOException;
|
|||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersResponseProto;
|
||||
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
@ -53,12 +53,12 @@ public class ContainerManagementProtocolPBServiceImpl implements ContainerManage
|
|||
}
|
||||
|
||||
@Override
|
||||
public GetContainerStatusResponseProto getContainerStatus(RpcController arg0,
|
||||
GetContainerStatusRequestProto proto) throws ServiceException {
|
||||
GetContainerStatusRequestPBImpl request = new GetContainerStatusRequestPBImpl(proto);
|
||||
public StartContainersResponseProto startContainers(RpcController arg0,
|
||||
StartContainersRequestProto proto) throws ServiceException {
|
||||
StartContainersRequestPBImpl request = new StartContainersRequestPBImpl(proto);
|
||||
try {
|
||||
GetContainerStatusResponse response = real.getContainerStatus(request);
|
||||
return ((GetContainerStatusResponsePBImpl)response).getProto();
|
||||
StartContainersResponse response = real.startContainers(request);
|
||||
return ((StartContainersResponsePBImpl)response).getProto();
|
||||
} catch (YarnException e) {
|
||||
throw new ServiceException(e);
|
||||
} catch (IOException e) {
|
||||
|
@ -67,12 +67,12 @@ public class ContainerManagementProtocolPBServiceImpl implements ContainerManage
|
|||
}
|
||||
|
||||
@Override
|
||||
public StartContainerResponseProto startContainer(RpcController arg0,
|
||||
StartContainerRequestProto proto) throws ServiceException {
|
||||
StartContainerRequestPBImpl request = new StartContainerRequestPBImpl(proto);
|
||||
public StopContainersResponseProto stopContainers(RpcController arg0,
|
||||
StopContainersRequestProto proto) throws ServiceException {
|
||||
StopContainersRequestPBImpl request = new StopContainersRequestPBImpl(proto);
|
||||
try {
|
||||
StartContainerResponse response = real.startContainer(request);
|
||||
return ((StartContainerResponsePBImpl)response).getProto();
|
||||
StopContainersResponse response = real.stopContainers(request);
|
||||
return ((StopContainersResponsePBImpl)response).getProto();
|
||||
} catch (YarnException e) {
|
||||
throw new ServiceException(e);
|
||||
} catch (IOException e) {
|
||||
|
@ -81,17 +81,17 @@ public class ContainerManagementProtocolPBServiceImpl implements ContainerManage
|
|||
}
|
||||
|
||||
@Override
|
||||
public StopContainerResponseProto stopContainer(RpcController arg0,
|
||||
StopContainerRequestProto proto) throws ServiceException {
|
||||
StopContainerRequestPBImpl request = new StopContainerRequestPBImpl(proto);
|
||||
public GetContainerStatusesResponseProto getContainerStatuses(
|
||||
RpcController arg0, GetContainerStatusesRequestProto proto)
|
||||
throws ServiceException {
|
||||
GetContainerStatusesRequestPBImpl request = new GetContainerStatusesRequestPBImpl(proto);
|
||||
try {
|
||||
StopContainerResponse response = real.stopContainer(request);
|
||||
return ((StopContainerResponsePBImpl)response).getProto();
|
||||
GetContainerStatusesResponse response = real.getContainerStatuses(request);
|
||||
return ((GetContainerStatusesResponsePBImpl)response).getProto();
|
||||
} catch (YarnException e) {
|
||||
throw new ServiceException(e);
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,130 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
|
||||
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusResponseProtoOrBuilder;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public class GetContainerStatusResponsePBImpl extends GetContainerStatusResponse {
|
||||
GetContainerStatusResponseProto proto = GetContainerStatusResponseProto.getDefaultInstance();
|
||||
GetContainerStatusResponseProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
||||
private ContainerStatus containerStatus = null;
|
||||
|
||||
|
||||
public GetContainerStatusResponsePBImpl() {
|
||||
builder = GetContainerStatusResponseProto.newBuilder();
|
||||
}
|
||||
|
||||
public GetContainerStatusResponsePBImpl(GetContainerStatusResponseProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public GetContainerStatusResponseProto getProto() {
|
||||
mergeLocalToProto();
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return getProto().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other == null)
|
||||
return false;
|
||||
if (other.getClass().isAssignableFrom(this.getClass())) {
|
||||
return this.getProto().equals(this.getClass().cast(other).getProto());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getProto().toString().replaceAll("\\n", ", ").replaceAll("\\s+", " ");
|
||||
}
|
||||
|
||||
private void mergeLocalToBuilder() {
|
||||
if (this.containerStatus != null) {
|
||||
builder.setStatus(convertToProtoFormat(this.containerStatus));
|
||||
}
|
||||
}
|
||||
|
||||
private void mergeLocalToProto() {
|
||||
if (viaProto)
|
||||
maybeInitBuilder();
|
||||
mergeLocalToBuilder();
|
||||
proto = builder.build();
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
private void maybeInitBuilder() {
|
||||
if (viaProto || builder == null) {
|
||||
builder = GetContainerStatusResponseProto.newBuilder(proto);
|
||||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public ContainerStatus getStatus() {
|
||||
GetContainerStatusResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||
if (this.containerStatus != null) {
|
||||
return this.containerStatus;
|
||||
}
|
||||
if (!p.hasStatus()) {
|
||||
return null;
|
||||
}
|
||||
this.containerStatus = convertFromProtoFormat(p.getStatus());
|
||||
return this.containerStatus;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setStatus(ContainerStatus status) {
|
||||
maybeInitBuilder();
|
||||
if (status == null)
|
||||
builder.clearStatus();
|
||||
this.containerStatus = status;
|
||||
}
|
||||
|
||||
private ContainerStatusPBImpl convertFromProtoFormat(ContainerStatusProto p) {
|
||||
return new ContainerStatusPBImpl(p);
|
||||
}
|
||||
|
||||
private ContainerStatusProto convertToProtoFormat(ContainerStatus t) {
|
||||
return ((ContainerStatusPBImpl)t).getProto();
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
|
@ -18,37 +18,41 @@
|
|||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProtoOrBuilder;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProtoOrBuilder;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public class GetContainerStatusRequestPBImpl extends GetContainerStatusRequest {
|
||||
GetContainerStatusRequestProto proto = GetContainerStatusRequestProto.getDefaultInstance();
|
||||
GetContainerStatusRequestProto.Builder builder = null;
|
||||
public class GetContainerStatusesRequestPBImpl extends
|
||||
GetContainerStatusesRequest {
|
||||
GetContainerStatusesRequestProto proto = GetContainerStatusesRequestProto
|
||||
.getDefaultInstance();
|
||||
GetContainerStatusesRequestProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
||||
private ContainerId containerId = null;
|
||||
|
||||
|
||||
public GetContainerStatusRequestPBImpl() {
|
||||
builder = GetContainerStatusRequestProto.newBuilder();
|
||||
|
||||
private List<ContainerId> containerIds = null;
|
||||
|
||||
public GetContainerStatusesRequestPBImpl() {
|
||||
builder = GetContainerStatusesRequestProto.newBuilder();
|
||||
}
|
||||
|
||||
public GetContainerStatusRequestPBImpl(GetContainerStatusRequestProto proto) {
|
||||
public GetContainerStatusesRequestPBImpl(
|
||||
GetContainerStatusesRequestProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public GetContainerStatusRequestProto getProto() {
|
||||
mergeLocalToProto();
|
||||
|
||||
public GetContainerStatusesRequestProto getProto() {
|
||||
mergeLocalToProto();
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
||||
|
@ -71,17 +75,18 @@ public class GetContainerStatusRequestPBImpl extends GetContainerStatusRequest {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getProto().toString().replaceAll("\\n", ", ").replaceAll("\\s+", " ");
|
||||
return getProto().toString().replaceAll("\\n", ", ")
|
||||
.replaceAll("\\s+", " ");
|
||||
}
|
||||
|
||||
private void mergeLocalToBuilder() {
|
||||
if (this.containerId != null) {
|
||||
builder.setContainerId(convertToProtoFormat(this.containerId));
|
||||
if (this.containerIds != null) {
|
||||
addLocalContainerIdsToProto();
|
||||
}
|
||||
}
|
||||
|
||||
private void mergeLocalToProto() {
|
||||
if (viaProto)
|
||||
if (viaProto)
|
||||
maybeInitBuilder();
|
||||
mergeLocalToBuilder();
|
||||
proto = builder.build();
|
||||
|
@ -90,31 +95,47 @@ public class GetContainerStatusRequestPBImpl extends GetContainerStatusRequest {
|
|||
|
||||
private void maybeInitBuilder() {
|
||||
if (viaProto || builder == null) {
|
||||
builder = GetContainerStatusRequestProto.newBuilder(proto);
|
||||
builder = GetContainerStatusesRequestProto.newBuilder(proto);
|
||||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public ContainerId getContainerId() {
|
||||
GetContainerStatusRequestProtoOrBuilder p = viaProto ? proto : builder;
|
||||
if (this.containerId != null) {
|
||||
return this.containerId;
|
||||
|
||||
private void addLocalContainerIdsToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearContainerId();
|
||||
if (this.containerIds == null)
|
||||
return;
|
||||
List<ContainerIdProto> protoList = new ArrayList<ContainerIdProto>();
|
||||
for (ContainerId id : containerIds) {
|
||||
protoList.add(convertToProtoFormat(id));
|
||||
}
|
||||
if (!p.hasContainerId()) {
|
||||
return null;
|
||||
builder.addAllContainerId(protoList);
|
||||
}
|
||||
|
||||
private void initLocalContainerIds() {
|
||||
if (this.containerIds != null) {
|
||||
return;
|
||||
}
|
||||
GetContainerStatusesRequestProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<ContainerIdProto> containerIds = p.getContainerIdList();
|
||||
this.containerIds = new ArrayList<ContainerId>();
|
||||
for (ContainerIdProto id : containerIds) {
|
||||
this.containerIds.add(convertFromProtoFormat(id));
|
||||
}
|
||||
this.containerId = convertFromProtoFormat(p.getContainerId());
|
||||
return this.containerId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setContainerId(ContainerId containerId) {
|
||||
public List<ContainerId> getContainerIds() {
|
||||
initLocalContainerIds();
|
||||
return this.containerIds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setContainerIds(List<ContainerId> containerIds) {
|
||||
maybeInitBuilder();
|
||||
if (containerId == null)
|
||||
if (containerIds == null)
|
||||
builder.clearContainerId();
|
||||
this.containerId = containerId;
|
||||
this.containerIds = containerIds;
|
||||
}
|
||||
|
||||
private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
|
||||
|
@ -122,9 +143,7 @@ public class GetContainerStatusRequestPBImpl extends GetContainerStatusRequest {
|
|||
}
|
||||
|
||||
private ContainerIdProto convertToProtoFormat(ContainerId t) {
|
||||
return ((ContainerIdPBImpl)t).getProto();
|
||||
return ((ContainerIdPBImpl) t).getProto();
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,223 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerExceptionMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesResponseProtoOrBuilder;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public class GetContainerStatusesResponsePBImpl extends
|
||||
GetContainerStatusesResponse {
|
||||
GetContainerStatusesResponseProto proto = GetContainerStatusesResponseProto
|
||||
.getDefaultInstance();
|
||||
GetContainerStatusesResponseProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
||||
private List<ContainerStatus> containerStatuses = null;
|
||||
private Map<ContainerId, SerializedException> failedRequests = null;
|
||||
|
||||
public GetContainerStatusesResponsePBImpl() {
|
||||
builder = GetContainerStatusesResponseProto.newBuilder();
|
||||
}
|
||||
|
||||
public GetContainerStatusesResponsePBImpl(
|
||||
GetContainerStatusesResponseProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public GetContainerStatusesResponseProto getProto() {
|
||||
mergeLocalToProto();
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return getProto().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other == null)
|
||||
return false;
|
||||
if (other.getClass().isAssignableFrom(this.getClass())) {
|
||||
return this.getProto().equals(this.getClass().cast(other).getProto());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getProto().toString().replaceAll("\\n", ", ")
|
||||
.replaceAll("\\s+", " ");
|
||||
}
|
||||
|
||||
private void mergeLocalToBuilder() {
|
||||
if (this.containerStatuses != null) {
|
||||
addLocalContainerStatusesToProto();
|
||||
}
|
||||
if (this.failedRequests != null) {
|
||||
addFailedRequestsToProto();
|
||||
}
|
||||
}
|
||||
|
||||
private void mergeLocalToProto() {
|
||||
if (viaProto)
|
||||
maybeInitBuilder();
|
||||
mergeLocalToBuilder();
|
||||
proto = builder.build();
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
private void maybeInitBuilder() {
|
||||
if (viaProto || builder == null) {
|
||||
builder = GetContainerStatusesResponseProto.newBuilder(proto);
|
||||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
private void addLocalContainerStatusesToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearStatus();
|
||||
if (this.containerStatuses == null)
|
||||
return;
|
||||
List<ContainerStatusProto> protoList =
|
||||
new ArrayList<ContainerStatusProto>();
|
||||
for (ContainerStatus status : containerStatuses) {
|
||||
protoList.add(convertToProtoFormat(status));
|
||||
}
|
||||
builder.addAllStatus(protoList);
|
||||
}
|
||||
|
||||
private void addFailedRequestsToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearFailedRequests();
|
||||
if (this.failedRequests == null)
|
||||
return;
|
||||
List<ContainerExceptionMapProto> protoList =
|
||||
new ArrayList<ContainerExceptionMapProto>();
|
||||
for (Map.Entry<ContainerId, SerializedException> entry : this.failedRequests
|
||||
.entrySet()) {
|
||||
protoList.add(ContainerExceptionMapProto.newBuilder()
|
||||
.setContainerId(convertToProtoFormat(entry.getKey()))
|
||||
.setException(convertToProtoFormat(entry.getValue())).build());
|
||||
}
|
||||
builder.addAllFailedRequests(protoList);
|
||||
}
|
||||
|
||||
private void initLocalContainerStatuses() {
|
||||
if (this.containerStatuses != null) {
|
||||
return;
|
||||
}
|
||||
GetContainerStatusesResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<ContainerStatusProto> statuses = p.getStatusList();
|
||||
this.containerStatuses = new ArrayList<ContainerStatus>();
|
||||
for (ContainerStatusProto status : statuses) {
|
||||
this.containerStatuses.add(convertFromProtoFormat(status));
|
||||
}
|
||||
}
|
||||
|
||||
private void initFailedRequests() {
|
||||
if (this.failedRequests != null) {
|
||||
return;
|
||||
}
|
||||
GetContainerStatusesResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<ContainerExceptionMapProto> protoList = p.getFailedRequestsList();
|
||||
this.failedRequests = new HashMap<ContainerId, SerializedException>();
|
||||
for (ContainerExceptionMapProto ce : protoList) {
|
||||
this.failedRequests.put(convertFromProtoFormat(ce.getContainerId()),
|
||||
convertFromProtoFormat(ce.getException()));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ContainerStatus> getContainerStatuses() {
|
||||
initLocalContainerStatuses();
|
||||
return this.containerStatuses;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setContainerStatuses(List<ContainerStatus> statuses) {
|
||||
maybeInitBuilder();
|
||||
if (statuses == null)
|
||||
builder.clearStatus();
|
||||
this.containerStatuses = statuses;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<ContainerId, SerializedException> getFailedRequests() {
|
||||
initFailedRequests();
|
||||
return this.failedRequests;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setFailedRequests(
|
||||
Map<ContainerId, SerializedException> failedRequests) {
|
||||
maybeInitBuilder();
|
||||
if (failedRequests == null)
|
||||
builder.clearFailedRequests();
|
||||
this.failedRequests = failedRequests;
|
||||
}
|
||||
|
||||
private ContainerStatusPBImpl convertFromProtoFormat(ContainerStatusProto p) {
|
||||
return new ContainerStatusPBImpl(p);
|
||||
}
|
||||
|
||||
private ContainerStatusProto convertToProtoFormat(ContainerStatus t) {
|
||||
return ((ContainerStatusPBImpl) t).getProto();
|
||||
}
|
||||
|
||||
private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
|
||||
return new ContainerIdPBImpl(p);
|
||||
}
|
||||
|
||||
private ContainerIdProto convertToProtoFormat(ContainerId t) {
|
||||
return ((ContainerIdPBImpl) t).getProto();
|
||||
}
|
||||
|
||||
private SerializedExceptionPBImpl convertFromProtoFormat(
|
||||
SerializedExceptionProto p) {
|
||||
return new SerializedExceptionPBImpl(p);
|
||||
}
|
||||
|
||||
private SerializedExceptionProto convertToProtoFormat(SerializedException t) {
|
||||
return ((SerializedExceptionPBImpl) t).getProto();
|
||||
}
|
||||
}
|
|
@ -1,176 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.StringBytesMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerResponseProtoOrBuilder;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public class StartContainerResponsePBImpl extends StartContainerResponse {
|
||||
StartContainerResponseProto proto = StartContainerResponseProto.getDefaultInstance();
|
||||
StartContainerResponseProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
||||
private Map<String, ByteBuffer> servicesMetaData = null;
|
||||
|
||||
public StartContainerResponsePBImpl() {
|
||||
builder = StartContainerResponseProto.newBuilder();
|
||||
}
|
||||
|
||||
public StartContainerResponsePBImpl(StartContainerResponseProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public synchronized StartContainerResponseProto getProto() {
|
||||
mergeLocalToProto();
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return getProto().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other == null)
|
||||
return false;
|
||||
if (other.getClass().isAssignableFrom(this.getClass())) {
|
||||
return this.getProto().equals(this.getClass().cast(other).getProto());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getProto().toString().replaceAll("\\n", ", ").replaceAll("\\s+", " ");
|
||||
}
|
||||
|
||||
private synchronized void mergeLocalToBuilder() {
|
||||
if (this.servicesMetaData != null) {
|
||||
addServicesMetaDataToProto();
|
||||
}
|
||||
}
|
||||
|
||||
protected final ByteBuffer convertFromProtoFormat(ByteString byteString) {
|
||||
return ProtoUtils.convertFromProtoFormat(byteString);
|
||||
}
|
||||
|
||||
protected final ByteString convertToProtoFormat(ByteBuffer byteBuffer) {
|
||||
return ProtoUtils.convertToProtoFormat(byteBuffer);
|
||||
}
|
||||
|
||||
private synchronized void mergeLocalToProto() {
|
||||
if (viaProto) {
|
||||
maybeInitBuilder();
|
||||
}
|
||||
mergeLocalToBuilder();
|
||||
proto = builder.build();
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
private synchronized void maybeInitBuilder() {
|
||||
if (viaProto || builder == null) {
|
||||
builder = StartContainerResponseProto.newBuilder(proto);
|
||||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public synchronized Map<String, ByteBuffer> getAllServicesMetaData() {
|
||||
initServicesMetaData();
|
||||
return this.servicesMetaData;
|
||||
}
|
||||
@Override
|
||||
public synchronized void setAllServicesMetaData(
|
||||
Map<String, ByteBuffer> servicesMetaData) {
|
||||
if(servicesMetaData == null) {
|
||||
return;
|
||||
}
|
||||
initServicesMetaData();
|
||||
this.servicesMetaData.clear();
|
||||
this.servicesMetaData.putAll(servicesMetaData);
|
||||
}
|
||||
|
||||
private synchronized void initServicesMetaData() {
|
||||
if (this.servicesMetaData != null) {
|
||||
return;
|
||||
}
|
||||
StartContainerResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<StringBytesMapProto> list = p.getServicesMetaDataList();
|
||||
this.servicesMetaData = new HashMap<String, ByteBuffer>();
|
||||
|
||||
for (StringBytesMapProto c : list) {
|
||||
this.servicesMetaData.put(c.getKey(), convertFromProtoFormat(c.getValue()));
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized void addServicesMetaDataToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearServicesMetaData();
|
||||
if (servicesMetaData == null)
|
||||
return;
|
||||
Iterable<StringBytesMapProto> iterable = new Iterable<StringBytesMapProto>() {
|
||||
|
||||
@Override
|
||||
public synchronized Iterator<StringBytesMapProto> iterator() {
|
||||
return new Iterator<StringBytesMapProto>() {
|
||||
|
||||
Iterator<String> keyIter = servicesMetaData.keySet().iterator();
|
||||
|
||||
@Override
|
||||
public synchronized void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized StringBytesMapProto next() {
|
||||
String key = keyIter.next();
|
||||
return StringBytesMapProto.newBuilder().setKey(key).setValue(convertToProtoFormat(servicesMetaData.get(key))).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized boolean hasNext() {
|
||||
return keyIter.hasNext();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
builder.addAllServicesMetaData(iterable);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,139 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProtoOrBuilder;
|
||||
|
||||
public class StartContainersRequestPBImpl extends StartContainersRequest {
|
||||
StartContainersRequestProto proto = StartContainersRequestProto
|
||||
.getDefaultInstance();
|
||||
StartContainersRequestProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
||||
private List<StartContainerRequest> requests = null;
|
||||
|
||||
public StartContainersRequestPBImpl() {
|
||||
builder = StartContainersRequestProto.newBuilder();
|
||||
}
|
||||
|
||||
public StartContainersRequestPBImpl(StartContainersRequestProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public StartContainersRequestProto getProto() {
|
||||
mergeLocalToProto();
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return getProto().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other == null)
|
||||
return false;
|
||||
if (other.getClass().isAssignableFrom(this.getClass())) {
|
||||
return this.getProto().equals(this.getClass().cast(other).getProto());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private void mergeLocalToProto() {
|
||||
if (viaProto)
|
||||
maybeInitBuilder();
|
||||
mergeLocalToBuilder();
|
||||
proto = builder.build();
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
private void mergeLocalToBuilder() {
|
||||
if (requests != null) {
|
||||
addLocalRequestsToProto();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void maybeInitBuilder() {
|
||||
if (viaProto || builder == null) {
|
||||
builder = StartContainersRequestProto.newBuilder(proto);
|
||||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
private void addLocalRequestsToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearStartContainerRequest();
|
||||
List<StartContainerRequestProto> protoList =
|
||||
new ArrayList<StartContainerRequestProto>();
|
||||
for (StartContainerRequest r : this.requests) {
|
||||
protoList.add(convertToProtoFormat(r));
|
||||
}
|
||||
builder.addAllStartContainerRequest(protoList);
|
||||
}
|
||||
|
||||
private void initLocalRequests() {
|
||||
StartContainersRequestProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<StartContainerRequestProto> requestList =
|
||||
p.getStartContainerRequestList();
|
||||
this.requests = new ArrayList<StartContainerRequest>();
|
||||
for (StartContainerRequestProto r : requestList) {
|
||||
this.requests.add(convertFromProtoFormat(r));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setStartContainerRequests(List<StartContainerRequest> requests) {
|
||||
maybeInitBuilder();
|
||||
if (requests == null) {
|
||||
builder.clearStartContainerRequest();
|
||||
}
|
||||
this.requests = requests;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<StartContainerRequest> getStartContainerRequests() {
|
||||
if (this.requests != null) {
|
||||
return this.requests;
|
||||
}
|
||||
initLocalRequests();
|
||||
return this.requests;
|
||||
}
|
||||
|
||||
private StartContainerRequestPBImpl convertFromProtoFormat(
|
||||
StartContainerRequestProto p) {
|
||||
return new StartContainerRequestPBImpl(p);
|
||||
}
|
||||
|
||||
private StartContainerRequestProto convertToProtoFormat(
|
||||
StartContainerRequest t) {
|
||||
return ((StartContainerRequestPBImpl) t).getProto();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,319 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.StringBytesMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerExceptionMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersResponseProtoOrBuilder;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public class StartContainersResponsePBImpl extends StartContainersResponse {
|
||||
StartContainersResponseProto proto = StartContainersResponseProto
|
||||
.getDefaultInstance();
|
||||
StartContainersResponseProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
||||
private Map<String, ByteBuffer> servicesMetaData = null;
|
||||
private List<ContainerId> succeededContainers = null;
|
||||
private Map<ContainerId, SerializedException> failedContainers = null;
|
||||
|
||||
public StartContainersResponsePBImpl() {
|
||||
builder = StartContainersResponseProto.newBuilder();
|
||||
}
|
||||
|
||||
public StartContainersResponsePBImpl(StartContainersResponseProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public StartContainersResponseProto getProto() {
|
||||
mergeLocalToProto();
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return getProto().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other == null)
|
||||
return false;
|
||||
if (other.getClass().isAssignableFrom(this.getClass())) {
|
||||
return this.getProto().equals(this.getClass().cast(other).getProto());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getProto().toString().replaceAll("\\n", ", ")
|
||||
.replaceAll("\\s+", " ");
|
||||
}
|
||||
|
||||
private void mergeLocalToBuilder() {
|
||||
if (this.servicesMetaData != null) {
|
||||
addServicesMetaDataToProto();
|
||||
}
|
||||
if (this.succeededContainers != null) {
|
||||
addSucceededContainersToProto();
|
||||
}
|
||||
if (this.failedContainers != null) {
|
||||
addFailedContainersToProto();
|
||||
}
|
||||
}
|
||||
|
||||
protected final ByteBuffer convertFromProtoFormat(ByteString byteString) {
|
||||
return ProtoUtils.convertFromProtoFormat(byteString);
|
||||
}
|
||||
|
||||
protected final ByteString convertToProtoFormat(ByteBuffer byteBuffer) {
|
||||
return ProtoUtils.convertToProtoFormat(byteBuffer);
|
||||
}
|
||||
|
||||
private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
|
||||
return new ContainerIdPBImpl(p);
|
||||
}
|
||||
|
||||
private ContainerIdProto convertToProtoFormat(ContainerId t) {
|
||||
return ((ContainerIdPBImpl) t).getProto();
|
||||
}
|
||||
|
||||
private SerializedExceptionPBImpl convertFromProtoFormat(
|
||||
SerializedExceptionProto p) {
|
||||
return new SerializedExceptionPBImpl(p);
|
||||
}
|
||||
|
||||
private SerializedExceptionProto convertToProtoFormat(SerializedException t) {
|
||||
return ((SerializedExceptionPBImpl) t).getProto();
|
||||
}
|
||||
|
||||
private void mergeLocalToProto() {
|
||||
if (viaProto) {
|
||||
maybeInitBuilder();
|
||||
}
|
||||
mergeLocalToBuilder();
|
||||
proto = builder.build();
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
private void maybeInitBuilder() {
|
||||
if (viaProto || builder == null) {
|
||||
builder = StartContainersResponseProto.newBuilder(proto);
|
||||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, ByteBuffer> getAllServicesMetaData() {
|
||||
initServicesMetaData();
|
||||
return this.servicesMetaData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setAllServicesMetaData(Map<String, ByteBuffer> servicesMetaData) {
|
||||
if (servicesMetaData == null) {
|
||||
return;
|
||||
}
|
||||
initServicesMetaData();
|
||||
this.servicesMetaData.clear();
|
||||
this.servicesMetaData.putAll(servicesMetaData);
|
||||
}
|
||||
|
||||
private void initServicesMetaData() {
|
||||
if (this.servicesMetaData != null) {
|
||||
return;
|
||||
}
|
||||
StartContainersResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<StringBytesMapProto> list = p.getServicesMetaDataList();
|
||||
this.servicesMetaData = new HashMap<String, ByteBuffer>();
|
||||
|
||||
for (StringBytesMapProto c : list) {
|
||||
this.servicesMetaData.put(c.getKey(),
|
||||
convertFromProtoFormat(c.getValue()));
|
||||
}
|
||||
}
|
||||
|
||||
private void addServicesMetaDataToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearServicesMetaData();
|
||||
if (servicesMetaData == null)
|
||||
return;
|
||||
Iterable<StringBytesMapProto> iterable =
|
||||
new Iterable<StringBytesMapProto>() {
|
||||
|
||||
@Override
|
||||
public Iterator<StringBytesMapProto> iterator() {
|
||||
return new Iterator<StringBytesMapProto>() {
|
||||
|
||||
Iterator<String> keyIter = servicesMetaData.keySet().iterator();
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public StringBytesMapProto next() {
|
||||
String key = keyIter.next();
|
||||
return StringBytesMapProto.newBuilder().setKey(key)
|
||||
.setValue(convertToProtoFormat(servicesMetaData.get(key)))
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return keyIter.hasNext();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
builder.addAllServicesMetaData(iterable);
|
||||
}
|
||||
|
||||
private void addFailedContainersToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearFailedRequests();
|
||||
if (this.failedContainers == null)
|
||||
return;
|
||||
List<ContainerExceptionMapProto> protoList =
|
||||
new ArrayList<ContainerExceptionMapProto>();
|
||||
|
||||
for (Map.Entry<ContainerId, SerializedException> entry : this.failedContainers
|
||||
.entrySet()) {
|
||||
protoList.add(ContainerExceptionMapProto.newBuilder()
|
||||
.setContainerId(convertToProtoFormat(entry.getKey()))
|
||||
.setException(convertToProtoFormat(entry.getValue())).build());
|
||||
}
|
||||
builder.addAllFailedRequests(protoList);
|
||||
}
|
||||
|
||||
private void addSucceededContainersToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearSucceededRequests();
|
||||
if (this.succeededContainers == null) {
|
||||
return;
|
||||
}
|
||||
Iterable<ContainerIdProto> iterable = new Iterable<ContainerIdProto>() {
|
||||
@Override
|
||||
public Iterator<ContainerIdProto> iterator() {
|
||||
return new Iterator<ContainerIdProto>() {
|
||||
|
||||
Iterator<ContainerId> iter = succeededContainers.iterator();
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return iter.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerIdProto next() {
|
||||
return convertToProtoFormat(iter.next());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
builder.addAllSucceededRequests(iterable);
|
||||
}
|
||||
|
||||
private void initSucceededContainers() {
|
||||
if (this.succeededContainers != null)
|
||||
return;
|
||||
StartContainersResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<ContainerIdProto> list = p.getSucceededRequestsList();
|
||||
this.succeededContainers = new ArrayList<ContainerId>();
|
||||
for (ContainerIdProto c : list) {
|
||||
this.succeededContainers.add(convertFromProtoFormat(c));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ContainerId> getSuccessfullyStartedContainers() {
|
||||
initSucceededContainers();
|
||||
return this.succeededContainers;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setSuccessfullyStartedContainers(
|
||||
List<ContainerId> succeededContainers) {
|
||||
maybeInitBuilder();
|
||||
if (succeededContainers == null) {
|
||||
builder.clearSucceededRequests();
|
||||
}
|
||||
this.succeededContainers = succeededContainers;
|
||||
}
|
||||
|
||||
private void initFailedContainers() {
|
||||
if (this.failedContainers != null) {
|
||||
return;
|
||||
}
|
||||
StartContainersResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<ContainerExceptionMapProto> protoList = p.getFailedRequestsList();
|
||||
this.failedContainers = new HashMap<ContainerId, SerializedException>();
|
||||
for (ContainerExceptionMapProto ce : protoList) {
|
||||
this.failedContainers.put(convertFromProtoFormat(ce.getContainerId()),
|
||||
convertFromProtoFormat(ce.getException()));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<ContainerId, SerializedException> getFailedRequests() {
|
||||
initFailedContainers();
|
||||
return this.failedContainers;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setFailedRequests(
|
||||
Map<ContainerId, SerializedException> failedContainers) {
|
||||
maybeInitBuilder();
|
||||
if (failedContainers == null)
|
||||
builder.clearFailedRequests();
|
||||
this.failedContainers = failedContainers;
|
||||
}
|
||||
}
|
|
@ -1,130 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
|
||||
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerRequestProtoOrBuilder;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public class StopContainerRequestPBImpl extends StopContainerRequest {
|
||||
StopContainerRequestProto proto = StopContainerRequestProto.getDefaultInstance();
|
||||
StopContainerRequestProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
||||
private ContainerId containerId = null;
|
||||
|
||||
|
||||
public StopContainerRequestPBImpl() {
|
||||
builder = StopContainerRequestProto.newBuilder();
|
||||
}
|
||||
|
||||
public StopContainerRequestPBImpl(StopContainerRequestProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public StopContainerRequestProto getProto() {
|
||||
mergeLocalToProto();
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return getProto().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other == null)
|
||||
return false;
|
||||
if (other.getClass().isAssignableFrom(this.getClass())) {
|
||||
return this.getProto().equals(this.getClass().cast(other).getProto());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getProto().toString().replaceAll("\\n", ", ").replaceAll("\\s+", " ");
|
||||
}
|
||||
|
||||
private void mergeLocalToBuilder() {
|
||||
if (this.containerId != null) {
|
||||
builder.setContainerId(convertToProtoFormat(this.containerId));
|
||||
}
|
||||
}
|
||||
|
||||
private void mergeLocalToProto() {
|
||||
if (viaProto)
|
||||
maybeInitBuilder();
|
||||
mergeLocalToBuilder();
|
||||
proto = builder.build();
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
private void maybeInitBuilder() {
|
||||
if (viaProto || builder == null) {
|
||||
builder = StopContainerRequestProto.newBuilder(proto);
|
||||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public ContainerId getContainerId() {
|
||||
StopContainerRequestProtoOrBuilder p = viaProto ? proto : builder;
|
||||
if (this.containerId != null) {
|
||||
return this.containerId;
|
||||
}
|
||||
if (!p.hasContainerId()) {
|
||||
return null;
|
||||
}
|
||||
this.containerId = convertFromProtoFormat(p.getContainerId());
|
||||
return this.containerId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setContainerId(ContainerId containerId) {
|
||||
maybeInitBuilder();
|
||||
if (containerId == null)
|
||||
builder.clearContainerId();
|
||||
this.containerId = containerId;
|
||||
}
|
||||
|
||||
private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
|
||||
return new ContainerIdPBImpl(p);
|
||||
}
|
||||
|
||||
private ContainerIdProto convertToProtoFormat(ContainerId t) {
|
||||
return ((ContainerIdPBImpl)t).getProto();
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
|
@ -1,68 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
|
||||
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerResponseProto;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public class StopContainerResponsePBImpl extends StopContainerResponse {
|
||||
StopContainerResponseProto proto = StopContainerResponseProto.getDefaultInstance();
|
||||
StopContainerResponseProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
||||
public StopContainerResponsePBImpl() {
|
||||
builder = StopContainerResponseProto.newBuilder();
|
||||
}
|
||||
|
||||
public StopContainerResponsePBImpl(StopContainerResponseProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public StopContainerResponseProto getProto() {
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return getProto().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other == null)
|
||||
return false;
|
||||
if (other.getClass().isAssignableFrom(this.getClass())) {
|
||||
return this.getProto().equals(this.getClass().cast(other).getProto());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getProto().toString().replaceAll("\\n", ", ").replaceAll("\\s+", " ");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,146 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProtoOrBuilder;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public class StopContainersRequestPBImpl extends StopContainersRequest {
|
||||
StopContainersRequestProto proto = StopContainersRequestProto
|
||||
.getDefaultInstance();
|
||||
StopContainersRequestProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
||||
private List<ContainerId> containerIds = null;
|
||||
|
||||
public StopContainersRequestPBImpl() {
|
||||
builder = StopContainersRequestProto.newBuilder();
|
||||
}
|
||||
|
||||
public StopContainersRequestPBImpl(StopContainersRequestProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public StopContainersRequestProto getProto() {
|
||||
mergeLocalToProto();
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return getProto().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other == null)
|
||||
return false;
|
||||
if (other.getClass().isAssignableFrom(this.getClass())) {
|
||||
return this.getProto().equals(this.getClass().cast(other).getProto());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getProto().toString().replaceAll("\\n", ", ")
|
||||
.replaceAll("\\s+", " ");
|
||||
}
|
||||
|
||||
private void mergeLocalToBuilder() {
|
||||
if (this.containerIds != null) {
|
||||
addLocalContainerIdsToProto();
|
||||
}
|
||||
}
|
||||
|
||||
private void mergeLocalToProto() {
|
||||
if (viaProto)
|
||||
maybeInitBuilder();
|
||||
mergeLocalToBuilder();
|
||||
proto = builder.build();
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
private void maybeInitBuilder() {
|
||||
if (viaProto || builder == null) {
|
||||
builder = StopContainersRequestProto.newBuilder(proto);
|
||||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
private void addLocalContainerIdsToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearContainerId();
|
||||
if (this.containerIds == null)
|
||||
return;
|
||||
List<ContainerIdProto> protoList = new ArrayList<ContainerIdProto>();
|
||||
for (ContainerId id : containerIds) {
|
||||
protoList.add(convertToProtoFormat(id));
|
||||
}
|
||||
builder.addAllContainerId(protoList);
|
||||
}
|
||||
|
||||
private void initLocalContainerIds() {
|
||||
if (this.containerIds != null) {
|
||||
return;
|
||||
}
|
||||
StopContainersRequestProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<ContainerIdProto> containerIds = p.getContainerIdList();
|
||||
this.containerIds = new ArrayList<ContainerId>();
|
||||
for (ContainerIdProto id : containerIds) {
|
||||
this.containerIds.add(convertFromProtoFormat(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ContainerId> getContainerIds() {
|
||||
initLocalContainerIds();
|
||||
return this.containerIds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setContainerIds(List<ContainerId> containerIds) {
|
||||
maybeInitBuilder();
|
||||
if (containerIds == null)
|
||||
builder.clearContainerId();
|
||||
this.containerIds = containerIds;
|
||||
}
|
||||
|
||||
private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
|
||||
return new ContainerIdPBImpl(p);
|
||||
}
|
||||
|
||||
private ContainerIdProto convertToProtoFormat(ContainerId t) {
|
||||
return ((ContainerIdPBImpl) t).getProto();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,234 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerExceptionMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersResponseProtoOrBuilder;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public class StopContainersResponsePBImpl extends StopContainersResponse {
|
||||
StopContainersResponseProto proto = StopContainersResponseProto
|
||||
.getDefaultInstance();
|
||||
StopContainersResponseProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
private List<ContainerId> succeededRequests = null;
|
||||
private Map<ContainerId, SerializedException> failedRequests = null;
|
||||
|
||||
public StopContainersResponsePBImpl() {
|
||||
builder = StopContainersResponseProto.newBuilder();
|
||||
}
|
||||
|
||||
public StopContainersResponsePBImpl(StopContainersResponseProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public StopContainersResponseProto getProto() {
|
||||
mergeLocalToProto();
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return getProto().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other == null)
|
||||
return false;
|
||||
if (other.getClass().isAssignableFrom(this.getClass())) {
|
||||
return this.getProto().equals(this.getClass().cast(other).getProto());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getProto().toString().replaceAll("\\n", ", ")
|
||||
.replaceAll("\\s+", " ");
|
||||
}
|
||||
|
||||
private void mergeLocalToProto() {
|
||||
if (viaProto) {
|
||||
maybeInitBuilder();
|
||||
}
|
||||
mergeLocalToBuilder();
|
||||
proto = builder.build();
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
private void maybeInitBuilder() {
|
||||
if (viaProto || builder == null) {
|
||||
builder = StopContainersResponseProto.newBuilder(proto);
|
||||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
private void mergeLocalToBuilder() {
|
||||
|
||||
if (this.succeededRequests != null) {
|
||||
addSucceededRequestsToProto();
|
||||
}
|
||||
if (this.failedRequests != null) {
|
||||
addFailedRequestsToProto();
|
||||
}
|
||||
}
|
||||
|
||||
private void addSucceededRequestsToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearSucceededRequests();
|
||||
if (this.succeededRequests == null) {
|
||||
return;
|
||||
}
|
||||
Iterable<ContainerIdProto> iterable = new Iterable<ContainerIdProto>() {
|
||||
@Override
|
||||
public Iterator<ContainerIdProto> iterator() {
|
||||
return new Iterator<ContainerIdProto>() {
|
||||
|
||||
Iterator<ContainerId> iter = succeededRequests.iterator();
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return iter.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerIdProto next() {
|
||||
return convertToProtoFormat(iter.next());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
builder.addAllSucceededRequests(iterable);
|
||||
}
|
||||
|
||||
private void addFailedRequestsToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearFailedRequests();
|
||||
if (this.failedRequests == null)
|
||||
return;
|
||||
List<ContainerExceptionMapProto> protoList =
|
||||
new ArrayList<ContainerExceptionMapProto>();
|
||||
|
||||
for (Map.Entry<ContainerId, SerializedException> entry : this.failedRequests
|
||||
.entrySet()) {
|
||||
protoList.add(ContainerExceptionMapProto.newBuilder()
|
||||
.setContainerId(convertToProtoFormat(entry.getKey()))
|
||||
.setException(convertToProtoFormat(entry.getValue())).build());
|
||||
}
|
||||
builder.addAllFailedRequests(protoList);
|
||||
}
|
||||
|
||||
private void initSucceededRequests() {
|
||||
if (this.succeededRequests != null)
|
||||
return;
|
||||
StopContainersResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<ContainerIdProto> list = p.getSucceededRequestsList();
|
||||
this.succeededRequests = new ArrayList<ContainerId>();
|
||||
for (ContainerIdProto c : list) {
|
||||
this.succeededRequests.add(convertFromProtoFormat(c));
|
||||
}
|
||||
}
|
||||
|
||||
private void initFailedRequests() {
|
||||
if (this.failedRequests != null) {
|
||||
return;
|
||||
}
|
||||
StopContainersResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<ContainerExceptionMapProto> protoList = p.getFailedRequestsList();
|
||||
this.failedRequests = new HashMap<ContainerId, SerializedException>();
|
||||
for (ContainerExceptionMapProto ce : protoList) {
|
||||
this.failedRequests.put(convertFromProtoFormat(ce.getContainerId()),
|
||||
convertFromProtoFormat(ce.getException()));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ContainerId> getSuccessfullyStoppedContainers() {
|
||||
initSucceededRequests();
|
||||
return this.succeededRequests;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setSuccessfullyStoppedContainers(List<ContainerId> succeededRequests) {
|
||||
maybeInitBuilder();
|
||||
if (succeededRequests == null) {
|
||||
builder.clearSucceededRequests();
|
||||
}
|
||||
this.succeededRequests = succeededRequests;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<ContainerId, SerializedException> getFailedRequests() {
|
||||
initFailedRequests();
|
||||
return this.failedRequests;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setFailedRequests(
|
||||
Map<ContainerId, SerializedException> failedRequests) {
|
||||
maybeInitBuilder();
|
||||
if (failedRequests == null)
|
||||
builder.clearFailedRequests();
|
||||
this.failedRequests = failedRequests;
|
||||
}
|
||||
|
||||
private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
|
||||
return new ContainerIdPBImpl(p);
|
||||
}
|
||||
|
||||
private ContainerIdProto convertToProtoFormat(ContainerId t) {
|
||||
return ((ContainerIdPBImpl) t).getProto();
|
||||
}
|
||||
|
||||
private SerializedExceptionPBImpl convertFromProtoFormat(
|
||||
SerializedExceptionProto p) {
|
||||
return new SerializedExceptionPBImpl(p);
|
||||
}
|
||||
|
||||
private SerializedExceptionProto convertToProtoFormat(SerializedException t) {
|
||||
return ((SerializedExceptionPBImpl) t).getProto();
|
||||
}
|
||||
}
|
|
@ -16,14 +16,19 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.api.records.impl.pb;
|
||||
package org.apache.hadoop.yarn.api.records.impl.pb;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
import java.io.StringWriter;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProtoOrBuilder;
|
||||
import org.apache.hadoop.yarn.server.api.records.SerializedException;
|
||||
|
||||
public class SerializedExceptionPBImpl extends SerializedException {
|
||||
|
||||
|
@ -58,7 +63,6 @@ public class SerializedExceptionPBImpl extends SerializedException {
|
|||
if (t.getCause() == null) {
|
||||
} else {
|
||||
builder.setCause(new SerializedExceptionPBImpl(t.getCause()).getProto());
|
||||
builder.setClassName(t.getClass().getCanonicalName());
|
||||
}
|
||||
StringWriter sw = new StringWriter();
|
||||
PrintWriter pw = new PrintWriter(sw);
|
||||
|
@ -68,6 +72,7 @@ public class SerializedExceptionPBImpl extends SerializedException {
|
|||
builder.setTrace(sw.toString());
|
||||
if (t.getMessage() != null)
|
||||
builder.setMessage(t.getMessage());
|
||||
builder.setClassName(t.getClass().getCanonicalName());
|
||||
}
|
||||
|
||||
public void init(String message, Throwable t) {
|
||||
|
@ -76,6 +81,32 @@ public class SerializedExceptionPBImpl extends SerializedException {
|
|||
builder.setMessage(message);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public Throwable deSerialize() {
|
||||
|
||||
SerializedException cause = getCause();
|
||||
SerializedExceptionProtoOrBuilder p = viaProto ? proto : builder;
|
||||
Class<?> realClass = null;
|
||||
try {
|
||||
realClass = Class.forName(p.getClassName());
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new YarnRuntimeException(e);
|
||||
}
|
||||
Class classType = null;
|
||||
if (YarnException.class.isAssignableFrom(realClass)) {
|
||||
classType = YarnException.class;
|
||||
} else if (IOException.class.isAssignableFrom(realClass)) {
|
||||
classType = IOException.class;
|
||||
} else if (RuntimeException.class.isAssignableFrom(realClass)) {
|
||||
classType = RuntimeException.class;
|
||||
} else {
|
||||
classType = Exception.class;
|
||||
}
|
||||
return instantiateException(realClass.asSubclass(classType), getMessage(),
|
||||
cause == null ? null : cause.deSerialize());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getMessage() {
|
||||
SerializedExceptionProtoOrBuilder p = viaProto ? proto : builder;
|
||||
|
@ -110,4 +141,29 @@ public class SerializedExceptionPBImpl extends SerializedException {
|
|||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
private static <T extends Throwable> T instantiateException(
|
||||
Class<? extends T> cls, String message, Throwable cause) {
|
||||
Constructor<? extends T> cn;
|
||||
T ex = null;
|
||||
try {
|
||||
cn = cls.getConstructor(String.class);
|
||||
cn.setAccessible(true);
|
||||
ex = cn.newInstance(message);
|
||||
ex.initCause(cause);
|
||||
} catch (SecurityException e) {
|
||||
throw new YarnRuntimeException(e);
|
||||
} catch (NoSuchMethodException e) {
|
||||
throw new YarnRuntimeException(e);
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new YarnRuntimeException(e);
|
||||
} catch (InstantiationException e) {
|
||||
throw new YarnRuntimeException(e);
|
||||
} catch (IllegalAccessException e) {
|
||||
throw new YarnRuntimeException(e);
|
||||
} catch (InvocationTargetException e) {
|
||||
throw new YarnRuntimeException(e);
|
||||
}
|
||||
return ex;
|
||||
}
|
||||
}
|
|
@ -21,6 +21,8 @@ package org.apache.hadoop.yarn;
|
|||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
|
@ -31,12 +33,13 @@ import org.apache.hadoop.ipc.Server;
|
|||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -104,12 +107,15 @@ public class TestContainerLaunchRPC {
|
|||
TestRPC.newContainerToken(nodeId, "password".getBytes(),
|
||||
containerTokenIdentifier);
|
||||
|
||||
StartContainerRequest scRequest = recordFactory
|
||||
.newRecordInstance(StartContainerRequest.class);
|
||||
scRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
scRequest.setContainerToken(containerToken);
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
containerToken);
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
try {
|
||||
proxy.startContainer(scRequest);
|
||||
proxy.startContainers(allRequests);
|
||||
} catch (Exception e) {
|
||||
LOG.info(StringUtils.stringifyException(e));
|
||||
Assert.assertEquals("Error, exception is not: "
|
||||
|
@ -129,17 +135,8 @@ public class TestContainerLaunchRPC {
|
|||
private ContainerStatus status = null;
|
||||
|
||||
@Override
|
||||
public GetContainerStatusResponse getContainerStatus(
|
||||
GetContainerStatusRequest request) throws YarnException {
|
||||
GetContainerStatusResponse response = recordFactory
|
||||
.newRecordInstance(GetContainerStatusResponse.class);
|
||||
response.setStatus(status);
|
||||
return response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StartContainerResponse startContainer(StartContainerRequest request)
|
||||
throws YarnException, IOException {
|
||||
public StartContainersResponse startContainers(
|
||||
StartContainersRequest requests) throws YarnException, IOException {
|
||||
try {
|
||||
// make the thread sleep to look like its not going to respond
|
||||
Thread.sleep(10000);
|
||||
|
@ -151,11 +148,22 @@ public class TestContainerLaunchRPC {
|
|||
}
|
||||
|
||||
@Override
|
||||
public StopContainerResponse stopContainer(StopContainerRequest request)
|
||||
throws YarnException {
|
||||
public StopContainersResponse
|
||||
stopContainers(StopContainersRequest requests) throws YarnException,
|
||||
IOException {
|
||||
Exception e = new Exception("Dummy function", new Exception(
|
||||
"Dummy function cause"));
|
||||
throw new YarnException(e);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetContainerStatusesResponse getContainerStatuses(
|
||||
GetContainerStatusesRequest request) throws YarnException, IOException {
|
||||
List<ContainerStatus> list = new ArrayList<ContainerStatus>();
|
||||
list.add(status);
|
||||
GetContainerStatusesResponse response =
|
||||
GetContainerStatusesResponse.newInstance(list, null);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.hadoop.yarn;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
|
@ -33,13 +35,14 @@ import org.apache.hadoop.security.SecurityUtil;
|
|||
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -122,9 +125,6 @@ public class TestRPC {
|
|||
ApplicationAttemptId.newInstance(applicationId, 0);
|
||||
ContainerId containerId =
|
||||
ContainerId.newInstance(applicationAttemptId, 100);
|
||||
StartContainerRequest scRequest =
|
||||
recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
scRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
NodeId nodeId = NodeId.newInstance("localhost", 1234);
|
||||
Resource resource = Resource.newInstance(1234, 2);
|
||||
ContainerTokenIdentifier containerTokenIdentifier =
|
||||
|
@ -132,22 +132,32 @@ public class TestRPC {
|
|||
resource, System.currentTimeMillis() + 10000, 42, 42);
|
||||
Token containerToken = newContainerToken(nodeId, "password".getBytes(),
|
||||
containerTokenIdentifier);
|
||||
scRequest.setContainerToken(containerToken);
|
||||
proxy.startContainer(scRequest);
|
||||
|
||||
GetContainerStatusRequest gcsRequest =
|
||||
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
|
||||
gcsRequest.setContainerId(containerId);
|
||||
GetContainerStatusResponse response = proxy.getContainerStatus(gcsRequest);
|
||||
ContainerStatus status = response.getStatus();
|
||||
|
||||
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
containerToken);
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
proxy.startContainers(allRequests);
|
||||
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(containerId);
|
||||
GetContainerStatusesRequest gcsRequest =
|
||||
GetContainerStatusesRequest.newInstance(containerIds);
|
||||
GetContainerStatusesResponse response =
|
||||
proxy.getContainerStatuses(gcsRequest);
|
||||
List<ContainerStatus> statuses = response.getContainerStatuses();
|
||||
|
||||
//test remote exception
|
||||
boolean exception = false;
|
||||
try {
|
||||
StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
|
||||
stopRequest.setContainerId(containerId);
|
||||
proxy.stopContainer(stopRequest);
|
||||
} catch (YarnException e) {
|
||||
StopContainersRequest stopRequest =
|
||||
recordFactory.newRecordInstance(StopContainersRequest.class);
|
||||
stopRequest.setContainerIds(containerIds);
|
||||
proxy.stopContainers(stopRequest);
|
||||
} catch (YarnException e) {
|
||||
exception = true;
|
||||
Assert.assertTrue(e.getMessage().contains(EXCEPTION_MSG));
|
||||
Assert.assertTrue(e.getMessage().contains(EXCEPTION_CAUSE));
|
||||
|
@ -158,46 +168,51 @@ public class TestRPC {
|
|||
Assert.assertTrue(exception);
|
||||
|
||||
server.stop();
|
||||
Assert.assertNotNull(status);
|
||||
Assert.assertEquals(ContainerState.RUNNING, status.getState());
|
||||
Assert.assertNotNull(statuses.get(0));
|
||||
Assert.assertEquals(ContainerState.RUNNING, statuses.get(0).getState());
|
||||
}
|
||||
|
||||
public class DummyContainerManager implements ContainerManagementProtocol {
|
||||
|
||||
private ContainerStatus status = null;
|
||||
|
||||
private List<ContainerStatus> statuses = new ArrayList<ContainerStatus>();
|
||||
|
||||
@Override
|
||||
public GetContainerStatusResponse getContainerStatus(
|
||||
GetContainerStatusRequest request)
|
||||
public GetContainerStatusesResponse getContainerStatuses(
|
||||
GetContainerStatusesRequest request)
|
||||
throws YarnException {
|
||||
GetContainerStatusResponse response =
|
||||
recordFactory.newRecordInstance(GetContainerStatusResponse.class);
|
||||
response.setStatus(status);
|
||||
GetContainerStatusesResponse response =
|
||||
recordFactory.newRecordInstance(GetContainerStatusesResponse.class);
|
||||
response.setContainerStatuses(statuses);
|
||||
return response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StartContainerResponse startContainer(StartContainerRequest request)
|
||||
throws YarnException {
|
||||
Token containerToken = request.getContainerToken();
|
||||
ContainerTokenIdentifier tokenId = null;
|
||||
public StartContainersResponse startContainers(
|
||||
StartContainersRequest requests) throws YarnException {
|
||||
StartContainersResponse response =
|
||||
recordFactory.newRecordInstance(StartContainersResponse.class);
|
||||
for (StartContainerRequest request : requests.getStartContainerRequests()) {
|
||||
Token containerToken = request.getContainerToken();
|
||||
ContainerTokenIdentifier tokenId = null;
|
||||
|
||||
try {
|
||||
tokenId = newContainerTokenIdentifier(containerToken);
|
||||
} catch (IOException e) {
|
||||
throw RPCUtil.getRemoteException(e);
|
||||
}
|
||||
ContainerStatus status =
|
||||
recordFactory.newRecordInstance(ContainerStatus.class);
|
||||
status.setState(ContainerState.RUNNING);
|
||||
status.setContainerId(tokenId.getContainerID());
|
||||
status.setExitStatus(0);
|
||||
statuses.add(status);
|
||||
|
||||
try {
|
||||
tokenId = newContainerTokenIdentifier(containerToken);
|
||||
} catch (IOException e) {
|
||||
throw RPCUtil.getRemoteException(e);
|
||||
}
|
||||
StartContainerResponse response =
|
||||
recordFactory.newRecordInstance(StartContainerResponse.class);
|
||||
status = recordFactory.newRecordInstance(ContainerStatus.class);
|
||||
status.setState(ContainerState.RUNNING);
|
||||
status.setContainerId(tokenId.getContainerID());
|
||||
status.setExitStatus(0);
|
||||
return response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StopContainerResponse stopContainer(StopContainerRequest request)
|
||||
public StopContainersResponse stopContainers(StopContainersRequest request)
|
||||
throws YarnException {
|
||||
Exception e = new Exception(EXCEPTION_MSG,
|
||||
new Exception(EXCEPTION_CAUSE));
|
||||
|
|
|
@ -1,39 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.api.records;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract class SerializedException {
|
||||
|
||||
public abstract void init(String message, Throwable cause);
|
||||
|
||||
public abstract void init(String message);
|
||||
|
||||
public abstract void init(Throwable cause);
|
||||
|
||||
public abstract String getMessage();
|
||||
|
||||
public abstract String getRemoteTrace();
|
||||
|
||||
public abstract SerializedException getCause();
|
||||
}
|
|
@ -22,12 +22,12 @@ import java.util.List;
|
|||
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
||||
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
||||
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
||||
import org.apache.hadoop.yarn.server.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
/**
|
||||
|
@ -59,10 +59,4 @@ public class YarnServerBuilderUtils {
|
|||
}
|
||||
return response;
|
||||
}
|
||||
|
||||
public static SerializedException newSerializedException(Throwable e) {
|
||||
SerializedException se = Records.newRecord(SerializedException.class);
|
||||
se.init(e);
|
||||
return se;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
|
|||
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodeHeartbeatRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodeHeartbeatResponsePBImpl;
|
||||
|
@ -45,7 +46,6 @@ import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
|||
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
||||
import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
|
||||
import org.apache.hadoop.yarn.server.api.records.impl.pb.NodeStatusPBImpl;
|
||||
import org.apache.hadoop.yarn.server.api.records.impl.pb.SerializedExceptionPBImpl;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
|
|
|
@ -18,8 +18,8 @@
|
|||
package org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords;
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.api.records.URL;
|
||||
import org.apache.hadoop.yarn.server.api.records.SerializedException;
|
||||
|
||||
public interface LocalResourceStatus {
|
||||
public LocalResource getResource();
|
||||
|
|
|
@ -18,9 +18,11 @@
|
|||
package org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb;
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.api.records.URL;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.URLPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
|
||||
|
@ -28,8 +30,6 @@ import org.apache.hadoop.yarn.proto.YarnProtos.URLProto;
|
|||
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalResourceStatusProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalResourceStatusProtoOrBuilder;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.ResourceStatusTypeProto;
|
||||
import org.apache.hadoop.yarn.server.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.server.api.records.impl.pb.SerializedExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType;
|
||||
|
||||
|
|
|
@ -23,12 +23,12 @@ import static org.apache.hadoop.service.Service.STATE.STARTED;
|
|||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -50,25 +50,25 @@ import org.apache.hadoop.service.Service;
|
|||
import org.apache.hadoop.service.ServiceStateChangeListener;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.exceptions.InvalidContainerException;
|
||||
import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.ipc.RPCUtil;
|
||||
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
||||
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
||||
|
@ -130,8 +130,6 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||
|
||||
private final NodeStatusUpdater nodeStatusUpdater;
|
||||
|
||||
private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
|
||||
|
||||
protected LocalDirsHandlerService dirsHandler;
|
||||
protected final AsyncDispatcher dispatcher;
|
||||
private final ApplicationACLsManager aclsManager;
|
||||
|
@ -316,18 +314,25 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||
return resultId;
|
||||
}
|
||||
|
||||
protected void authorizeUser(UserGroupInformation remoteUgi,
|
||||
NMTokenIdentifier nmTokenIdentifier) throws YarnException {
|
||||
if (!remoteUgi.getUserName().equals(
|
||||
nmTokenIdentifier.getApplicationAttemptId().toString())) {
|
||||
throw RPCUtil.getRemoteException("Expected applicationAttemptId: "
|
||||
+ remoteUgi.getUserName() + "Found: "
|
||||
+ nmTokenIdentifier.getApplicationAttemptId());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param containerTokenIdentifier
|
||||
* of the container to be started
|
||||
* @param ugi
|
||||
* ugi corresponding to the remote end making the api-call
|
||||
* @throws YarnException
|
||||
*/
|
||||
@Private
|
||||
@VisibleForTesting
|
||||
protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier,
|
||||
ContainerTokenIdentifier containerTokenIdentifier,
|
||||
UserGroupInformation ugi) throws YarnException {
|
||||
ContainerTokenIdentifier containerTokenIdentifier) throws YarnException {
|
||||
|
||||
ContainerId containerId = containerTokenIdentifier.getContainerID();
|
||||
String containerIDStr = containerId.toString();
|
||||
|
@ -342,12 +347,6 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||
.append(" was used for starting container with container token")
|
||||
.append(" issued for application attempt : ")
|
||||
.append(containerId.getApplicationAttemptId());
|
||||
} else if (!ugi.getUserName().equals(
|
||||
nmTokenIdentifier.getApplicationAttemptId().toString())) {
|
||||
unauthorized = true;
|
||||
messageBuilder.append("\nExpected applicationAttemptId: ")
|
||||
.append(ugi.getUserName()).append(" Found: ")
|
||||
.append(nmTokenIdentifier.getApplicationAttemptId().toString());
|
||||
} else if (!this.context.getContainerTokenSecretManager()
|
||||
.isValidStartContainerRequest(containerTokenIdentifier)) {
|
||||
// Is the container being relaunched? Or RPC layer let startCall with
|
||||
|
@ -363,7 +362,6 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||
.append(System.currentTimeMillis()).append(" found ")
|
||||
.append(containerTokenIdentifier.getExpiryTimeStamp());
|
||||
}
|
||||
|
||||
if (unauthorized) {
|
||||
String msg = messageBuilder.toString();
|
||||
LOG.error(msg);
|
||||
|
@ -372,18 +370,53 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||
}
|
||||
|
||||
/**
|
||||
* Start a container on this NodeManager.
|
||||
* Start a list of containers on this NodeManager.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public StartContainerResponse startContainer(StartContainerRequest request)
|
||||
throws YarnException, IOException {
|
||||
|
||||
public StartContainersResponse
|
||||
startContainers(StartContainersRequest requests) throws YarnException,
|
||||
IOException {
|
||||
if (blockNewContainerRequests.get()) {
|
||||
throw new NMNotYetReadyException(
|
||||
"Rejecting new containers as NodeManager has not"
|
||||
+ " yet connected with ResourceManager");
|
||||
}
|
||||
UserGroupInformation remoteUgi = getRemoteUgi();
|
||||
NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
|
||||
authorizeUser(remoteUgi,nmTokenIdentifier);
|
||||
List<ContainerId> succeededContainers = new ArrayList<ContainerId>();
|
||||
Map<ContainerId, SerializedException> failedContainers =
|
||||
new HashMap<ContainerId, SerializedException>();
|
||||
for (StartContainerRequest request : requests.getStartContainerRequests()) {
|
||||
ContainerId containerId = null;
|
||||
try {
|
||||
ContainerTokenIdentifier containerTokenIdentifier =
|
||||
BuilderUtils.newContainerTokenIdentifier(request.getContainerToken());
|
||||
verifyAndGetContainerTokenIdentifier(request.getContainerToken(),
|
||||
containerTokenIdentifier);
|
||||
containerId = containerTokenIdentifier.getContainerID();
|
||||
startContainerInternal(nmTokenIdentifier, containerTokenIdentifier,
|
||||
request);
|
||||
succeededContainers.add(containerId);
|
||||
} catch (YarnException e) {
|
||||
failedContainers.put(containerId, SerializedException.newInstance(e));
|
||||
} catch (InvalidToken ie) {
|
||||
failedContainers.put(containerId, SerializedException.newInstance(ie));
|
||||
throw ie;
|
||||
} catch (IOException e) {
|
||||
throw RPCUtil.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
return StartContainersResponse.newInstance(auxiliaryServices.getMetaData(),
|
||||
succeededContainers, failedContainers);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private void startContainerInternal(NMTokenIdentifier nmTokenIdentifier,
|
||||
ContainerTokenIdentifier containerTokenIdentifier,
|
||||
StartContainerRequest request) throws YarnException, IOException {
|
||||
|
||||
/*
|
||||
* 1) It should save the NMToken into NMTokenSecretManager. This is done
|
||||
* here instead of RPC layer because at the time of opening/authenticating
|
||||
|
@ -395,18 +428,8 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||
* belongs to correct Node Manager (part of retrieve password). c) It has
|
||||
* correct RMIdentifier. d) It is not expired.
|
||||
*/
|
||||
// update NMToken
|
||||
|
||||
UserGroupInformation remoteUgi = getRemoteUgi();
|
||||
NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
|
||||
|
||||
// Validate containerToken
|
||||
ContainerTokenIdentifier containerTokenIdentifier =
|
||||
verifyAndGetContainerTokenIdentifier(request.getContainerToken());
|
||||
|
||||
authorizeStartRequest(nmTokenIdentifier, containerTokenIdentifier,
|
||||
remoteUgi);
|
||||
|
||||
authorizeStartRequest(nmTokenIdentifier, containerTokenIdentifier);
|
||||
|
||||
if (containerTokenIdentifier.getRMIdentifer() != nodeStatusUpdater
|
||||
.getRMIdentifier()) {
|
||||
// Is the container coming from unknown RM
|
||||
|
@ -415,9 +438,9 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||
.append(" rejected as it is allocated by a previous RM");
|
||||
throw new InvalidContainerException(sb.toString());
|
||||
}
|
||||
|
||||
// update NMToken
|
||||
updateNMTokenIdentifier(nmTokenIdentifier);
|
||||
|
||||
|
||||
ContainerId containerId = containerTokenIdentifier.getContainerID();
|
||||
String containerIdStr = containerId.toString();
|
||||
String user = containerTokenIdentifier.getApplicationSubmitter();
|
||||
|
@ -461,26 +484,16 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||
containerTokenIdentifier);
|
||||
NMAuditLogger.logSuccess(user, AuditConstants.START_CONTAINER,
|
||||
"ContainerManageImpl", applicationID, containerId);
|
||||
StartContainerResponse response =
|
||||
recordFactory.newRecordInstance(StartContainerResponse.class);
|
||||
response.setAllServicesMetaData(auxiliaryServices.getMetaData());
|
||||
// TODO launchedContainer misplaced -> doesn't necessarily mean a container
|
||||
// launch. A finished Application will not launch containers.
|
||||
metrics.launchedContainer();
|
||||
metrics.allocateContainer(containerTokenIdentifier.getResource());
|
||||
return response;
|
||||
metrics.allocateContainer(containerTokenIdentifier.getResource());
|
||||
}
|
||||
|
||||
protected ContainerTokenIdentifier verifyAndGetContainerTokenIdentifier(
|
||||
org.apache.hadoop.yarn.api.records.Token token) throws YarnException,
|
||||
org.apache.hadoop.yarn.api.records.Token token,
|
||||
ContainerTokenIdentifier containerTokenIdentifier) throws YarnException,
|
||||
InvalidToken {
|
||||
ContainerTokenIdentifier containerTokenIdentifier = null;
|
||||
try {
|
||||
containerTokenIdentifier =
|
||||
BuilderUtils.newContainerTokenIdentifier(token);
|
||||
} catch (IOException e) {
|
||||
throw RPCUtil.getRemoteException(e);
|
||||
}
|
||||
byte[] password =
|
||||
context.getContainerTokenSecretManager().retrievePassword(
|
||||
containerTokenIdentifier);
|
||||
|
@ -528,21 +541,37 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||
}
|
||||
|
||||
/**
|
||||
* Stop the container running on this NodeManager.
|
||||
* Stop a list of containers running on this NodeManager.
|
||||
*/
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public StopContainerResponse stopContainer(StopContainerRequest request)
|
||||
public StopContainersResponse stopContainers(StopContainersRequest requests)
|
||||
throws YarnException, IOException {
|
||||
|
||||
ContainerId containerID = request.getContainerId();
|
||||
List<ContainerId> succeededRequests = new ArrayList<ContainerId>();
|
||||
Map<ContainerId, SerializedException> failedRequests =
|
||||
new HashMap<ContainerId, SerializedException>();
|
||||
UserGroupInformation remoteUgi = getRemoteUgi();
|
||||
NMTokenIdentifier identifier = selectNMTokenIdentifier(remoteUgi);
|
||||
for (ContainerId id : requests.getContainerIds()) {
|
||||
try {
|
||||
stopContainerInternal(identifier, id);
|
||||
succeededRequests.add(id);
|
||||
} catch (YarnException e) {
|
||||
failedRequests.put(id, SerializedException.newInstance(e));
|
||||
}
|
||||
}
|
||||
return StopContainersResponse
|
||||
.newInstance(succeededRequests, failedRequests);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private void stopContainerInternal(NMTokenIdentifier nmTokenIdentifier,
|
||||
ContainerId containerID) throws YarnException {
|
||||
String containerIDStr = containerID.toString();
|
||||
Container container = this.context.getContainers().get(containerID);
|
||||
LOG.info("Getting container-status for " + containerIDStr);
|
||||
authorizeGetAndStopContainerRequest(containerID, container, true);
|
||||
|
||||
StopContainerResponse response =
|
||||
recordFactory.newRecordInstance(StopContainerResponse.class);
|
||||
LOG.info("Stopping container with container Id: " + containerIDStr);
|
||||
authorizeGetAndStopContainerRequest(containerID, container, true,
|
||||
nmTokenIdentifier);
|
||||
|
||||
dispatcher.getEventHandler().handle(
|
||||
new ContainerKillEvent(containerID,
|
||||
|
@ -555,37 +584,51 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||
// TODO: Move this code to appropriate place once kill_container is
|
||||
// implemented.
|
||||
nodeStatusUpdater.sendOutofBandHeartBeat();
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a list of container statuses running on this NodeManager
|
||||
*/
|
||||
@Override
|
||||
public GetContainerStatusResponse getContainerStatus(
|
||||
GetContainerStatusRequest request) throws YarnException, IOException {
|
||||
public GetContainerStatusesResponse getContainerStatuses(
|
||||
GetContainerStatusesRequest request) throws YarnException, IOException {
|
||||
|
||||
ContainerId containerID = request.getContainerId();
|
||||
List<ContainerStatus> succeededRequests = new ArrayList<ContainerStatus>();
|
||||
Map<ContainerId, SerializedException> failedRequests =
|
||||
new HashMap<ContainerId, SerializedException>();
|
||||
UserGroupInformation remoteUgi = getRemoteUgi();
|
||||
NMTokenIdentifier identifier = selectNMTokenIdentifier(remoteUgi);
|
||||
for (ContainerId id : request.getContainerIds()) {
|
||||
try {
|
||||
ContainerStatus status = getContainerStatusInternal(id, identifier);
|
||||
succeededRequests.add(status);
|
||||
} catch (YarnException e) {
|
||||
failedRequests.put(id, SerializedException.newInstance(e));
|
||||
}
|
||||
}
|
||||
return GetContainerStatusesResponse.newInstance(succeededRequests,
|
||||
failedRequests);
|
||||
}
|
||||
|
||||
private ContainerStatus getContainerStatusInternal(ContainerId containerID,
|
||||
NMTokenIdentifier nmTokenIdentifier) throws YarnException {
|
||||
String containerIDStr = containerID.toString();
|
||||
Container container = this.context.getContainers().get(containerID);
|
||||
|
||||
LOG.info("Getting container-status for " + containerIDStr);
|
||||
authorizeGetAndStopContainerRequest(containerID, container, false);
|
||||
authorizeGetAndStopContainerRequest(containerID, container, false,
|
||||
nmTokenIdentifier);
|
||||
|
||||
ContainerStatus containerStatus = container.cloneAndGetContainerStatus();
|
||||
LOG.info("Returning " + containerStatus);
|
||||
GetContainerStatusResponse response =
|
||||
recordFactory.newRecordInstance(GetContainerStatusResponse.class);
|
||||
response.setStatus(containerStatus);
|
||||
return response;
|
||||
return containerStatus;
|
||||
}
|
||||
|
||||
@Private
|
||||
@VisibleForTesting
|
||||
protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
|
||||
Container container, boolean stopRequest) throws YarnException {
|
||||
|
||||
UserGroupInformation remoteUgi = getRemoteUgi();
|
||||
NMTokenIdentifier identifier = selectNMTokenIdentifier(remoteUgi);
|
||||
|
||||
Container container, boolean stopRequest, NMTokenIdentifier identifier)
|
||||
throws YarnException {
|
||||
/*
|
||||
* For get/stop container status; we need to verify that 1) User (NMToken)
|
||||
* application attempt only has started container. 2) Requested containerId
|
||||
|
|
|
@ -103,7 +103,7 @@ public class ContainerImpl implements Container {
|
|||
public ContainerImpl(Configuration conf, Dispatcher dispatcher,
|
||||
ContainerLaunchContext launchContext, Credentials creds,
|
||||
NodeManagerMetrics metrics,
|
||||
ContainerTokenIdentifier containerTokenIdentifier) throws IOException {
|
||||
ContainerTokenIdentifier containerTokenIdentifier) {
|
||||
this.daemonConf = conf;
|
||||
this.dispatcher = dispatcher;
|
||||
this.launchContext = launchContext;
|
||||
|
|
|
@ -53,6 +53,7 @@ import org.apache.hadoop.security.token.TokenIdentifier;
|
|||
import org.apache.hadoop.util.DiskChecker;
|
||||
import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
|
@ -288,11 +289,10 @@ public class ContainerLocalizer {
|
|||
stat.setStatus(ResourceStatusType.FETCH_SUCCESS);
|
||||
} catch (ExecutionException e) {
|
||||
stat.setStatus(ResourceStatusType.FETCH_FAILURE);
|
||||
stat.setException(
|
||||
YarnServerBuilderUtils.newSerializedException(e.getCause()));
|
||||
stat.setException(SerializedException.newInstance(e.getCause()));
|
||||
} catch (CancellationException e) {
|
||||
stat.setStatus(ResourceStatusType.FETCH_FAILURE);
|
||||
stat.setException(YarnServerBuilderUtils.newSerializedException(e));
|
||||
stat.setException(SerializedException.newInstance(e));
|
||||
}
|
||||
// TODO shouldn't remove until ACK
|
||||
i.remove();
|
||||
|
|
|
@ -195,14 +195,13 @@ public class DummyContainerManager extends ContainerManagerImpl {
|
|||
|
||||
@Override
|
||||
protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier,
|
||||
ContainerTokenIdentifier containerTokenIdentifier,
|
||||
UserGroupInformation ugi) throws YarnException {
|
||||
ContainerTokenIdentifier containerTokenIdentifier) throws YarnException {
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
|
||||
Container container, boolean stopRequest) throws YarnException {
|
||||
Container container, boolean stopRequest, NMTokenIdentifier identifier) throws YarnException {
|
||||
// do nothing
|
||||
}
|
||||
|
||||
|
|
|
@ -146,7 +146,7 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
|
|||
|
||||
@Override
|
||||
public void testContainerLaunchFromPreviousRM() throws InterruptedException,
|
||||
IOException {
|
||||
IOException, YarnException {
|
||||
// Don't run the test if the binary is not available.
|
||||
if (!shouldRunTest()) {
|
||||
LOG.info("LCE binary path is not passed. Not running the test");
|
||||
|
@ -155,6 +155,29 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
|
|||
LOG.info("Running testContainerLaunchFromPreviousRM");
|
||||
super.testContainerLaunchFromPreviousRM();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void testMultipleContainersLaunch() throws Exception {
|
||||
// Don't run the test if the binary is not available.
|
||||
if (!shouldRunTest()) {
|
||||
LOG.info("LCE binary path is not passed. Not running the test");
|
||||
return;
|
||||
}
|
||||
LOG.info("Running testContainerLaunchFromPreviousRM");
|
||||
super.testMultipleContainersLaunch();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void testMultipleContainersStopAndGetStatus() throws Exception {
|
||||
// Don't run the test if the binary is not available.
|
||||
if (!shouldRunTest()) {
|
||||
LOG.info("LCE binary path is not passed. Not running the test");
|
||||
return;
|
||||
}
|
||||
LOG.info("Running testContainerLaunchFromPreviousRM");
|
||||
super.testMultipleContainersStopAndGetStatus();
|
||||
}
|
||||
|
||||
private boolean shouldRunTest() {
|
||||
return System
|
||||
.getProperty(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH) != null;
|
||||
|
|
|
@ -20,18 +20,19 @@ package org.apache.hadoop.yarn.server.nodemanager;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.fs.FileContext;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerState;
|
||||
import org.apache.hadoop.yarn.api.records.Token;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
|
@ -46,7 +47,6 @@ import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
|||
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
|
||||
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||
import org.junit.Test;
|
||||
|
||||
|
||||
|
@ -62,7 +62,7 @@ public class TestEventFlow {
|
|||
private static File remoteLogDir = new File("target",
|
||||
TestEventFlow.class.getName() + "-remoteLogDir").getAbsoluteFile();
|
||||
private static final long SIMULATED_RM_IDENTIFIER = 1234;
|
||||
|
||||
|
||||
@Test
|
||||
public void testSuccessfulContainerLaunch() throws InterruptedException,
|
||||
IOException, YarnException {
|
||||
|
@ -140,21 +140,25 @@ public class TestEventFlow {
|
|||
ContainerId cID = ContainerId.newInstance(applicationAttemptId, 0);
|
||||
|
||||
String user = "testing";
|
||||
StartContainerRequest request =
|
||||
recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
request.setContainerLaunchContext(launchContext);
|
||||
request.setContainerToken(TestContainerManager.createContainerToken(cID,
|
||||
SIMULATED_RM_IDENTIFIER, context.getNodeId(), user,
|
||||
context.getContainerTokenSecretManager()));
|
||||
containerManager.startContainer(request);
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(launchContext,
|
||||
TestContainerManager.createContainerToken(cID,
|
||||
SIMULATED_RM_IDENTIFIER, context.getNodeId(), user,
|
||||
context.getContainerTokenSecretManager()));
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
containerManager.startContainers(allRequests);
|
||||
|
||||
BaseContainerManagerTest.waitForContainerState(containerManager, cID,
|
||||
ContainerState.RUNNING);
|
||||
|
||||
StopContainerRequest stopRequest =
|
||||
recordFactory.newRecordInstance(StopContainerRequest.class);
|
||||
stopRequest.setContainerId(cID);
|
||||
containerManager.stopContainer(stopRequest);
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(cID);
|
||||
StopContainersRequest stopRequest =
|
||||
StopContainersRequest.newInstance(containerIds);
|
||||
containerManager.stopContainers(stopRequest);
|
||||
BaseContainerManagerTest.waitForContainerState(containerManager, cID,
|
||||
ContainerState.COMPLETE);
|
||||
|
||||
|
|
|
@ -39,8 +39,9 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -129,13 +130,17 @@ public class TestNodeManagerReboot {
|
|||
List<String> commands = new ArrayList<String>();
|
||||
containerLaunchContext.setCommands(commands);
|
||||
|
||||
final StartContainerRequest startRequest =
|
||||
Records.newRecord(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
NodeId nodeId = nm.getNMContext().getNodeId();
|
||||
startRequest.setContainerToken(TestContainerManager.createContainerToken(
|
||||
cId, 0, nodeId, destinationFile, nm.getNMContext()
|
||||
.getContainerTokenSecretManager()));
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
TestContainerManager.createContainerToken(
|
||||
cId, 0, nodeId, destinationFile, nm.getNMContext()
|
||||
.getContainerTokenSecretManager()));
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
final StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
|
||||
final UserGroupInformation currentUser =
|
||||
UserGroupInformation.createRemoteUser(cId.getApplicationAttemptId()
|
||||
.toString());
|
||||
|
@ -145,16 +150,17 @@ public class TestNodeManagerReboot {
|
|||
currentUser.doAs(new PrivilegedExceptionAction<Void>() {
|
||||
@Override
|
||||
public Void run() throws YarnException, IOException {
|
||||
nm.getContainerManager().startContainer(startRequest);
|
||||
nm.getContainerManager().startContainers(allRequests);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
||||
GetContainerStatusRequest request =
|
||||
Records.newRecord(GetContainerStatusRequest.class);
|
||||
request.setContainerId(cId);
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(cId);
|
||||
GetContainerStatusesRequest request =
|
||||
GetContainerStatusesRequest.newInstance(containerIds);
|
||||
Container container =
|
||||
nm.getNMContext().getContainers().get(request.getContainerId());
|
||||
nm.getNMContext().getContainers().get(request.getContainerIds().get(0));
|
||||
|
||||
final int MAX_TRIES = 20;
|
||||
int numTries = 0;
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.server.nodemanager;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.BrokenBarrierException;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.CyclicBarrier;
|
||||
|
@ -31,6 +33,7 @@ import org.apache.hadoop.fs.FileContext;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
|
@ -282,15 +285,18 @@ public class TestNodeManagerResync {
|
|||
try {
|
||||
while (!isStopped && numContainers < 10) {
|
||||
ContainerId cId = TestNodeManagerShutdown.createContainerId();
|
||||
StartContainerRequest startRequest =
|
||||
recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
startRequest.setContainerToken(null);
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
null);
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
System.out.println("no. of containers to be launched: "
|
||||
+ numContainers);
|
||||
numContainers++;
|
||||
try {
|
||||
getContainerManager().startContainer(startRequest);
|
||||
getContainerManager().startContainers(allRequests);
|
||||
} catch (YarnException e) {
|
||||
numContainersRejected++;
|
||||
Assert.assertTrue(e.getMessage().contains(
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.io.IOException;
|
|||
import java.io.PrintWriter;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.security.PrivilegedAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -40,8 +41,9 @@ import org.apache.hadoop.net.NetUtils;
|
|||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.util.Shell;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -53,7 +55,6 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType;
|
|||
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.URL;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
|
@ -181,12 +182,6 @@ public class TestNodeManagerShutdown {
|
|||
containerLaunchContext.setLocalResources(localResources);
|
||||
List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
|
||||
containerLaunchContext.setCommands(commands);
|
||||
StartContainerRequest startRequest =
|
||||
recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
startRequest
|
||||
.setContainerToken(TestContainerManager.createContainerToken(cId, 0,
|
||||
nodeId, user, nm.getNMContext().getContainerTokenSecretManager()));
|
||||
final InetSocketAddress containerManagerBindAddress =
|
||||
NetUtils.createSocketAddrForHost("127.0.0.1", 12345);
|
||||
UserGroupInformation currentUser = UserGroupInformation
|
||||
|
@ -210,13 +205,22 @@ public class TestNodeManagerShutdown {
|
|||
containerManagerBindAddress, conf);
|
||||
}
|
||||
});
|
||||
containerManager.startContainer(startRequest);
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
TestContainerManager.createContainerToken(cId, 0,
|
||||
nodeId, user, nm.getNMContext().getContainerTokenSecretManager()));
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
containerManager.startContainers(allRequests);
|
||||
|
||||
GetContainerStatusRequest request =
|
||||
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
|
||||
request.setContainerId(cId);
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(cId);
|
||||
GetContainerStatusesRequest request =
|
||||
GetContainerStatusesRequest.newInstance(containerIds);
|
||||
ContainerStatus containerStatus =
|
||||
containerManager.getContainerStatus(request).getStatus();
|
||||
containerManager.getContainerStatuses(request).getContainerStatuses().get(0);
|
||||
Assert.assertEquals(ContainerState.RUNNING, containerStatus.getState());
|
||||
}
|
||||
|
||||
|
|
|
@ -45,8 +45,8 @@ import org.apache.hadoop.io.retry.RetryPolicy;
|
|||
import org.apache.hadoop.io.retry.RetryProxy;
|
||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.service.ServiceOperations;
|
||||
import org.apache.hadoop.service.Service.STATE;
|
||||
import org.apache.hadoop.service.ServiceOperations;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -65,7 +65,6 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
|
|||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.server.api.ResourceTracker;
|
||||
import org.apache.hadoop.yarn.server.api.ServerRMProxy;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
||||
|
|
|
@ -21,16 +21,21 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.io.DataOutputBuffer;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalResourceStatusProto;
|
||||
|
@ -42,7 +47,6 @@ import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAc
|
|||
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType;
|
||||
import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -80,7 +84,7 @@ public class TestPBRecordImpl {
|
|||
e.setStackTrace(new StackTraceElement[] {
|
||||
new StackTraceElement("foo", "bar", "baz", 10),
|
||||
new StackTraceElement("sbb", "one", "onm", 10) });
|
||||
ret.setException(YarnServerBuilderUtils.newSerializedException(e));
|
||||
ret.setException(SerializedException.newInstance(e));
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
@ -176,4 +180,33 @@ public class TestPBRecordImpl {
|
|||
assertEquals(createResource(), rsrcD.getResourceSpecs().get(0).getResource());
|
||||
}
|
||||
|
||||
|
||||
@Test(timeout=10000)
|
||||
public void testSerializedExceptionDeSer() throws Exception{
|
||||
// without cause
|
||||
YarnException yarnEx = new YarnException("Yarn_Exception");
|
||||
SerializedException serEx = SerializedException.newInstance(yarnEx);
|
||||
Throwable throwable = serEx.deSerialize();
|
||||
Assert.assertEquals(yarnEx.getClass(), throwable.getClass());
|
||||
Assert.assertEquals(yarnEx.getMessage(), throwable.getMessage());
|
||||
|
||||
// with cause
|
||||
IOException ioe = new IOException("Test_IOException");
|
||||
RuntimeException runtimeException =
|
||||
new RuntimeException("Test_RuntimeException", ioe);
|
||||
YarnException yarnEx2 =
|
||||
new YarnException("Test_YarnException", runtimeException);
|
||||
|
||||
SerializedException serEx2 = SerializedException.newInstance(yarnEx2);
|
||||
Throwable throwable2 = serEx2.deSerialize();
|
||||
throwable2.printStackTrace();
|
||||
Assert.assertEquals(yarnEx2.getClass(), throwable2.getClass());
|
||||
Assert.assertEquals(yarnEx2.getMessage(), throwable2.getMessage());
|
||||
|
||||
Assert.assertEquals(runtimeException.getClass(), throwable2.getCause().getClass());
|
||||
Assert.assertEquals(runtimeException.getMessage(), throwable2.getCause().getMessage());
|
||||
|
||||
Assert.assertEquals(ioe.getClass(), throwable2.getCause().getCause().getClass());
|
||||
Assert.assertEquals(ioe.getMessage(), throwable2.getCause().getCause().getMessage());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
|
@ -32,7 +34,7 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
|||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerState;
|
||||
|
@ -189,15 +191,18 @@ public abstract class BaseContainerManagerTest {
|
|||
|
||||
@Override
|
||||
protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
|
||||
Container container, boolean stopRequest) throws YarnException {
|
||||
Container container, boolean stopRequest, NMTokenIdentifier identifier) throws YarnException {
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void authorizeUser(UserGroupInformation remoteUgi,
|
||||
NMTokenIdentifier nmTokenIdentifier) {
|
||||
// do nothing
|
||||
}
|
||||
@Override
|
||||
protected void authorizeStartRequest(
|
||||
NMTokenIdentifier nmTokenIdentifier,
|
||||
ContainerTokenIdentifier containerTokenIdentifier,
|
||||
UserGroupInformation ugi) throws YarnException {
|
||||
ContainerTokenIdentifier containerTokenIdentifier) throws YarnException {
|
||||
// do nothing
|
||||
}
|
||||
|
||||
|
@ -238,18 +243,20 @@ public abstract class BaseContainerManagerTest {
|
|||
public static void waitForContainerState(ContainerManagementProtocol containerManager,
|
||||
ContainerId containerID, ContainerState finalState, int timeOutMax)
|
||||
throws InterruptedException, YarnException, IOException {
|
||||
GetContainerStatusRequest request =
|
||||
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
|
||||
request.setContainerId(containerID);
|
||||
ContainerStatus containerStatus =
|
||||
containerManager.getContainerStatus(request).getStatus();
|
||||
int timeoutSecs = 0;
|
||||
List<ContainerId> list = new ArrayList<ContainerId>();
|
||||
list.add(containerID);
|
||||
GetContainerStatusesRequest request =
|
||||
GetContainerStatusesRequest.newInstance(list);
|
||||
ContainerStatus containerStatus =
|
||||
containerManager.getContainerStatuses(request).getContainerStatuses()
|
||||
.get(0);
|
||||
int timeoutSecs = 0;
|
||||
while (!containerStatus.getState().equals(finalState)
|
||||
&& timeoutSecs++ < timeOutMax) {
|
||||
Thread.sleep(1000);
|
||||
LOG.info("Waiting for container to get into state " + finalState
|
||||
+ ". Current state is " + containerStatus.getState());
|
||||
containerStatus = containerManager.getContainerStatus(request).getStatus();
|
||||
containerStatus = containerManager.getContainerStatuses(request).getContainerStatuses().get(0);
|
||||
}
|
||||
LOG.info("Container state is " + containerStatus.getState());
|
||||
Assert.assertEquals("ContainerState is not correct (timedout)",
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.io.FileReader;
|
|||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
import java.net.InetAddress;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -37,9 +38,13 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.util.Shell;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -51,6 +56,7 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType;
|
|||
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.api.records.Token;
|
||||
import org.apache.hadoop.yarn.api.records.URL;
|
||||
import org.apache.hadoop.yarn.exceptions.InvalidContainerException;
|
||||
|
@ -63,6 +69,7 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
|
|||
import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
|
||||
|
@ -88,11 +95,11 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
super.setup();
|
||||
}
|
||||
|
||||
private ContainerId createContainerId() {
|
||||
private ContainerId createContainerId(int id) {
|
||||
ApplicationId appId = ApplicationId.newInstance(0, 0);
|
||||
ApplicationAttemptId appAttemptId =
|
||||
ApplicationAttemptId.newInstance(appId, 1);
|
||||
ContainerId containerId = ContainerId.newInstance(appAttemptId, 0);
|
||||
ContainerId containerId = ContainerId.newInstance(appAttemptId, id);
|
||||
return containerId;
|
||||
}
|
||||
|
||||
|
@ -119,6 +126,14 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
.getKeyId()));
|
||||
return ugi;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
|
||||
Container container, boolean stopRequest, NMTokenIdentifier identifier) throws YarnException {
|
||||
if(container == null || container.getUser().equals("Fail")){
|
||||
throw new YarnException("Reject this container");
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -138,12 +153,17 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
// Just do a query for a non-existing container.
|
||||
boolean throwsException = false;
|
||||
try {
|
||||
GetContainerStatusRequest request =
|
||||
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
|
||||
ContainerId cId = createContainerId();
|
||||
request.setContainerId(cId);
|
||||
containerManager.getContainerStatus(request);
|
||||
} catch (YarnException e) {
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
ContainerId id =createContainerId(0);
|
||||
containerIds.add(id);
|
||||
GetContainerStatusesRequest request =
|
||||
GetContainerStatusesRequest.newInstance(containerIds);
|
||||
GetContainerStatusesResponse response =
|
||||
containerManager.getContainerStatuses(request);
|
||||
if(response.getFailedRequests().containsKey(id)){
|
||||
throw response.getFailedRequests().get(id).deSerialize();
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
throwsException = true;
|
||||
}
|
||||
Assert.assertTrue(throwsException);
|
||||
|
@ -163,7 +183,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
fileWriter.close();
|
||||
|
||||
// ////// Construct the Container-id
|
||||
ContainerId cId = createContainerId();
|
||||
ContainerId cId = createContainerId(0);
|
||||
|
||||
// ////// Construct the container-spec.
|
||||
ContainerLaunchContext containerLaunchContext =
|
||||
|
@ -182,14 +202,17 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
new HashMap<String, LocalResource>();
|
||||
localResources.put(destinationFile, rsrc_alpha);
|
||||
containerLaunchContext.setLocalResources(localResources);
|
||||
StartContainerRequest startRequest =
|
||||
recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
startRequest.setContainerToken(createContainerToken(cId,
|
||||
DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
|
||||
context.getContainerTokenSecretManager()));
|
||||
|
||||
containerManager.startContainer(startRequest);
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(
|
||||
containerLaunchContext,
|
||||
createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(),
|
||||
user, context.getContainerTokenSecretManager()));
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
containerManager.startContainers(allRequests);
|
||||
|
||||
BaseContainerManagerTest.waitForContainerState(containerManager, cId,
|
||||
ContainerState.COMPLETE);
|
||||
|
@ -237,7 +260,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
new File(tmpDir, "start_file.txt").getAbsoluteFile();
|
||||
|
||||
// ////// Construct the Container-id
|
||||
ContainerId cId = createContainerId();
|
||||
ContainerId cId = createContainerId(0);
|
||||
|
||||
if (Shell.WINDOWS) {
|
||||
fileWriter.println("@echo Hello World!> " + processStartFile);
|
||||
|
@ -272,13 +295,17 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
|
||||
containerLaunchContext.setCommands(commands);
|
||||
|
||||
StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
startRequest.setContainerToken(createContainerToken(cId,
|
||||
DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
|
||||
context.getContainerTokenSecretManager()));
|
||||
containerManager.startContainer(startRequest);
|
||||
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
createContainerToken(cId,
|
||||
DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
|
||||
context.getContainerTokenSecretManager()));
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
containerManager.startContainers(allRequests);
|
||||
|
||||
int timeoutSecs = 0;
|
||||
while (!processStartFile.exists() && timeoutSecs++ < 20) {
|
||||
Thread.sleep(1000);
|
||||
|
@ -305,18 +332,18 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
Assert.assertTrue("Process is not alive!",
|
||||
DefaultContainerExecutor.containerIsAlive(pid));
|
||||
|
||||
StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
|
||||
stopRequest.setContainerId(cId);
|
||||
containerManager.stopContainer(stopRequest);
|
||||
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(cId);
|
||||
StopContainersRequest stopRequest =
|
||||
StopContainersRequest.newInstance(containerIds);
|
||||
containerManager.stopContainers(stopRequest);
|
||||
BaseContainerManagerTest.waitForContainerState(containerManager, cId,
|
||||
ContainerState.COMPLETE);
|
||||
|
||||
GetContainerStatusRequest gcsRequest =
|
||||
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
|
||||
gcsRequest.setContainerId(cId);
|
||||
GetContainerStatusesRequest gcsRequest =
|
||||
GetContainerStatusesRequest.newInstance(containerIds);
|
||||
ContainerStatus containerStatus =
|
||||
containerManager.getContainerStatus(gcsRequest).getStatus();
|
||||
containerManager.getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
|
||||
int expectedExitCode = Shell.WINDOWS ? ExitCode.FORCE_KILLED.getExitCode() :
|
||||
ExitCode.TERMINATED.getExitCode();
|
||||
Assert.assertEquals(expectedExitCode, containerStatus.getExitStatus());
|
||||
|
@ -325,7 +352,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
Assert.assertFalse("Process is still alive!",
|
||||
DefaultContainerExecutor.containerIsAlive(pid));
|
||||
}
|
||||
|
||||
|
||||
private void testContainerLaunchAndExit(int exitCode) throws IOException,
|
||||
InterruptedException, YarnException {
|
||||
|
||||
|
@ -335,7 +362,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
new File(tmpDir, "start_file.txt").getAbsoluteFile();
|
||||
|
||||
// ////// Construct the Container-id
|
||||
ContainerId cId = createContainerId();
|
||||
ContainerId cId = createContainerId(0);
|
||||
|
||||
if (Shell.WINDOWS) {
|
||||
fileWriter.println("@echo Hello World!> " + processStartFile);
|
||||
|
@ -376,21 +403,26 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
|
||||
containerLaunchContext.setCommands(commands);
|
||||
|
||||
StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
startRequest.setContainerToken(createContainerToken(cId,
|
||||
DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
|
||||
context.getContainerTokenSecretManager()));
|
||||
containerManager.startContainer(startRequest);
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(
|
||||
containerLaunchContext,
|
||||
createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(),
|
||||
user, context.getContainerTokenSecretManager()));
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
containerManager.startContainers(allRequests);
|
||||
|
||||
BaseContainerManagerTest.waitForContainerState(containerManager, cId,
|
||||
ContainerState.COMPLETE);
|
||||
|
||||
GetContainerStatusRequest gcsRequest =
|
||||
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
|
||||
gcsRequest.setContainerId(cId);
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(cId);
|
||||
GetContainerStatusesRequest gcsRequest =
|
||||
GetContainerStatusesRequest.newInstance(containerIds);
|
||||
ContainerStatus containerStatus =
|
||||
containerManager.getContainerStatus(gcsRequest).getStatus();
|
||||
containerManager.getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
|
||||
|
||||
// Verify exit status matches exit state of script
|
||||
Assert.assertEquals(exitCode,
|
||||
|
@ -439,7 +471,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
fileWriter.close();
|
||||
|
||||
// ////// Construct the Container-id
|
||||
ContainerId cId = createContainerId();
|
||||
ContainerId cId = createContainerId(0);
|
||||
ApplicationId appId = cId.getApplicationAttemptId().getApplicationId();
|
||||
|
||||
// ////// Construct the container-spec.
|
||||
|
@ -460,11 +492,17 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
new HashMap<String, LocalResource>();
|
||||
localResources.put(destinationFile, rsrc_alpha);
|
||||
containerLaunchContext.setLocalResources(localResources);
|
||||
StartContainerRequest request = recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
request.setContainerLaunchContext(containerLaunchContext);
|
||||
request.setContainerToken(createContainerToken(cId, DUMMY_RM_IDENTIFIER,
|
||||
context.getNodeId(), user, context.getContainerTokenSecretManager()));
|
||||
containerManager.startContainer(request);
|
||||
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(
|
||||
containerLaunchContext,
|
||||
createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(),
|
||||
user, context.getContainerTokenSecretManager()));
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
containerManager.startContainers(allRequests);
|
||||
|
||||
BaseContainerManagerTest.waitForContainerState(containerManager, cId,
|
||||
ContainerState.COMPLETE);
|
||||
|
@ -528,29 +566,37 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
|
||||
@Test
|
||||
public void testContainerLaunchFromPreviousRM() throws IOException,
|
||||
InterruptedException {
|
||||
InterruptedException, YarnException {
|
||||
containerManager.start();
|
||||
|
||||
ContainerLaunchContext containerLaunchContext =
|
||||
recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
||||
|
||||
ContainerId cId1 = createContainerId();
|
||||
ContainerId cId2 = createContainerId();
|
||||
ContainerId cId1 = createContainerId(0);
|
||||
ContainerId cId2 = createContainerId(0);
|
||||
containerLaunchContext
|
||||
.setLocalResources(new HashMap<String, LocalResource>());
|
||||
|
||||
// Construct the Container with Invalid RMIdentifier
|
||||
StartContainerRequest startRequest1 =
|
||||
recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
startRequest1.setContainerLaunchContext(containerLaunchContext);
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
createContainerToken(cId1,
|
||||
ResourceManagerConstants.RM_INVALID_IDENTIFIER, context.getNodeId(),
|
||||
user, context.getContainerTokenSecretManager()));
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(startRequest1);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
containerManager.startContainers(allRequests);
|
||||
|
||||
startRequest1.setContainerToken(createContainerToken(cId1,
|
||||
ResourceManagerConstants.RM_INVALID_IDENTIFIER, context.getNodeId(),
|
||||
user, context.getContainerTokenSecretManager()));
|
||||
boolean catchException = false;
|
||||
try {
|
||||
containerManager.startContainer(startRequest1);
|
||||
} catch (YarnException e) {
|
||||
StartContainersResponse response = containerManager.startContainers(allRequests);
|
||||
if(response.getFailedRequests().containsKey(cId1)) {
|
||||
throw response.getFailedRequests().get(cId1).deSerialize();
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
e.printStackTrace();
|
||||
catchException = true;
|
||||
Assert.assertTrue(e.getMessage().contains(
|
||||
"Container " + cId1 + " rejected as it is allocated by a previous RM"));
|
||||
|
@ -563,21 +609,143 @@ public class TestContainerManager extends BaseContainerManagerTest {
|
|||
|
||||
// Construct the Container with a RMIdentifier within current RM
|
||||
StartContainerRequest startRequest2 =
|
||||
recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
startRequest2.setContainerLaunchContext(containerLaunchContext);
|
||||
startRequest2.setContainerToken(createContainerToken(cId2,
|
||||
DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
|
||||
context.getContainerTokenSecretManager()));
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
createContainerToken(cId2,
|
||||
DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
|
||||
context.getContainerTokenSecretManager()));
|
||||
List<StartContainerRequest> list2 = new ArrayList<StartContainerRequest>();
|
||||
list.add(startRequest2);
|
||||
StartContainersRequest allRequests2 =
|
||||
StartContainersRequest.newInstance(list2);
|
||||
containerManager.startContainers(allRequests2);
|
||||
|
||||
boolean noException = true;
|
||||
try {
|
||||
containerManager.startContainer(startRequest2);
|
||||
containerManager.startContainers(allRequests2);
|
||||
} catch (YarnException e) {
|
||||
noException = false;
|
||||
}
|
||||
// Verify that startContainer get no YarnException
|
||||
Assert.assertTrue(noException);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testMultipleContainersLaunch() throws Exception {
|
||||
containerManager.start();
|
||||
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
ContainerLaunchContext containerLaunchContext =
|
||||
recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
ContainerId cId = createContainerId(i);
|
||||
long identifier = 0;
|
||||
if ((i & 1) == 0)
|
||||
// container with even id fail
|
||||
identifier = ResourceManagerConstants.RM_INVALID_IDENTIFIER;
|
||||
else
|
||||
identifier = DUMMY_RM_IDENTIFIER;
|
||||
Token containerToken =
|
||||
createContainerToken(cId, identifier, context.getNodeId(), user,
|
||||
context.getContainerTokenSecretManager());
|
||||
StartContainerRequest request =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
containerToken);
|
||||
list.add(request);
|
||||
}
|
||||
StartContainersRequest requestList =
|
||||
StartContainersRequest.newInstance(list);
|
||||
|
||||
StartContainersResponse response =
|
||||
containerManager.startContainers(requestList);
|
||||
|
||||
Assert.assertEquals(5, response.getSuccessfullyStartedContainers().size());
|
||||
for (ContainerId id : response.getSuccessfullyStartedContainers()) {
|
||||
// Containers with odd id should succeed.
|
||||
Assert.assertEquals(1, id.getId() & 1);
|
||||
}
|
||||
Assert.assertEquals(5, response.getFailedRequests().size());
|
||||
for (Map.Entry<ContainerId, SerializedException> entry : response
|
||||
.getFailedRequests().entrySet()) {
|
||||
// Containers with even id should fail.
|
||||
Assert.assertEquals(0, entry.getKey().getId() & 1);
|
||||
Assert.assertTrue(entry.getValue().getMessage()
|
||||
.contains(
|
||||
"Container " + entry.getKey() + " rejected as it is allocated by a previous RM"));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultipleContainersStopAndGetStatus() throws Exception {
|
||||
containerManager.start();
|
||||
List<StartContainerRequest> startRequest =
|
||||
new ArrayList<StartContainerRequest>();
|
||||
ContainerLaunchContext containerLaunchContext =
|
||||
recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
||||
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
for (int i = 0; i < 10; i++) {
|
||||
ContainerId cId = createContainerId(i);
|
||||
String user = null;
|
||||
if ((i & 1) == 0) {
|
||||
// container with even id fail
|
||||
user = "Fail";
|
||||
} else {
|
||||
user = "Pass";
|
||||
}
|
||||
Token containerToken =
|
||||
createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(),
|
||||
user, context.getContainerTokenSecretManager());
|
||||
StartContainerRequest request =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
containerToken);
|
||||
startRequest.add(request);
|
||||
containerIds.add(cId);
|
||||
}
|
||||
// start containers
|
||||
StartContainersRequest requestList =
|
||||
StartContainersRequest.newInstance(startRequest);
|
||||
containerManager.startContainers(requestList);
|
||||
|
||||
// Get container statuses
|
||||
GetContainerStatusesRequest statusRequest =
|
||||
GetContainerStatusesRequest.newInstance(containerIds);
|
||||
GetContainerStatusesResponse statusResponse =
|
||||
containerManager.getContainerStatuses(statusRequest);
|
||||
Assert.assertEquals(5, statusResponse.getContainerStatuses().size());
|
||||
for (ContainerStatus status : statusResponse.getContainerStatuses()) {
|
||||
// Containers with odd id should succeed
|
||||
Assert.assertEquals(1, status.getContainerId().getId() & 1);
|
||||
}
|
||||
Assert.assertEquals(5, statusResponse.getFailedRequests().size());
|
||||
for (Map.Entry<ContainerId, SerializedException> entry : statusResponse
|
||||
.getFailedRequests().entrySet()) {
|
||||
// Containers with even id should fail.
|
||||
Assert.assertEquals(0, entry.getKey().getId() & 1);
|
||||
Assert.assertTrue(entry.getValue().getMessage()
|
||||
.contains("Reject this container"));
|
||||
}
|
||||
|
||||
// stop containers
|
||||
StopContainersRequest stopRequest =
|
||||
StopContainersRequest.newInstance(containerIds);
|
||||
StopContainersResponse stopResponse =
|
||||
containerManager.stopContainers(stopRequest);
|
||||
Assert.assertEquals(5, stopResponse.getSuccessfullyStoppedContainers()
|
||||
.size());
|
||||
for (ContainerId id : stopResponse.getSuccessfullyStoppedContainers()) {
|
||||
// Containers with odd id should succeed.
|
||||
Assert.assertEquals(1, id.getId() & 1);
|
||||
}
|
||||
Assert.assertEquals(5, stopResponse.getFailedRequests().size());
|
||||
for (Map.Entry<ContainerId, SerializedException> entry : stopResponse
|
||||
.getFailedRequests().entrySet()) {
|
||||
// Containers with even id should fail.
|
||||
Assert.assertEquals(0, entry.getKey().getId() & 1);
|
||||
Assert.assertTrue(entry.getValue().getMessage()
|
||||
.contains("Reject this container"));
|
||||
}
|
||||
}
|
||||
|
||||
public static Token createContainerToken(ContainerId cId, long rmIdentifier,
|
||||
NodeId nodeId, String user,
|
||||
NMContainerTokenSecretManager containerTokenSecretManager)
|
||||
|
|
|
@ -42,9 +42,10 @@ import org.apache.hadoop.util.Shell;
|
|||
import org.apache.hadoop.util.Shell.ExitCodeException;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -399,10 +400,14 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|||
// set up the rest of the container
|
||||
List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
|
||||
containerLaunchContext.setCommands(commands);
|
||||
StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
startRequest.setContainerToken(createContainerToken(cId));
|
||||
containerManager.startContainer(startRequest);
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
createContainerToken(cId));
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
containerManager.startContainers(allRequests);
|
||||
|
||||
int timeoutSecs = 0;
|
||||
while (!processStartFile.exists() && timeoutSecs++ < 20) {
|
||||
|
@ -465,18 +470,20 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|||
Assert.assertTrue("Process is not alive!",
|
||||
DefaultContainerExecutor.containerIsAlive(pid));
|
||||
|
||||
StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
|
||||
stopRequest.setContainerId(cId);
|
||||
containerManager.stopContainer(stopRequest);
|
||||
// Now test the stop functionality.
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(cId);
|
||||
StopContainersRequest stopRequest =
|
||||
StopContainersRequest.newInstance(containerIds);
|
||||
containerManager.stopContainers(stopRequest);
|
||||
|
||||
BaseContainerManagerTest.waitForContainerState(containerManager, cId,
|
||||
ContainerState.COMPLETE);
|
||||
|
||||
GetContainerStatusRequest gcsRequest =
|
||||
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
|
||||
gcsRequest.setContainerId(cId);
|
||||
GetContainerStatusesRequest gcsRequest =
|
||||
GetContainerStatusesRequest.newInstance(containerIds);
|
||||
ContainerStatus containerStatus =
|
||||
containerManager.getContainerStatus(gcsRequest).getStatus();
|
||||
containerManager.getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
|
||||
int expectedExitCode = Shell.WINDOWS ? ExitCode.FORCE_KILLED.getExitCode() :
|
||||
ExitCode.TERMINATED.getExitCode();
|
||||
Assert.assertEquals(expectedExitCode, containerStatus.getExitStatus());
|
||||
|
@ -544,11 +551,15 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|||
List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
|
||||
containerLaunchContext.setCommands(commands);
|
||||
Token containerToken = createContainerToken(cId);
|
||||
StartContainerRequest startRequest =
|
||||
recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
startRequest.setContainerToken(containerToken);
|
||||
containerManager.startContainer(startRequest);
|
||||
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
containerToken);
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
containerManager.startContainers(allRequests);
|
||||
|
||||
int timeoutSecs = 0;
|
||||
while (!processStartFile.exists() && timeoutSecs++ < 20) {
|
||||
|
@ -559,19 +570,22 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|||
processStartFile.exists());
|
||||
|
||||
// Now test the stop functionality.
|
||||
StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
|
||||
stopRequest.setContainerId(cId);
|
||||
containerManager.stopContainer(stopRequest);
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(cId);
|
||||
StopContainersRequest stopRequest =
|
||||
StopContainersRequest.newInstance(containerIds);
|
||||
containerManager.stopContainers(stopRequest);
|
||||
|
||||
BaseContainerManagerTest.waitForContainerState(containerManager, cId,
|
||||
ContainerState.COMPLETE);
|
||||
|
||||
// container stop sends a sigterm followed by a sigkill
|
||||
GetContainerStatusRequest gcsRequest =
|
||||
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
|
||||
gcsRequest.setContainerId(cId);
|
||||
GetContainerStatusesRequest gcsRequest =
|
||||
GetContainerStatusesRequest.newInstance(containerIds);
|
||||
|
||||
ContainerStatus containerStatus =
|
||||
containerManager.getContainerStatus(gcsRequest).getStatus();
|
||||
containerManager.getContainerStatuses(gcsRequest)
|
||||
.getContainerStatuses().get(0);
|
||||
Assert.assertEquals(ExitCode.FORCE_KILLED.getExitCode(),
|
||||
containerStatus.getExitStatus());
|
||||
|
||||
|
|
|
@ -18,8 +18,8 @@
|
|||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.api.records.URL;
|
||||
import org.apache.hadoop.yarn.server.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType;
|
||||
|
||||
|
|
|
@ -80,6 +80,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
|
|||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResourceType;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.api.records.URL;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
||||
|
@ -1013,8 +1014,8 @@ public class TestResourceLocalizationService {
|
|||
String localizerId, LocalResourceRequest req) {
|
||||
LocalizerStatus status = createLocalizerStatus(localizerId);
|
||||
LocalResourceStatus resourceStatus = new LocalResourceStatusPBImpl();
|
||||
resourceStatus.setException(YarnServerBuilderUtils
|
||||
.newSerializedException(new YarnException("test")));
|
||||
resourceStatus.setException(SerializedException
|
||||
.newInstance(new YarnException("test")));
|
||||
resourceStatus.setStatus(ResourceStatusType.FETCH_FAILURE);
|
||||
resourceStatus.setResource(req);
|
||||
status.addResourceStatus(resourceStatus);
|
||||
|
|
|
@ -63,6 +63,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
|
|||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
|
@ -809,14 +810,18 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|||
commands.add("/bin/bash");
|
||||
commands.add(scriptFile.getAbsolutePath());
|
||||
containerLaunchContext.setCommands(commands);
|
||||
StartContainerRequest startRequest =
|
||||
recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
startRequest.setContainerToken(TestContainerManager.createContainerToken(
|
||||
cId, DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
|
||||
context.getContainerTokenSecretManager()));
|
||||
this.containerManager.startContainer(startRequest);
|
||||
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
TestContainerManager.createContainerToken(
|
||||
cId, DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
|
||||
context.getContainerTokenSecretManager()));
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
this.containerManager.startContainers(allRequests);
|
||||
|
||||
BaseContainerManagerTest.waitForContainerState(this.containerManager,
|
||||
cId, ContainerState.COMPLETE);
|
||||
|
||||
|
|
|
@ -40,8 +40,9 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -227,9 +228,6 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
|
|||
commands.add(scriptFile.getAbsolutePath());
|
||||
containerLaunchContext.setCommands(commands);
|
||||
Resource r = BuilderUtils.newResource(8 * 1024 * 1024, 1);
|
||||
StartContainerRequest startRequest =
|
||||
recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
ContainerTokenIdentifier containerIdentifier =
|
||||
new ContainerTokenIdentifier(cId, context.getNodeId().toString(), user,
|
||||
r, System.currentTimeMillis() + 120000, 123, DUMMY_RM_IDENTIFIER);
|
||||
|
@ -237,8 +235,14 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
|
|||
BuilderUtils.newContainerToken(context.getNodeId(),
|
||||
containerManager.getContext().getContainerTokenSecretManager()
|
||||
.createPassword(containerIdentifier), containerIdentifier);
|
||||
startRequest.setContainerToken(containerToken);
|
||||
containerManager.startContainer(startRequest);
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(containerLaunchContext,
|
||||
containerToken);
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
containerManager.startContainers(allRequests);
|
||||
|
||||
int timeoutSecs = 0;
|
||||
while (!processStartFile.exists() && timeoutSecs++ < 20) {
|
||||
|
@ -260,11 +264,12 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
|
|||
BaseContainerManagerTest.waitForContainerState(containerManager, cId,
|
||||
ContainerState.COMPLETE, 60);
|
||||
|
||||
GetContainerStatusRequest gcsRequest =
|
||||
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
|
||||
gcsRequest.setContainerId(cId);
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(cId);
|
||||
GetContainerStatusesRequest gcsRequest =
|
||||
GetContainerStatusesRequest.newInstance(containerIds);
|
||||
ContainerStatus containerStatus =
|
||||
containerManager.getContainerStatus(gcsRequest).getStatus();
|
||||
containerManager.getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
|
||||
Assert.assertEquals(ExitCode.TERMINATED.getExitCode(),
|
||||
containerStatus.getExitStatus());
|
||||
String expectedMsgPattern =
|
||||
|
|
|
@ -22,6 +22,8 @@ import java.io.IOException;
|
|||
import java.net.InetSocketAddress;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.PrivilegedAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -33,11 +35,15 @@ import org.apache.hadoop.net.NetUtils;
|
|||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.yarn.api.ApplicationConstants;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||
import org.apache.hadoop.yarn.api.records.Container;
|
||||
|
@ -46,8 +52,6 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
||||
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
||||
|
@ -68,8 +72,6 @@ public class AMLauncher implements Runnable {
|
|||
|
||||
private final RMAppAttempt application;
|
||||
private final Configuration conf;
|
||||
private final RecordFactory recordFactory =
|
||||
RecordFactoryProvider.getRecordFactory(null);
|
||||
private final AMLauncherEventType eventType;
|
||||
private final RMContext rmContext;
|
||||
private final Container masterContainer;
|
||||
|
@ -102,22 +104,42 @@ public class AMLauncher implements Runnable {
|
|||
+ " for AM " + application.getAppAttemptId());
|
||||
ContainerLaunchContext launchContext =
|
||||
createAMContainerLaunchContext(applicationContext, masterContainerID);
|
||||
StartContainerRequest request =
|
||||
recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
request.setContainerLaunchContext(launchContext);
|
||||
request.setContainerToken(masterContainer.getContainerToken());
|
||||
containerMgrProxy.startContainer(request);
|
||||
LOG.info("Done launching container " + masterContainer
|
||||
+ " for AM " + application.getAppAttemptId());
|
||||
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(launchContext,
|
||||
masterContainer.getContainerToken());
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
|
||||
StartContainersResponse response =
|
||||
containerMgrProxy.startContainers(allRequests);
|
||||
if (response.getFailedRequests() != null
|
||||
&& response.getFailedRequests().containsKey(masterContainerID)) {
|
||||
Throwable t =
|
||||
response.getFailedRequests().get(masterContainerID).deSerialize();
|
||||
parseAndThrowException(t);
|
||||
} else {
|
||||
LOG.info("Done launching container " + masterContainer + " for AM "
|
||||
+ application.getAppAttemptId());
|
||||
}
|
||||
}
|
||||
|
||||
private void cleanup() throws IOException, YarnException {
|
||||
connect();
|
||||
ContainerId containerId = masterContainer.getId();
|
||||
StopContainerRequest stopRequest =
|
||||
recordFactory.newRecordInstance(StopContainerRequest.class);
|
||||
stopRequest.setContainerId(containerId);
|
||||
containerMgrProxy.stopContainer(stopRequest);
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(containerId);
|
||||
StopContainersRequest stopRequest =
|
||||
StopContainersRequest.newInstance(containerIds);
|
||||
StopContainersResponse response =
|
||||
containerMgrProxy.stopContainers(stopRequest);
|
||||
if (response.getFailedRequests() != null
|
||||
&& response.getFailedRequests().containsKey(containerId)) {
|
||||
Throwable t = response.getFailedRequests().get(containerId).deSerialize();
|
||||
parseAndThrowException(t);
|
||||
}
|
||||
}
|
||||
|
||||
// Protected. For tests.
|
||||
|
@ -254,4 +276,15 @@ public class AMLauncher implements Runnable {
|
|||
break;
|
||||
}
|
||||
}
|
||||
|
||||
private void parseAndThrowException(Throwable t) throws YarnException,
|
||||
IOException {
|
||||
if (t instanceof YarnException) {
|
||||
throw (YarnException) t;
|
||||
} else if (t instanceof InvalidToken) {
|
||||
throw (InvalidToken) t;
|
||||
} else {
|
||||
throw (IOException) t;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,7 +35,8 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.net.NetworkTopology;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
|
@ -212,9 +213,11 @@ public class Application {
|
|||
NodeManager nodeManager = task.getNodeManager();
|
||||
ContainerId containerId = task.getContainerId();
|
||||
task.stop();
|
||||
StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
|
||||
stopRequest.setContainerId(containerId);
|
||||
nodeManager.stopContainer(stopRequest);
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(containerId);
|
||||
StopContainersRequest stopRequest =
|
||||
StopContainersRequest.newInstance(containerIds);
|
||||
nodeManager.stopContainers(stopRequest);
|
||||
|
||||
Resources.subtractFrom(used, requestSpec.get(task.getPriority()));
|
||||
|
||||
|
@ -339,10 +342,15 @@ public class Application {
|
|||
updateResourceRequests(requests.get(priority), type, task);
|
||||
|
||||
// Launch the container
|
||||
StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(createCLC());
|
||||
startRequest.setContainerToken(container.getContainerToken());
|
||||
nodeManager.startContainer(startRequest);
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(createCLC(),
|
||||
container.getContainerToken());
|
||||
List<StartContainerRequest> list =
|
||||
new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
nodeManager.startContainers(allRequests);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,12 +31,13 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.Container;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -158,61 +159,60 @@ public class NodeManager implements ContainerManagementProtocol {
|
|||
}
|
||||
|
||||
@Override
|
||||
synchronized public StartContainerResponse startContainer(
|
||||
StartContainerRequest request)
|
||||
synchronized public StartContainersResponse startContainers(
|
||||
StartContainersRequest requests)
|
||||
throws YarnException {
|
||||
|
||||
Token containerToken = request.getContainerToken();
|
||||
ContainerTokenIdentifier tokenId = null;
|
||||
for (StartContainerRequest request : requests.getStartContainerRequests()) {
|
||||
Token containerToken = request.getContainerToken();
|
||||
ContainerTokenIdentifier tokenId = null;
|
||||
|
||||
try {
|
||||
tokenId = BuilderUtils.newContainerTokenIdentifier(containerToken);
|
||||
} catch (IOException e) {
|
||||
throw RPCUtil.getRemoteException(e);
|
||||
}
|
||||
|
||||
ContainerId containerID = tokenId.getContainerID();
|
||||
ApplicationId applicationId =
|
||||
containerID.getApplicationAttemptId().getApplicationId();
|
||||
|
||||
List<Container> applicationContainers = containers.get(applicationId);
|
||||
if (applicationContainers == null) {
|
||||
applicationContainers = new ArrayList<Container>();
|
||||
containers.put(applicationId, applicationContainers);
|
||||
}
|
||||
|
||||
// Sanity check
|
||||
for (Container container : applicationContainers) {
|
||||
if (container.getId().compareTo(containerID)
|
||||
== 0) {
|
||||
throw new IllegalStateException(
|
||||
"Container " + containerID +
|
||||
" already setup on node " + containerManagerAddress);
|
||||
try {
|
||||
tokenId = BuilderUtils.newContainerTokenIdentifier(containerToken);
|
||||
} catch (IOException e) {
|
||||
throw RPCUtil.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
Container container =
|
||||
BuilderUtils.newContainer(containerID,
|
||||
this.nodeId, nodeHttpAddress,
|
||||
tokenId.getResource(),
|
||||
null, null // DKDC - Doesn't matter
|
||||
ContainerId containerID = tokenId.getContainerID();
|
||||
ApplicationId applicationId =
|
||||
containerID.getApplicationAttemptId().getApplicationId();
|
||||
|
||||
List<Container> applicationContainers = containers.get(applicationId);
|
||||
if (applicationContainers == null) {
|
||||
applicationContainers = new ArrayList<Container>();
|
||||
containers.put(applicationId, applicationContainers);
|
||||
}
|
||||
|
||||
// Sanity check
|
||||
for (Container container : applicationContainers) {
|
||||
if (container.getId().compareTo(containerID) == 0) {
|
||||
throw new IllegalStateException("Container " + containerID
|
||||
+ " already setup on node " + containerManagerAddress);
|
||||
}
|
||||
}
|
||||
|
||||
Container container =
|
||||
BuilderUtils.newContainer(containerID, this.nodeId, nodeHttpAddress,
|
||||
tokenId.getResource(), null, null // DKDC - Doesn't matter
|
||||
);
|
||||
|
||||
ContainerStatus containerStatus =
|
||||
BuilderUtils.newContainerStatus(container.getId(), ContainerState.NEW,
|
||||
"", -1000);
|
||||
applicationContainers.add(container);
|
||||
containerStatusMap.put(container, containerStatus);
|
||||
Resources.subtractFrom(available, tokenId.getResource());
|
||||
Resources.addTo(used, tokenId.getResource());
|
||||
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("startContainer:" + " node=" + containerManagerAddress
|
||||
+ " application=" + applicationId + " container=" + container
|
||||
+ " available=" + available + " used=" + used);
|
||||
}
|
||||
ContainerStatus containerStatus =
|
||||
BuilderUtils.newContainerStatus(container.getId(),
|
||||
ContainerState.NEW, "", -1000);
|
||||
applicationContainers.add(container);
|
||||
containerStatusMap.put(container, containerStatus);
|
||||
Resources.subtractFrom(available, tokenId.getResource());
|
||||
Resources.addTo(used, tokenId.getResource());
|
||||
|
||||
StartContainerResponse response = recordFactory.newRecordInstance(StartContainerResponse.class);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("startContainer:" + " node=" + containerManagerAddress
|
||||
+ " application=" + applicationId + " container=" + container
|
||||
+ " available=" + available + " used=" + used);
|
||||
}
|
||||
|
||||
}
|
||||
StartContainersResponse response =
|
||||
StartContainersResponse.newInstance(null, null, null);
|
||||
return response;
|
||||
}
|
||||
|
||||
|
@ -225,76 +225,79 @@ public class NodeManager implements ContainerManagementProtocol {
|
|||
}
|
||||
|
||||
@Override
|
||||
synchronized public StopContainerResponse stopContainer(StopContainerRequest request)
|
||||
synchronized public StopContainersResponse stopContainers(StopContainersRequest request)
|
||||
throws YarnException {
|
||||
ContainerId containerID = request.getContainerId();
|
||||
String applicationId = String.valueOf(
|
||||
containerID.getApplicationAttemptId().getApplicationId().getId());
|
||||
|
||||
// Mark the container as COMPLETE
|
||||
List<Container> applicationContainers = containers.get(applicationId);
|
||||
for (Container c : applicationContainers) {
|
||||
if (c.getId().compareTo(containerID) == 0) {
|
||||
ContainerStatus containerStatus = containerStatusMap.get(c);
|
||||
containerStatus.setState(ContainerState.COMPLETE);
|
||||
containerStatusMap.put(c, containerStatus);
|
||||
for (ContainerId containerID : request.getContainerIds()) {
|
||||
String applicationId =
|
||||
String.valueOf(containerID.getApplicationAttemptId()
|
||||
.getApplicationId().getId());
|
||||
|
||||
// Mark the container as COMPLETE
|
||||
List<Container> applicationContainers = containers.get(applicationId);
|
||||
for (Container c : applicationContainers) {
|
||||
if (c.getId().compareTo(containerID) == 0) {
|
||||
ContainerStatus containerStatus = containerStatusMap.get(c);
|
||||
containerStatus.setState(ContainerState.COMPLETE);
|
||||
containerStatusMap.put(c, containerStatus);
|
||||
}
|
||||
}
|
||||
|
||||
// Send a heartbeat
|
||||
try {
|
||||
heartbeat();
|
||||
} catch (IOException ioe) {
|
||||
throw RPCUtil.getRemoteException(ioe);
|
||||
}
|
||||
|
||||
// Remove container and update status
|
||||
int ctr = 0;
|
||||
Container container = null;
|
||||
for (Iterator<Container> i = applicationContainers.iterator(); i
|
||||
.hasNext();) {
|
||||
container = i.next();
|
||||
if (container.getId().compareTo(containerID) == 0) {
|
||||
i.remove();
|
||||
++ctr;
|
||||
}
|
||||
}
|
||||
|
||||
if (ctr != 1) {
|
||||
throw new IllegalStateException("Container " + containerID
|
||||
+ " stopped " + ctr + " times!");
|
||||
}
|
||||
|
||||
Resources.addTo(available, container.getResource());
|
||||
Resources.subtractFrom(used, container.getResource());
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("stopContainer:" + " node=" + containerManagerAddress
|
||||
+ " application=" + applicationId + " container=" + containerID
|
||||
+ " available=" + available + " used=" + used);
|
||||
}
|
||||
}
|
||||
|
||||
// Send a heartbeat
|
||||
try {
|
||||
heartbeat();
|
||||
} catch (IOException ioe) {
|
||||
throw RPCUtil.getRemoteException(ioe);
|
||||
}
|
||||
|
||||
// Remove container and update status
|
||||
int ctr = 0;
|
||||
Container container = null;
|
||||
for (Iterator<Container> i=applicationContainers.iterator(); i.hasNext();) {
|
||||
container = i.next();
|
||||
if (container.getId().compareTo(containerID) == 0) {
|
||||
i.remove();
|
||||
++ctr;
|
||||
}
|
||||
}
|
||||
|
||||
if (ctr != 1) {
|
||||
throw new IllegalStateException("Container " + containerID +
|
||||
" stopped " + ctr + " times!");
|
||||
}
|
||||
|
||||
Resources.addTo(available, container.getResource());
|
||||
Resources.subtractFrom(used, container.getResource());
|
||||
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("stopContainer:" + " node=" + containerManagerAddress
|
||||
+ " application=" + applicationId + " container=" + containerID
|
||||
+ " available=" + available + " used=" + used);
|
||||
}
|
||||
|
||||
StopContainerResponse response = recordFactory.newRecordInstance(StopContainerResponse.class);
|
||||
return response;
|
||||
return StopContainersResponse.newInstance(null,null);
|
||||
}
|
||||
|
||||
@Override
|
||||
synchronized public GetContainerStatusResponse getContainerStatus(GetContainerStatusRequest request) throws YarnException {
|
||||
ContainerId containerId = request.getContainerId();
|
||||
List<Container> appContainers =
|
||||
containers.get(
|
||||
containerId.getApplicationAttemptId().getApplicationId());
|
||||
Container container = null;
|
||||
for (Container c : appContainers) {
|
||||
if (c.getId().equals(containerId)) {
|
||||
container = c;
|
||||
synchronized public GetContainerStatusesResponse getContainerStatuses(
|
||||
GetContainerStatusesRequest request) throws YarnException {
|
||||
List<ContainerStatus> statuses = new ArrayList<ContainerStatus>();
|
||||
for (ContainerId containerId : request.getContainerIds()) {
|
||||
List<Container> appContainers =
|
||||
containers.get(containerId.getApplicationAttemptId()
|
||||
.getApplicationId());
|
||||
Container container = null;
|
||||
for (Container c : appContainers) {
|
||||
if (c.getId().equals(containerId)) {
|
||||
container = c;
|
||||
}
|
||||
}
|
||||
if (container != null
|
||||
&& containerStatusMap.get(container).getState() != null) {
|
||||
statuses.add(containerStatusMap.get(container));
|
||||
}
|
||||
}
|
||||
GetContainerStatusResponse response =
|
||||
recordFactory.newRecordInstance(GetContainerStatusResponse.class);
|
||||
if (container != null && containerStatusMap.get(container).getState() != null) {
|
||||
response.setStatus(containerStatusMap.get(container));
|
||||
}
|
||||
return response;
|
||||
return GetContainerStatusesResponse.newInstance(statuses, null);
|
||||
}
|
||||
|
||||
public static org.apache.hadoop.yarn.server.api.records.NodeStatus
|
||||
|
|
|
@ -36,14 +36,14 @@ import org.apache.hadoop.security.Credentials;
|
|||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
|
@ -98,23 +98,23 @@ public class TestAMAuthorization {
|
|||
}
|
||||
|
||||
@Override
|
||||
public StartContainerResponse
|
||||
startContainer(StartContainerRequest request)
|
||||
public StartContainersResponse
|
||||
startContainers(StartContainersRequest request)
|
||||
throws YarnException {
|
||||
containerTokens = request.getContainerLaunchContext().getTokens();
|
||||
return null;
|
||||
containerTokens = request.getStartContainerRequests().get(0).getContainerLaunchContext().getTokens();
|
||||
return StartContainersResponse.newInstance(null, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StopContainerResponse stopContainer(StopContainerRequest request)
|
||||
public StopContainersResponse stopContainers(StopContainersRequest request)
|
||||
throws YarnException {
|
||||
return null;
|
||||
return StopContainersResponse.newInstance(null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetContainerStatusResponse getContainerStatus(
|
||||
GetContainerStatusRequest request) throws YarnException {
|
||||
return null;
|
||||
public GetContainerStatusesResponse getContainerStatuses(
|
||||
GetContainerStatusesRequest request) throws YarnException {
|
||||
return GetContainerStatusesResponse.newInstance(null, null);
|
||||
}
|
||||
|
||||
public Credentials getContainerCredentials() throws IOException {
|
||||
|
|
|
@ -19,7 +19,10 @@
|
|||
package org.apache.hadoop.yarn.server.resourcemanager;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -27,16 +30,18 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.yarn.api.ApplicationConstants;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerState;
|
||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.api.records.Token;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
|
@ -69,9 +74,10 @@ public class TestApplicationMasterLauncher {
|
|||
int maxAppAttempts;
|
||||
|
||||
@Override
|
||||
public StartContainerResponse
|
||||
startContainer(StartContainerRequest request)
|
||||
public StartContainersResponse
|
||||
startContainers(StartContainersRequest requests)
|
||||
throws YarnException {
|
||||
StartContainerRequest request = requests.getStartContainerRequests().get(0);
|
||||
LOG.info("Container started by MyContainerManager: " + request);
|
||||
launched = true;
|
||||
Map<String, String> env =
|
||||
|
@ -95,11 +101,13 @@ public class TestApplicationMasterLauncher {
|
|||
Long.parseLong(env.get(ApplicationConstants.APP_SUBMIT_TIME_ENV));
|
||||
maxAppAttempts =
|
||||
Integer.parseInt(env.get(ApplicationConstants.MAX_APP_ATTEMPTS_ENV));
|
||||
return null;
|
||||
return StartContainersResponse.newInstance(
|
||||
new HashMap<String, ByteBuffer>(), new ArrayList<ContainerId>(),
|
||||
new HashMap<ContainerId, SerializedException>());
|
||||
}
|
||||
|
||||
@Override
|
||||
public StopContainerResponse stopContainer(StopContainerRequest request)
|
||||
public StopContainersResponse stopContainers(StopContainersRequest request)
|
||||
throws YarnException {
|
||||
LOG.info("Container cleaned up by MyContainerManager");
|
||||
cleanedup = true;
|
||||
|
@ -107,11 +115,10 @@ public class TestApplicationMasterLauncher {
|
|||
}
|
||||
|
||||
@Override
|
||||
public GetContainerStatusResponse getContainerStatus(
|
||||
GetContainerStatusRequest request) throws YarnException {
|
||||
public GetContainerStatusesResponse getContainerStatuses(
|
||||
GetContainerStatusesRequest request) throws YarnException {
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -19,7 +19,9 @@
|
|||
package org.apache.hadoop.yarn.server.resourcemanager.security;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.annotation.Annotation;
|
||||
|
@ -50,6 +52,8 @@ import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
|||
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
|
@ -158,6 +162,9 @@ public class TestClientToAMTokens {
|
|||
|
||||
ContainerManagementProtocol containerManager =
|
||||
mock(ContainerManagementProtocol.class);
|
||||
StartContainersResponse mockResponse = mock(StartContainersResponse.class);
|
||||
when(containerManager.startContainers((StartContainersRequest) any()))
|
||||
.thenReturn(mockResponse);
|
||||
final DrainDispatcher dispatcher = new DrainDispatcher();
|
||||
|
||||
MockRM rm = new MockRMWithCustomAMLauncher(conf, containerManager) {
|
||||
|
|
|
@ -23,6 +23,8 @@ import static org.junit.Assert.fail;
|
|||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.security.PrivilegedAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
|
@ -32,15 +34,20 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.api.records.SerializedException;
|
||||
import org.apache.hadoop.yarn.api.records.Token;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
|
@ -328,18 +335,21 @@ public class TestContainerManagerSecurity {
|
|||
ContainerId containerId,
|
||||
ApplicationAttemptId appAttemptId, NodeId nodeId,
|
||||
boolean isExceptionExpected) throws Exception {
|
||||
GetContainerStatusRequest request =
|
||||
Records.newRecord(GetContainerStatusRequest.class);
|
||||
request.setContainerId(containerId);
|
||||
|
||||
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
||||
containerIds.add(containerId);
|
||||
GetContainerStatusesRequest request =
|
||||
GetContainerStatusesRequest.newInstance(containerIds);
|
||||
ContainerManagementProtocol proxy = null;
|
||||
|
||||
try {
|
||||
proxy =
|
||||
getContainerManagementProtocolProxy(rpc, nmToken, nodeId,
|
||||
appAttemptId.toString());
|
||||
proxy.getContainerStatus(request);
|
||||
|
||||
GetContainerStatusesResponse statuses = proxy.getContainerStatuses(request);
|
||||
if (statuses.getFailedRequests() != null
|
||||
&& statuses.getFailedRequests().containsKey(containerId)) {
|
||||
parseAndThrowException(statuses.getFailedRequests().get(containerId)
|
||||
.deSerialize());
|
||||
}
|
||||
} finally {
|
||||
if (proxy != null) {
|
||||
rpc.stopProxy(proxy, conf);
|
||||
|
@ -352,17 +362,21 @@ public class TestContainerManagerSecurity {
|
|||
org.apache.hadoop.yarn.api.records.Token containerToken,
|
||||
NodeId nodeId, String user) throws Exception {
|
||||
|
||||
StartContainerRequest request =
|
||||
Records.newRecord(StartContainerRequest.class);
|
||||
request.setContainerToken(containerToken);
|
||||
ContainerLaunchContext context =
|
||||
Records.newRecord(ContainerLaunchContext.class);
|
||||
request.setContainerLaunchContext(context);
|
||||
|
||||
StartContainerRequest scRequest =
|
||||
StartContainerRequest.newInstance(context,containerToken);
|
||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
||||
list.add(scRequest);
|
||||
StartContainersRequest allRequests =
|
||||
StartContainersRequest.newInstance(list);
|
||||
ContainerManagementProtocol proxy = null;
|
||||
try {
|
||||
proxy = getContainerManagementProtocolProxy(rpc, nmToken, nodeId, user);
|
||||
proxy.startContainer(request);
|
||||
StartContainersResponse response = proxy.startContainers(allRequests);
|
||||
for(SerializedException ex : response.getFailedRequests().values()){
|
||||
parseAndThrowException(ex.deSerialize());
|
||||
}
|
||||
} finally {
|
||||
if (proxy != null) {
|
||||
rpc.stopProxy(proxy, conf);
|
||||
|
@ -370,6 +384,17 @@ public class TestContainerManagerSecurity {
|
|||
}
|
||||
}
|
||||
|
||||
private void parseAndThrowException(Throwable t) throws YarnException,
|
||||
IOException {
|
||||
if (t instanceof YarnException) {
|
||||
throw (YarnException) t;
|
||||
} else if (t instanceof InvalidToken) {
|
||||
throw (InvalidToken) t;
|
||||
} else {
|
||||
throw (IOException) t;
|
||||
}
|
||||
}
|
||||
|
||||
protected ContainerManagementProtocol getContainerManagementProtocolProxy(
|
||||
final YarnRPC rpc, org.apache.hadoop.yarn.api.records.Token nmToken,
|
||||
NodeId nodeId, String user) {
|
||||
|
|
Loading…
Reference in New Issue