Browse Source

YARN-7890. NPE during container relaunch. Contributed by Jason Lowe

Billie Rinaldi 7 years ago
parent
commit
a196ee9362

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerRelaunch.java

@@ -35,7 +35,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
 import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -97,6 +96,8 @@ public class ContainerRelaunch extends ContainerLaunch {
       List<String> logDirs = dirsHandler.getLogDirs();
       List<String> containerLocalDirs = getContainerLocalDirs(localDirs);
       List<String> containerLogDirs = getContainerLogDirs(logDirs);
+      List<String> filecacheDirs = getNMFilecacheDirs(localDirs);
+      List<String> userLocalDirs = getUserLocalDirs(localDirs);
 
       if (!dirsHandler.areDisksHealthy()) {
         ret = ContainerExitStatus.DISKS_FAILED;
@@ -114,6 +115,8 @@ public class ContainerRelaunch extends ContainerLaunch {
           .setContainerWorkDir(containerWorkDir)
           .setLocalDirs(localDirs)
           .setLogDirs(logDirs)
+          .setFilecacheDirs(filecacheDirs)
+          .setUserLocalDirs(userLocalDirs)
           .setContainerLocalDirs(containerLocalDirs)
           .setContainerLogDirs(containerLogDirs)
           .build());

+ 97 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerRelaunch.java

@@ -0,0 +1,97 @@
+/*
+ * 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.nodemanager.containermanager.launcher;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+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.event.Dispatcher;
+import org.apache.hadoop.yarn.event.InlineDispatcher;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
+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.executor.ContainerStartContext;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+/** Unit tests for relaunching containers. */
+public class TestContainerRelaunch {
+
+  @Test
+  public void testRelaunchContext() throws Exception {
+    Configuration conf = new Configuration();
+
+    Context mockContext = mock(Context.class);
+    doReturn(new NMNullStateStoreService()).when(mockContext).getNMStateStore();
+    ApplicationId appId = ApplicationId.newInstance(1, 1);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(appId, 1);
+    ContainerId cid = ContainerId.newContainerId(appAttemptId, 1);
+    Application mockApp = mock(Application.class);
+    doReturn(appId).when(mockApp).getAppId();
+    Container mockContainer = mock(Container.class);
+    doReturn("somebody").when(mockContainer).getUser();
+    doReturn(cid).when(mockContainer).getContainerId();
+    doReturn("/foo").when(mockContainer).getWorkDir();
+    doReturn("/bar").when(mockContainer).getLogDir();
+    LocalDirsHandlerService mockDirsHandler =
+        mock(LocalDirsHandlerService.class);
+    doReturn(true).when(mockDirsHandler).isGoodLocalDir(any(String.class));
+    doReturn(true).when(mockDirsHandler).isGoodLogDir(anyString());
+    doReturn(true).when(mockDirsHandler).areDisksHealthy();
+    doReturn(new Path("/some/file")).when(mockDirsHandler)
+        .getLocalPathForRead(anyString());
+    Dispatcher dispatcher = new InlineDispatcher();
+    ContainerExecutor mockExecutor = mock(ContainerExecutor.class);
+    ContainerRelaunch cr = new ContainerRelaunch(mockContext, conf, dispatcher,
+        mockExecutor, mockApp, mockContainer, mockDirsHandler, null);
+    int result = cr.call();
+    assertEquals("relaunch failed", 0, result);
+    ArgumentCaptor<ContainerStartContext> captor =
+        ArgumentCaptor.forClass(ContainerStartContext.class);
+    verify(mockExecutor).launchContainer(captor.capture());
+    ContainerStartContext csc = captor.getValue();
+    assertNotNull("app ID null", csc.getAppId());
+    assertNotNull("container null", csc.getContainer());
+    assertNotNull("container local dirs null", csc.getContainerLocalDirs());
+    assertNotNull("container log dirs null", csc.getContainerLogDirs());
+    assertNotNull("work dir null", csc.getContainerWorkDir());
+    assertNotNull("filecache dirs null", csc.getFilecacheDirs());
+    assertNotNull("local dirs null", csc.getLocalDirs());
+    assertNotNull("localized resources null", csc.getLocalizedResources());
+    assertNotNull("log dirs null", csc.getLogDirs());
+    assertNotNull("script path null", csc.getNmPrivateContainerScriptPath());
+    assertNotNull("tokens path null", csc.getNmPrivateTokensPath());
+    assertNotNull("user null", csc.getUser());
+    assertNotNull("user local dirs null", csc.getUserLocalDirs());
+  }
+}