|
@@ -18,21 +18,35 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.nodemanager;
|
|
package org.apache.hadoop.yarn.server.nodemanager;
|
|
|
|
|
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
import static org.mockito.Mockito.mock;
|
|
import static org.mockito.Mockito.mock;
|
|
import static org.mockito.Mockito.when;
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
import java.io.File;
|
|
import java.io.File;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
+import java.io.PrintWriter;
|
|
|
|
+import java.nio.ByteBuffer;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
|
|
+import java.util.Arrays;
|
|
|
|
+import java.util.HashMap;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.Map;
|
|
import java.util.concurrent.BrokenBarrierException;
|
|
import java.util.concurrent.BrokenBarrierException;
|
|
import java.util.concurrent.ConcurrentMap;
|
|
import java.util.concurrent.ConcurrentMap;
|
|
import java.util.concurrent.CyclicBarrier;
|
|
import java.util.concurrent.CyclicBarrier;
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
|
|
|
|
|
+import org.apache.commons.logging.Log;
|
|
|
|
+import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.fs.FileContext;
|
|
import org.apache.hadoop.fs.FileContext;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
|
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
|
|
|
+import org.apache.hadoop.security.UserGroupInformation;
|
|
|
|
+import org.apache.hadoop.security.token.SecretManager;
|
|
|
|
+import org.apache.hadoop.util.Shell;
|
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
|
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
@@ -41,8 +55,13 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
|
+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.Priority;
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.Token;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.URL;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
|
import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
|
|
import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
|
|
@@ -50,6 +69,8 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
|
|
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
|
|
|
+import org.apache.hadoop.yarn.security.NMTokenIdentifier;
|
|
import org.apache.hadoop.yarn.server.api.ResourceTracker;
|
|
import org.apache.hadoop.yarn.server.api.ResourceTracker;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
|
|
@@ -57,12 +78,15 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
|
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
|
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
|
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
|
|
import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
|
|
|
|
+import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
import org.junit.Assert;
|
|
import org.junit.Assert;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
@@ -87,7 +111,10 @@ public class TestNodeManagerResync {
|
|
private AtomicBoolean isNMShutdownCalled = new AtomicBoolean(false);
|
|
private AtomicBoolean isNMShutdownCalled = new AtomicBoolean(false);
|
|
private final NodeManagerEvent resyncEvent =
|
|
private final NodeManagerEvent resyncEvent =
|
|
new NodeManagerEvent(NodeManagerEventType.RESYNC);
|
|
new NodeManagerEvent(NodeManagerEventType.RESYNC);
|
|
|
|
+ private final long DUMMY_RM_IDENTIFIER = 1234;
|
|
|
|
|
|
|
|
+ protected static Log LOG = LogFactory
|
|
|
|
+ .getLog(TestNodeManagerResync.class);
|
|
|
|
|
|
@Before
|
|
@Before
|
|
public void setup() throws UnsupportedFileSystemException {
|
|
public void setup() throws UnsupportedFileSystemException {
|
|
@@ -209,6 +236,32 @@ public class TestNodeManagerResync {
|
|
nm.stop();
|
|
nm.stop();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
|
+ @Test(timeout=60000)
|
|
|
|
+ public void testContainerResourceIncreaseIsSynchronizedWithRMResync()
|
|
|
|
+ throws IOException, InterruptedException, YarnException {
|
|
|
|
+ NodeManager nm = new TestNodeManager4();
|
|
|
|
+ YarnConfiguration conf = createNMConfig();
|
|
|
|
+ conf.setBoolean(
|
|
|
|
+ YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
|
|
|
|
+ nm.init(conf);
|
|
|
|
+ nm.start();
|
|
|
|
+ // Start a container and make sure it is in RUNNING state
|
|
|
|
+ ((TestNodeManager4)nm).startContainer();
|
|
|
|
+ // Simulate a container resource increase in a separate thread
|
|
|
|
+ ((TestNodeManager4)nm).increaseContainersResource();
|
|
|
|
+ // Simulate RM restart by sending a RESYNC event
|
|
|
|
+ LOG.info("Sending out RESYNC event");
|
|
|
|
+ nm.getNMDispatcher().getEventHandler().handle(
|
|
|
|
+ new NodeManagerEvent(NodeManagerEventType.RESYNC));
|
|
|
|
+ try {
|
|
|
|
+ syncBarrier.await();
|
|
|
|
+ } catch (BrokenBarrierException e) {
|
|
|
|
+ e.printStackTrace();
|
|
|
|
+ }
|
|
|
|
+ Assert.assertFalse(assertionFailedInThread.get());
|
|
|
|
+ nm.stop();
|
|
|
|
+ }
|
|
|
|
|
|
// This is to test when NM gets the resync response from last heart beat, it
|
|
// This is to test when NM gets the resync response from last heart beat, it
|
|
// should be able to send the already-sent-via-last-heart-beat container
|
|
// should be able to send the already-sent-via-last-heart-beat container
|
|
@@ -588,6 +641,211 @@ public class TestNodeManagerResync {
|
|
}
|
|
}
|
|
}}
|
|
}}
|
|
|
|
|
|
|
|
+ class TestNodeManager4 extends NodeManager {
|
|
|
|
+
|
|
|
|
+ private Thread increaseContainerResourceThread = null;
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ protected NodeStatusUpdater createNodeStatusUpdater(Context context,
|
|
|
|
+ Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
|
|
|
|
+ return new TestNodeStatusUpdaterImpl4(context, dispatcher,
|
|
|
|
+ healthChecker, metrics);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ protected ContainerManagerImpl createContainerManager(Context context,
|
|
|
|
+ ContainerExecutor exec, DeletionService del,
|
|
|
|
+ NodeStatusUpdater nodeStatusUpdater,
|
|
|
|
+ ApplicationACLsManager aclsManager,
|
|
|
|
+ LocalDirsHandlerService dirsHandler) {
|
|
|
|
+ return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
|
|
|
|
+ metrics, dirsHandler){
|
|
|
|
+ @Override
|
|
|
|
+ public void
|
|
|
|
+ setBlockNewContainerRequests(boolean blockNewContainerRequests) {
|
|
|
|
+ // do nothing
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ protected void authorizeGetAndStopContainerRequest(
|
|
|
|
+ ContainerId containerId, Container container,
|
|
|
|
+ boolean stopRequest, NMTokenIdentifier identifier)
|
|
|
|
+ throws YarnException {
|
|
|
|
+ // do nothing
|
|
|
|
+ }
|
|
|
|
+ @Override
|
|
|
|
+ protected void authorizeUser(UserGroupInformation remoteUgi,
|
|
|
|
+ NMTokenIdentifier nmTokenIdentifier) {
|
|
|
|
+ // do nothing
|
|
|
|
+ }
|
|
|
|
+ @Override
|
|
|
|
+ protected void authorizeStartAndResourceIncreaseRequest(
|
|
|
|
+ NMTokenIdentifier nmTokenIdentifier,
|
|
|
|
+ ContainerTokenIdentifier containerTokenIdentifier,
|
|
|
|
+ boolean startRequest) throws YarnException {
|
|
|
|
+ try {
|
|
|
|
+ // Sleep 2 seconds to simulate a pro-longed increase action.
|
|
|
|
+ // If during this time a RESYNC event is sent by RM, the
|
|
|
|
+ // resync action should block until the increase action is
|
|
|
|
+ // completed.
|
|
|
|
+ // See testContainerResourceIncreaseIsSynchronizedWithRMResync()
|
|
|
|
+ Thread.sleep(2000);
|
|
|
|
+ } catch (InterruptedException e) {
|
|
|
|
+ e.printStackTrace();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ @Override
|
|
|
|
+ protected void updateNMTokenIdentifier(
|
|
|
|
+ NMTokenIdentifier nmTokenIdentifier)
|
|
|
|
+ throws SecretManager.InvalidToken {
|
|
|
|
+ // Do nothing
|
|
|
|
+ }
|
|
|
|
+ @Override
|
|
|
|
+ public Map<String, ByteBuffer> getAuxServiceMetaData() {
|
|
|
|
+ return new HashMap<>();
|
|
|
|
+ }
|
|
|
|
+ @Override
|
|
|
|
+ protected NMTokenIdentifier selectNMTokenIdentifier(
|
|
|
|
+ UserGroupInformation remoteUgi) {
|
|
|
|
+ return new NMTokenIdentifier();
|
|
|
|
+ }
|
|
|
|
+ };
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Start a container in NM
|
|
|
|
+ public void startContainer()
|
|
|
|
+ throws IOException, InterruptedException, YarnException {
|
|
|
|
+ LOG.info("Start a container and wait until it is in RUNNING state");
|
|
|
|
+ File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
|
|
|
|
+ PrintWriter fileWriter = new PrintWriter(scriptFile);
|
|
|
|
+ if (Shell.WINDOWS) {
|
|
|
|
+ fileWriter.println("@ping -n 100 127.0.0.1 >nul");
|
|
|
|
+ } else {
|
|
|
|
+ fileWriter.write("\numask 0");
|
|
|
|
+ fileWriter.write("\nexec sleep 100");
|
|
|
|
+ }
|
|
|
|
+ fileWriter.close();
|
|
|
|
+ ContainerLaunchContext containerLaunchContext =
|
|
|
|
+ recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
|
|
+ URL resource_alpha =
|
|
|
|
+ ConverterUtils.getYarnUrlFromPath(localFS
|
|
|
|
+ .makeQualified(new Path(scriptFile.getAbsolutePath())));
|
|
|
|
+ LocalResource rsrc_alpha =
|
|
|
|
+ recordFactory.newRecordInstance(LocalResource.class);
|
|
|
|
+ rsrc_alpha.setResource(resource_alpha);
|
|
|
|
+ rsrc_alpha.setSize(-1);
|
|
|
|
+ rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
|
|
|
|
+ rsrc_alpha.setType(LocalResourceType.FILE);
|
|
|
|
+ rsrc_alpha.setTimestamp(scriptFile.lastModified());
|
|
|
|
+ String destinationFile = "dest_file";
|
|
|
|
+ Map<String, LocalResource> localResources =
|
|
|
|
+ new HashMap<String, LocalResource>();
|
|
|
|
+ localResources.put(destinationFile, rsrc_alpha);
|
|
|
|
+ containerLaunchContext.setLocalResources(localResources);
|
|
|
|
+ List<String> commands =
|
|
|
|
+ Arrays.asList(Shell.getRunScriptCommand(scriptFile));
|
|
|
|
+ containerLaunchContext.setCommands(commands);
|
|
|
|
+ Resource resource = Resource.newInstance(1024, 1);
|
|
|
|
+ StartContainerRequest scRequest =
|
|
|
|
+ StartContainerRequest.newInstance(
|
|
|
|
+ containerLaunchContext,
|
|
|
|
+ getContainerToken(resource));
|
|
|
|
+ List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
|
|
|
+ list.add(scRequest);
|
|
|
|
+ StartContainersRequest allRequests =
|
|
|
|
+ StartContainersRequest.newInstance(list);
|
|
|
|
+ getContainerManager().startContainers(allRequests);
|
|
|
|
+ // Make sure the container reaches RUNNING state
|
|
|
|
+ ContainerId cId = TestContainerManager.createContainerId(0);
|
|
|
|
+ BaseContainerManagerTest.waitForNMContainerState(
|
|
|
|
+ getContainerManager(), cId,
|
|
|
|
+ org.apache.hadoop.yarn.server.nodemanager.
|
|
|
|
+ containermanager.container.ContainerState.RUNNING);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Increase container resource in a thread
|
|
|
|
+ public void increaseContainersResource()
|
|
|
|
+ throws InterruptedException {
|
|
|
|
+ LOG.info("Increase a container resource in a separate thread");
|
|
|
|
+ increaseContainerResourceThread = new IncreaseContainersResourceThread();
|
|
|
|
+ increaseContainerResourceThread.start();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ class TestNodeStatusUpdaterImpl4 extends MockNodeStatusUpdater {
|
|
|
|
+
|
|
|
|
+ public TestNodeStatusUpdaterImpl4(Context context, Dispatcher dispatcher,
|
|
|
|
+ NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
|
|
|
|
+ super(context, dispatcher, healthChecker, metrics);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ protected void rebootNodeStatusUpdaterAndRegisterWithRM() {
|
|
|
|
+ try {
|
|
|
|
+ try {
|
|
|
|
+ // Check status before registerWithRM
|
|
|
|
+ List<ContainerId> containerIds = new ArrayList<>();
|
|
|
|
+ ContainerId cId = TestContainerManager.createContainerId(0);
|
|
|
|
+ containerIds.add(cId);
|
|
|
|
+ GetContainerStatusesRequest gcsRequest =
|
|
|
|
+ GetContainerStatusesRequest.newInstance(containerIds);
|
|
|
|
+ ContainerStatus containerStatus = getContainerManager()
|
|
|
|
+ .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
|
|
|
|
+ assertEquals(Resource.newInstance(1024, 1),
|
|
|
|
+ containerStatus.getCapability());
|
|
|
|
+ // Call the actual rebootNodeStatusUpdaterAndRegisterWithRM().
|
|
|
|
+ // This function should be synchronized with
|
|
|
|
+ // increaseContainersResource().
|
|
|
|
+ super.rebootNodeStatusUpdaterAndRegisterWithRM();
|
|
|
|
+ // Check status after registerWithRM
|
|
|
|
+ containerStatus = getContainerManager()
|
|
|
|
+ .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
|
|
|
|
+ assertEquals(Resource.newInstance(4096, 2),
|
|
|
|
+ containerStatus.getCapability());
|
|
|
|
+ } catch (AssertionError ae) {
|
|
|
|
+ ae.printStackTrace();
|
|
|
|
+ assertionFailedInThread.set(true);
|
|
|
|
+ } finally {
|
|
|
|
+ syncBarrier.await();
|
|
|
|
+ }
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ e.printStackTrace();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ class IncreaseContainersResourceThread extends Thread {
|
|
|
|
+ @Override
|
|
|
|
+ public void run() {
|
|
|
|
+ // Construct container resource increase request
|
|
|
|
+ List<Token> increaseTokens = new ArrayList<Token>();
|
|
|
|
+ // Add increase request.
|
|
|
|
+ Resource targetResource = Resource.newInstance(4096, 2);
|
|
|
|
+ try {
|
|
|
|
+ increaseTokens.add(getContainerToken(targetResource));
|
|
|
|
+ IncreaseContainersResourceRequest increaseRequest =
|
|
|
|
+ IncreaseContainersResourceRequest.newInstance(increaseTokens);
|
|
|
|
+ IncreaseContainersResourceResponse increaseResponse =
|
|
|
|
+ getContainerManager()
|
|
|
|
+ .increaseContainersResource(increaseRequest);
|
|
|
|
+ Assert.assertEquals(
|
|
|
|
+ 1, increaseResponse.getSuccessfullyIncreasedContainers()
|
|
|
|
+ .size());
|
|
|
|
+ Assert.assertTrue(increaseResponse.getFailedRequests().isEmpty());
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ e.printStackTrace();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private Token getContainerToken(Resource resource) throws IOException {
|
|
|
|
+ ContainerId cId = TestContainerManager.createContainerId(0);
|
|
|
|
+ return TestContainerManager.createContainerToken(
|
|
|
|
+ cId, DUMMY_RM_IDENTIFIER,
|
|
|
|
+ getNMContext().getNodeId(), user, resource,
|
|
|
|
+ getNMContext().getContainerTokenSecretManager(), null);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
public static NMContainerStatus createNMContainerStatus(int id,
|
|
public static NMContainerStatus createNMContainerStatus(int id,
|
|
ContainerState containerState) {
|
|
ContainerState containerState) {
|
|
ApplicationId applicationId = ApplicationId.newInstance(0, 1);
|
|
ApplicationId applicationId = ApplicationId.newInstance(0, 1);
|