|
@@ -20,65 +20,59 @@ package org.apache.hadoop.yarn.server.nodemanager;
|
|
|
|
|
|
import static org.apache.hadoop.fs.CreateFlag.CREATE;
|
|
|
import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+import static org.mockito.Matchers.any;
|
|
|
+import static org.mockito.Matchers.isA;
|
|
|
+import static org.mockito.Mockito.doAnswer;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
-import static org.mockito.Mockito.when;
|
|
|
import static org.mockito.Mockito.spy;
|
|
|
-import static org.mockito.Mockito.any;
|
|
|
-import static org.mockito.Mockito.doAnswer;
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
+import static org.mockito.Mockito.times;
|
|
|
+import static org.mockito.Mockito.verify;
|
|
|
+import static org.mockito.Mockito.when;
|
|
|
|
|
|
import java.io.BufferedWriter;
|
|
|
import java.io.DataOutputStream;
|
|
|
import java.io.File;
|
|
|
import java.io.FileNotFoundException;
|
|
|
-import java.io.FileReader;
|
|
|
import java.io.FileWriter;
|
|
|
-import java.io.InputStream;
|
|
|
import java.io.IOException;
|
|
|
-import java.io.LineNumberReader;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.HashMap;
|
|
|
-import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
import java.util.Random;
|
|
|
|
|
|
-
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.fs.AbstractFileSystem;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
-import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileContext;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
-import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.fs.FsStatus;
|
|
|
-import org.apache.hadoop.fs.Options.CreateOpts;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
-import org.apache.hadoop.io.DataInputBuffer;
|
|
|
-import org.apache.hadoop.io.DataOutputBuffer;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
-import org.apache.hadoop.util.Progressable;
|
|
|
import org.apache.hadoop.util.Shell;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
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.api.records.ContainerLaunchContext;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
+import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerDiagnosticsUpdateEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.FakeFSDataInputStream;
|
|
|
-
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.MockLocalizerHeartbeatResponse;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.executor.LocalizerStartContext;
|
|
|
-import org.junit.After;
|
|
|
import org.junit.AfterClass;
|
|
|
import org.junit.Assert;
|
|
|
-import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
@@ -336,10 +330,9 @@ public class TestDefaultContainerExecutor {
|
|
|
}
|
|
|
|
|
|
@Test(timeout = 30000)
|
|
|
- public void testStartLocalizer()
|
|
|
- throws IOException, InterruptedException {
|
|
|
- InetSocketAddress localizationServerAddress;
|
|
|
-
|
|
|
+ public void testStartLocalizer() throws IOException, InterruptedException,
|
|
|
+ YarnException {
|
|
|
+
|
|
|
final Path firstDir = new Path(BASE_TMP_PATH, "localDir1");
|
|
|
List<String> localDirs = new ArrayList<String>();
|
|
|
final Path secondDir = new Path(BASE_TMP_PATH, "localDir2");
|
|
@@ -349,11 +342,6 @@ public class TestDefaultContainerExecutor {
|
|
|
FsPermission perms = new FsPermission((short)0770);
|
|
|
|
|
|
Configuration conf = new Configuration();
|
|
|
- localizationServerAddress = conf.getSocketAddr(
|
|
|
- YarnConfiguration.NM_BIND_HOST,
|
|
|
- YarnConfiguration.NM_LOCALIZER_ADDRESS,
|
|
|
- YarnConfiguration.DEFAULT_NM_LOCALIZER_ADDRESS,
|
|
|
- YarnConfiguration.DEFAULT_NM_LOCALIZER_PORT);
|
|
|
|
|
|
final FileContext mockLfs = spy(FileContext.getLocalFSFileContext(conf));
|
|
|
final FileContext.Util mockUtil = spy(mockLfs.util());
|
|
@@ -391,6 +379,7 @@ public class TestDefaultContainerExecutor {
|
|
|
return null;
|
|
|
}
|
|
|
}).when(mockUtil).copy(any(Path.class), any(Path.class));
|
|
|
+
|
|
|
doAnswer(new Answer() {
|
|
|
@Override
|
|
|
public Object answer(InvocationOnMock invocationOnMock)
|
|
@@ -406,8 +395,33 @@ public class TestDefaultContainerExecutor {
|
|
|
}
|
|
|
}).when(mockLfs).getFsStatus(any(Path.class));
|
|
|
|
|
|
- DefaultContainerExecutor mockExec = spy(new DefaultContainerExecutor(
|
|
|
- mockLfs));
|
|
|
+ DefaultContainerExecutor mockExec =
|
|
|
+ spy(new DefaultContainerExecutor(mockLfs) {
|
|
|
+ @Override
|
|
|
+ public ContainerLocalizer createContainerLocalizer(String user,
|
|
|
+ String appId, String locId, List<String> localDirs,
|
|
|
+ FileContext localizerFc) throws IOException {
|
|
|
+
|
|
|
+ // Spy on the localizer and make it return valid heart-beat
|
|
|
+ // responses even though there is no real NodeManager.
|
|
|
+ ContainerLocalizer localizer =
|
|
|
+ super.createContainerLocalizer(user, appId, locId, localDirs,
|
|
|
+ localizerFc);
|
|
|
+ ContainerLocalizer spyLocalizer = spy(localizer);
|
|
|
+ LocalizationProtocol nmProxy = mock(LocalizationProtocol.class);
|
|
|
+ try {
|
|
|
+ when(nmProxy.heartbeat(isA(LocalizerStatus.class))).thenReturn(
|
|
|
+ new MockLocalizerHeartbeatResponse(LocalizerAction.DIE,
|
|
|
+ new ArrayList<ResourceLocalizationSpec>()));
|
|
|
+ } catch (YarnException e) {
|
|
|
+ throw new IOException(e);
|
|
|
+ }
|
|
|
+ when(spyLocalizer.getProxy(any(InetSocketAddress.class)))
|
|
|
+ .thenReturn(nmProxy);
|
|
|
+
|
|
|
+ return spyLocalizer;
|
|
|
+ }
|
|
|
+ });
|
|
|
mockExec.setConf(conf);
|
|
|
localDirs.add(mockLfs.makeQualified(firstDir).toString());
|
|
|
localDirs.add(mockLfs.makeQualified(secondDir).toString());
|
|
@@ -424,18 +438,20 @@ public class TestDefaultContainerExecutor {
|
|
|
LocalDirsHandlerService dirsHandler = mock(LocalDirsHandlerService.class);
|
|
|
when(dirsHandler.getLocalDirs()).thenReturn(localDirs);
|
|
|
when(dirsHandler.getLogDirs()).thenReturn(logDirs);
|
|
|
-
|
|
|
+
|
|
|
try {
|
|
|
mockExec.startLocalizer(new LocalizerStartContext.Builder()
|
|
|
.setNmPrivateContainerTokens(nmPrivateCTokensPath)
|
|
|
- .setNmAddr(localizationServerAddress)
|
|
|
+ .setNmAddr(null)
|
|
|
.setUser(appSubmitter)
|
|
|
.setAppId(appId)
|
|
|
.setLocId(locId)
|
|
|
.setDirsHandler(dirsHandler)
|
|
|
.build());
|
|
|
+
|
|
|
} catch (IOException e) {
|
|
|
- Assert.fail("StartLocalizer failed to copy token file " + e);
|
|
|
+ Assert.fail("StartLocalizer failed to copy token file: "
|
|
|
+ + StringUtils.stringifyException(e));
|
|
|
} finally {
|
|
|
mockExec.deleteAsUser(new DeletionAsUserContext.Builder()
|
|
|
.setUser(appSubmitter)
|
|
@@ -451,7 +467,12 @@ public class TestDefaultContainerExecutor {
|
|
|
.build());
|
|
|
deleteTmpFiles();
|
|
|
}
|
|
|
+
|
|
|
+ // Verify that the calls happen the expected number of times
|
|
|
+ verify(mockUtil, times(1)).copy(any(Path.class), any(Path.class));
|
|
|
+ verify(mockLfs, times(2)).getFsStatus(any(Path.class));
|
|
|
}
|
|
|
+
|
|
|
// @Test
|
|
|
// public void testInit() throws IOException, InterruptedException {
|
|
|
// Configuration conf = new Configuration();
|