|
@@ -61,6 +61,7 @@ import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
|
|
|
import org.apache.hadoop.service.Service.STATE;
|
|
|
import org.apache.hadoop.service.ServiceOperations;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
@@ -69,6 +70,7 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
+import org.apache.hadoop.yarn.api.records.SignalContainerCommand;
|
|
|
import org.apache.hadoop.yarn.api.records.Token;
|
|
|
import org.apache.hadoop.yarn.client.RMProxy;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
@@ -166,9 +168,11 @@ public class TestNodeStatusUpdater {
|
|
|
private class MyResourceTracker implements ResourceTracker {
|
|
|
|
|
|
private final Context context;
|
|
|
+ private boolean signalContainer;
|
|
|
|
|
|
- public MyResourceTracker(Context context) {
|
|
|
+ public MyResourceTracker(Context context, boolean signalContainer) {
|
|
|
this.context = context;
|
|
|
+ this.signalContainer = signalContainer;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -222,17 +226,19 @@ public class TestNodeStatusUpdater {
|
|
|
nodeStatus.setResponseId(heartBeatID++);
|
|
|
Map<ApplicationId, List<ContainerStatus>> appToContainers =
|
|
|
getAppToContainerStatusMap(nodeStatus.getContainersStatuses());
|
|
|
+ List<SignalContainerRequest> containersToSignal = null;
|
|
|
|
|
|
ApplicationId appId1 = ApplicationId.newInstance(0, 1);
|
|
|
ApplicationId appId2 = ApplicationId.newInstance(0, 2);
|
|
|
|
|
|
+ ContainerId firstContainerID = null;
|
|
|
if (heartBeatID == 1) {
|
|
|
Assert.assertEquals(0, nodeStatus.getContainersStatuses().size());
|
|
|
|
|
|
// Give a container to the NM.
|
|
|
ApplicationAttemptId appAttemptID =
|
|
|
ApplicationAttemptId.newInstance(appId1, 0);
|
|
|
- ContainerId firstContainerID =
|
|
|
+ firstContainerID =
|
|
|
ContainerId.newContainerId(appAttemptID, heartBeatID);
|
|
|
ContainerLaunchContext launchContext = recordFactory
|
|
|
.newRecordInstance(ContainerLaunchContext.class);
|
|
@@ -259,6 +265,15 @@ public class TestNodeStatusUpdater {
|
|
|
this.context.getContainers();
|
|
|
Assert.assertEquals(1, activeContainers.size());
|
|
|
|
|
|
+ if (this.signalContainer) {
|
|
|
+ containersToSignal = new ArrayList<SignalContainerRequest>();
|
|
|
+ SignalContainerRequest signalReq = recordFactory
|
|
|
+ .newRecordInstance(SignalContainerRequest.class);
|
|
|
+ signalReq.setContainerId(firstContainerID);
|
|
|
+ signalReq.setCommand(SignalContainerCommand.OUTPUT_THREAD_DUMP);
|
|
|
+ containersToSignal.add(signalReq);
|
|
|
+ }
|
|
|
+
|
|
|
// Give another container to the NM.
|
|
|
ApplicationAttemptId appAttemptID =
|
|
|
ApplicationAttemptId.newInstance(appId2, 0);
|
|
@@ -295,6 +310,9 @@ public class TestNodeStatusUpdater {
|
|
|
NodeHeartbeatResponse nhResponse = YarnServerBuilderUtils.
|
|
|
newNodeHeartbeatResponse(heartBeatID, null, null, null, null, null,
|
|
|
1000L);
|
|
|
+ if (containersToSignal != null) {
|
|
|
+ nhResponse.addAllContainersToSignal(containersToSignal);
|
|
|
+ }
|
|
|
return nhResponse;
|
|
|
}
|
|
|
|
|
@@ -306,15 +324,40 @@ public class TestNodeStatusUpdater {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private class MyContainerManager extends ContainerManagerImpl {
|
|
|
+ public boolean signaled = false;
|
|
|
+
|
|
|
+ public MyContainerManager(Context context, ContainerExecutor exec,
|
|
|
+ DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater,
|
|
|
+ NodeManagerMetrics metrics,
|
|
|
+ LocalDirsHandlerService dirsHandler) {
|
|
|
+ super(context, exec, deletionContext, nodeStatusUpdater,
|
|
|
+ metrics, dirsHandler);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void handle(ContainerManagerEvent event) {
|
|
|
+ if (event.getType() == ContainerManagerEventType.SIGNAL_CONTAINERS) {
|
|
|
+ signaled = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private class MyNodeStatusUpdater extends NodeStatusUpdaterImpl {
|
|
|
public ResourceTracker resourceTracker;
|
|
|
private Context context;
|
|
|
|
|
|
public MyNodeStatusUpdater(Context context, Dispatcher dispatcher,
|
|
|
NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
|
|
|
+ this(context, dispatcher, healthChecker, metrics, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ public MyNodeStatusUpdater(Context context, Dispatcher dispatcher,
|
|
|
+ NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics,
|
|
|
+ boolean signalContainer) {
|
|
|
super(context, dispatcher, healthChecker, metrics);
|
|
|
this.context = context;
|
|
|
- resourceTracker = new MyResourceTracker(this.context);
|
|
|
+ resourceTracker = new MyResourceTracker(this.context, signalContainer);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -1547,6 +1590,66 @@ public class TestNodeStatusUpdater {
|
|
|
nm.stop();
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+ //Verify that signalContainer request can be dispatched from
|
|
|
+ //NodeStatusUpdaterImpl to ContainerManagerImpl.
|
|
|
+ @Test
|
|
|
+ public void testSignalContainerToContainerManager() throws Exception {
|
|
|
+ nm = new NodeManager() {
|
|
|
+ @Override
|
|
|
+ protected NodeStatusUpdater createNodeStatusUpdater(Context context,
|
|
|
+ Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
|
|
|
+ return new MyNodeStatusUpdater(
|
|
|
+ context, dispatcher, healthChecker, metrics, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected ContainerManagerImpl createContainerManager(Context context,
|
|
|
+ ContainerExecutor exec, DeletionService del,
|
|
|
+ NodeStatusUpdater nodeStatusUpdater,
|
|
|
+ ApplicationACLsManager aclsManager,
|
|
|
+ LocalDirsHandlerService diskhandler) {
|
|
|
+ return new MyContainerManager(context, exec, del, nodeStatusUpdater,
|
|
|
+ metrics, diskhandler);
|
|
|
+ }
|
|
|
+ };
|
|
|
+
|
|
|
+ YarnConfiguration conf = createNMConfig();
|
|
|
+ nm.init(conf);
|
|
|
+ nm.start();
|
|
|
+
|
|
|
+ System.out.println(" ----- thread already started.."
|
|
|
+ + nm.getServiceState());
|
|
|
+
|
|
|
+ int waitCount = 0;
|
|
|
+ while (nm.getServiceState() == STATE.INITED && waitCount++ != 20) {
|
|
|
+ LOG.info("Waiting for NM to start..");
|
|
|
+ if (nmStartError != null) {
|
|
|
+ LOG.error("Error during startup. ", nmStartError);
|
|
|
+ Assert.fail(nmStartError.getCause().getMessage());
|
|
|
+ }
|
|
|
+ Thread.sleep(1000);
|
|
|
+ }
|
|
|
+ if (nm.getServiceState() != STATE.STARTED) {
|
|
|
+ // NM could have failed.
|
|
|
+ Assert.fail("NodeManager failed to start");
|
|
|
+ }
|
|
|
+
|
|
|
+ waitCount = 0;
|
|
|
+ while (heartBeatID <= 3 && waitCount++ != 20) {
|
|
|
+ Thread.sleep(500);
|
|
|
+ }
|
|
|
+ Assert.assertFalse(heartBeatID <= 3);
|
|
|
+ Assert.assertEquals("Number of registered NMs is wrong!!", 1,
|
|
|
+ this.registeredNodes.size());
|
|
|
+
|
|
|
+ MyContainerManager containerManager =
|
|
|
+ (MyContainerManager)nm.getContainerManager();
|
|
|
+ Assert.assertTrue(containerManager.signaled);
|
|
|
+
|
|
|
+ nm.stop();
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testConcurrentAccessToSystemCredentials(){
|
|
|
final Map<ApplicationId, ByteBuffer> testCredentials = new HashMap<>();
|