|
@@ -24,17 +24,12 @@ import static org.mockito.Mockito.when;
|
|
|
import java.io.BufferedReader;
|
|
|
import java.io.File;
|
|
|
import java.io.FileReader;
|
|
|
-import java.io.FileWriter;
|
|
|
import java.io.IOException;
|
|
|
import java.io.PrintWriter;
|
|
|
-import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
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;
|
|
|
|
|
@@ -59,12 +54,9 @@ 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.DefaultContainerExecutor;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
|
|
import org.apache.hadoop.yarn.util.BuilderUtils;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
import org.junit.After;
|
|
@@ -86,7 +78,6 @@ public class TestNodeManagerShutdown {
|
|
|
static final String user = "nobody";
|
|
|
private FileContext localFS;
|
|
|
private ContainerId cId;
|
|
|
- private CyclicBarrier syncBarrier = new CyclicBarrier(2);
|
|
|
|
|
|
@Before
|
|
|
public void setup() throws UnsupportedFileSystemException {
|
|
@@ -110,7 +101,7 @@ public class TestNodeManagerShutdown {
|
|
|
NodeManager nm = getNodeManager();
|
|
|
nm.init(createNMConfig());
|
|
|
nm.start();
|
|
|
- startContainers(nm);
|
|
|
+ startContainer(nm, cId, localFS, tmpDir, processStartFile);
|
|
|
|
|
|
final int MAX_TRIES=20;
|
|
|
int numTries = 0;
|
|
@@ -150,29 +141,13 @@ public class TestNodeManagerShutdown {
|
|
|
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 {
|
|
|
+ public static void startContainer(NodeManager nm, ContainerId cId,
|
|
|
+ FileContext localFS, File scriptFileDir, File processStartFile)
|
|
|
+ throws IOException {
|
|
|
ContainerManagerImpl containerManager = nm.getContainerManager();
|
|
|
- File scriptFile = createUnhaltingScriptFile();
|
|
|
+ File scriptFile =
|
|
|
+ createUnhaltingScriptFile(cId, scriptFileDir, processStartFile);
|
|
|
|
|
|
ContainerLaunchContext containerLaunchContext =
|
|
|
recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
@@ -218,7 +193,7 @@ public class TestNodeManagerShutdown {
|
|
|
Assert.assertEquals(ContainerState.RUNNING, containerStatus.getState());
|
|
|
}
|
|
|
|
|
|
- private ContainerId createContainerId() {
|
|
|
+ public static ContainerId createContainerId() {
|
|
|
ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
|
|
|
appId.setClusterTimestamp(0);
|
|
|
appId.setId(0);
|
|
@@ -247,8 +222,9 @@ public class TestNodeManagerShutdown {
|
|
|
* Creates a script to run a container that will run forever unless
|
|
|
* stopped by external means.
|
|
|
*/
|
|
|
- private File createUnhaltingScriptFile() throws IOException {
|
|
|
- File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
|
|
|
+ private static File createUnhaltingScriptFile(ContainerId cId,
|
|
|
+ File scriptFileDir, File processStartFile) throws IOException {
|
|
|
+ File scriptFile = Shell.appendScriptExtension(scriptFileDir, "scriptFile");
|
|
|
PrintWriter fileWriter = new PrintWriter(scriptFile);
|
|
|
if (Shell.WINDOWS) {
|
|
|
fileWriter.println("@echo \"Running testscript for delayed kill\"");
|
|
@@ -282,48 +258,4 @@ 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, org.apache.hadoop.yarn.server.nodemanager.containermanager.container.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) {
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
}
|