|
@@ -28,6 +28,9 @@ import java.util.ArrayList;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
+import java.util.concurrent.BrokenBarrierException;
|
|
|
+import java.util.concurrent.ConcurrentMap;
|
|
|
+import java.util.concurrent.CyclicBarrier;
|
|
|
|
|
|
import junit.framework.Assert;
|
|
|
|
|
@@ -49,9 +52,12 @@ import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.URL;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
|
|
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Before;
|
|
@@ -71,6 +77,7 @@ public class TestNodeManagerShutdown {
|
|
|
.getRecordFactory(null);
|
|
|
static final String user = "nobody";
|
|
|
private FileContext localFS;
|
|
|
+ private CyclicBarrier syncBarrier = new CyclicBarrier(2);
|
|
|
|
|
|
@Before
|
|
|
public void setup() throws UnsupportedFileSystemException {
|
|
@@ -91,11 +98,64 @@ public class TestNodeManagerShutdown {
|
|
|
NodeManager nm = getNodeManager();
|
|
|
nm.init(createNMConfig());
|
|
|
nm.start();
|
|
|
+ startContainers(nm);
|
|
|
|
|
|
+ final int MAX_TRIES=20;
|
|
|
+ int numTries = 0;
|
|
|
+ while (!processStartFile.exists() && numTries < MAX_TRIES) {
|
|
|
+ try {
|
|
|
+ Thread.sleep(500);
|
|
|
+ } catch (InterruptedException ex) {ex.printStackTrace();}
|
|
|
+ numTries++;
|
|
|
+ }
|
|
|
+
|
|
|
+ nm.stop();
|
|
|
+
|
|
|
+ // Now verify the contents of the file
|
|
|
+ // Script generates a message when it receives a sigterm
|
|
|
+ // so we look for that
|
|
|
+ BufferedReader reader =
|
|
|
+ new BufferedReader(new FileReader(processStartFile));
|
|
|
+
|
|
|
+ boolean foundSigTermMessage = false;
|
|
|
+ while (true) {
|
|
|
+ String line = reader.readLine();
|
|
|
+ if (line == null) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ if (line.contains("SIGTERM")) {
|
|
|
+ foundSigTermMessage = true;
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ Assert.assertTrue("Did not find sigterm message", foundSigTermMessage);
|
|
|
+ reader.close();
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ @Test
|
|
|
+ public void testKillContainersOnResync() throws IOException, InterruptedException {
|
|
|
+ NodeManager nm = new TestNodeManager();
|
|
|
+ YarnConfiguration conf = createNMConfig();
|
|
|
+ nm.init(conf);
|
|
|
+ nm.start();
|
|
|
+ startContainers(nm);
|
|
|
+
|
|
|
+ assert ((TestNodeManager) nm).getNMRegistrationCount() == 1;
|
|
|
+ nm.getNMDispatcher().getEventHandler().
|
|
|
+ handle( new NodeManagerEvent(NodeManagerEventType.RESYNC));
|
|
|
+ try {
|
|
|
+ syncBarrier.await();
|
|
|
+ } catch (BrokenBarrierException e) {
|
|
|
+ }
|
|
|
+ assert ((TestNodeManager) nm).getNMRegistrationCount() == 2;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void startContainers(NodeManager nm) throws IOException {
|
|
|
ContainerManagerImpl containerManager = nm.getContainerManager();
|
|
|
File scriptFile = createUnhaltingScriptFile();
|
|
|
|
|
|
- ContainerLaunchContext containerLaunchContext =
|
|
|
+ ContainerLaunchContext containerLaunchContext =
|
|
|
recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
|
|
|
|
// Construct the Container-id
|
|
@@ -127,7 +187,8 @@ public class TestNodeManagerShutdown {
|
|
|
containerLaunchContext.setResource(recordFactory
|
|
|
.newRecordInstance(Resource.class));
|
|
|
containerLaunchContext.getResource().setMemory(1024);
|
|
|
- StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
|
|
|
+ StartContainerRequest startRequest =
|
|
|
+ recordFactory.newRecordInstance(StartContainerRequest.class);
|
|
|
startRequest.setContainerLaunchContext(containerLaunchContext);
|
|
|
containerManager.startContainer(startRequest);
|
|
|
|
|
@@ -137,37 +198,6 @@ public class TestNodeManagerShutdown {
|
|
|
ContainerStatus containerStatus =
|
|
|
containerManager.getContainerStatus(request).getStatus();
|
|
|
Assert.assertEquals(ContainerState.RUNNING, containerStatus.getState());
|
|
|
-
|
|
|
- final int MAX_TRIES=20;
|
|
|
- int numTries = 0;
|
|
|
- while (!processStartFile.exists() && numTries < MAX_TRIES) {
|
|
|
- try {
|
|
|
- Thread.sleep(500);
|
|
|
- } catch (InterruptedException ex) {ex.printStackTrace();}
|
|
|
- numTries++;
|
|
|
- }
|
|
|
-
|
|
|
- nm.stop();
|
|
|
-
|
|
|
- // Now verify the contents of the file
|
|
|
- // Script generates a message when it receives a sigterm
|
|
|
- // so we look for that
|
|
|
- BufferedReader reader =
|
|
|
- new BufferedReader(new FileReader(processStartFile));
|
|
|
-
|
|
|
- boolean foundSigTermMessage = false;
|
|
|
- while (true) {
|
|
|
- String line = reader.readLine();
|
|
|
- if (line == null) {
|
|
|
- break;
|
|
|
- }
|
|
|
- if (line.contains("SIGTERM")) {
|
|
|
- foundSigTermMessage = true;
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- Assert.assertTrue("Did not find sigterm message", foundSigTermMessage);
|
|
|
- reader.close();
|
|
|
}
|
|
|
|
|
|
private ContainerId createContainerId() {
|
|
@@ -226,4 +256,48 @@ public class TestNodeManagerShutdown {
|
|
|
}
|
|
|
};
|
|
|
}
|
|
|
+
|
|
|
+ class TestNodeManager extends NodeManager {
|
|
|
+
|
|
|
+ private int registrationCount = 0;
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected NodeStatusUpdater createNodeStatusUpdater(Context context,
|
|
|
+ Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
|
|
|
+ return new TestNodeStatusUpdaterImpl(context, dispatcher,
|
|
|
+ healthChecker, metrics);
|
|
|
+ }
|
|
|
+
|
|
|
+ public int getNMRegistrationCount() {
|
|
|
+ return registrationCount;
|
|
|
+ }
|
|
|
+
|
|
|
+ class TestNodeStatusUpdaterImpl extends MockNodeStatusUpdater {
|
|
|
+
|
|
|
+ public TestNodeStatusUpdaterImpl(Context context, Dispatcher dispatcher,
|
|
|
+ NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
|
|
|
+ super(context, dispatcher, healthChecker, metrics);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void registerWithRM() throws YarnRemoteException {
|
|
|
+ super.registerWithRM();
|
|
|
+ registrationCount++;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void rebootNodeStatusUpdater() {
|
|
|
+ ConcurrentMap<ContainerId, Container> containers =
|
|
|
+ getNMContext().getContainers();
|
|
|
+ // ensure that containers are empty before restart nodeStatusUpdater
|
|
|
+ Assert.assertTrue(containers.isEmpty());
|
|
|
+ super.rebootNodeStatusUpdater();
|
|
|
+ try {
|
|
|
+ syncBarrier.await();
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ } catch (BrokenBarrierException e) {
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|