|
@@ -17,8 +17,6 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
|
|
|
|
|
|
-import static junit.framework.TestCase.assertFalse;
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
import static org.mockito.Matchers.any;
|
|
|
import static org.mockito.Matchers.anyBoolean;
|
|
@@ -27,7 +25,6 @@ import static org.mockito.Matchers.argThat;
|
|
|
import static org.mockito.Matchers.eq;
|
|
|
import static org.mockito.Matchers.isA;
|
|
|
import static org.mockito.Matchers.same;
|
|
|
-import static org.mockito.Mockito.doAnswer;
|
|
|
import static org.mockito.Mockito.doNothing;
|
|
|
import static org.mockito.Mockito.doReturn;
|
|
|
import static org.mockito.Mockito.doThrow;
|
|
@@ -48,9 +45,7 @@ import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.CompletionService;
|
|
|
import java.util.concurrent.ExecutorService;
|
|
|
import java.util.concurrent.Future;
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
-import com.google.common.base.Supplier;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -65,9 +60,6 @@ import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
-import org.apache.hadoop.test.GenericTestUtils;
|
|
|
-import org.apache.hadoop.util.Shell;
|
|
|
-import org.apache.hadoop.util.Shell.ShellCommandExecutor;
|
|
|
import org.apache.hadoop.yarn.api.records.LocalResource;
|
|
|
import org.apache.hadoop.yarn.api.records.LocalResourceType;
|
|
|
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
|
@@ -84,7 +76,6 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.ArgumentMatcher;
|
|
|
-import org.mockito.Mockito;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
|
|
|
@@ -101,18 +92,18 @@ public class TestContainerLocalizer {
|
|
|
static final InetSocketAddress nmAddr =
|
|
|
new InetSocketAddress("foobar", 8040);
|
|
|
|
|
|
+ private AbstractFileSystem spylfs;
|
|
|
+ private Random random;
|
|
|
+ private List<Path> localDirs;
|
|
|
+ private Path tokenPath;
|
|
|
+ private LocalizationProtocol nmProxy;
|
|
|
|
|
|
@Test
|
|
|
public void testMain() throws Exception {
|
|
|
- ContainerLocalizerWrapper wrapper = new ContainerLocalizerWrapper();
|
|
|
+ FileContext fs = FileContext.getLocalFSFileContext();
|
|
|
+ spylfs = spy(fs.getDefaultFileSystem());
|
|
|
ContainerLocalizer localizer =
|
|
|
- wrapper.setupContainerLocalizerForTest();
|
|
|
- Random random = wrapper.random;
|
|
|
- List<Path> localDirs = wrapper.localDirs;
|
|
|
- Path tokenPath = wrapper.tokenPath;
|
|
|
- LocalizationProtocol nmProxy = wrapper.nmProxy;
|
|
|
- AbstractFileSystem spylfs = wrapper.spylfs;
|
|
|
- mockOutDownloads(localizer);
|
|
|
+ setupContainerLocalizerForTest();
|
|
|
|
|
|
// verify created cache
|
|
|
List<Path> privCacheList = new ArrayList<Path>();
|
|
@@ -140,7 +131,7 @@ public class TestContainerLocalizer {
|
|
|
ResourceLocalizationSpec rsrcD =
|
|
|
getMockRsrc(random, LocalResourceVisibility.PRIVATE,
|
|
|
privCacheList.get(0));
|
|
|
-
|
|
|
+
|
|
|
when(nmProxy.heartbeat(isA(LocalizerStatus.class)))
|
|
|
.thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.LIVE,
|
|
|
Collections.singletonList(rsrcA)))
|
|
@@ -211,10 +202,10 @@ public class TestContainerLocalizer {
|
|
|
|
|
|
@Test(timeout = 15000)
|
|
|
public void testMainFailure() throws Exception {
|
|
|
- ContainerLocalizerWrapper wrapper = new ContainerLocalizerWrapper();
|
|
|
- ContainerLocalizer localizer = wrapper.setupContainerLocalizerForTest();
|
|
|
- LocalizationProtocol nmProxy = wrapper.nmProxy;
|
|
|
- mockOutDownloads(localizer);
|
|
|
+
|
|
|
+ FileContext fs = FileContext.getLocalFSFileContext();
|
|
|
+ spylfs = spy(fs.getDefaultFileSystem());
|
|
|
+ ContainerLocalizer localizer = setupContainerLocalizerForTest();
|
|
|
|
|
|
// Assume the NM heartbeat fails say because of absent tokens.
|
|
|
when(nmProxy.heartbeat(isA(LocalizerStatus.class))).thenThrow(
|
|
@@ -232,11 +223,9 @@ public class TestContainerLocalizer {
|
|
|
@Test
|
|
|
@SuppressWarnings("unchecked")
|
|
|
public void testLocalizerTokenIsGettingRemoved() throws Exception {
|
|
|
- ContainerLocalizerWrapper wrapper = new ContainerLocalizerWrapper();
|
|
|
- ContainerLocalizer localizer = wrapper.setupContainerLocalizerForTest();
|
|
|
- Path tokenPath = wrapper.tokenPath;
|
|
|
- AbstractFileSystem spylfs = wrapper.spylfs;
|
|
|
- mockOutDownloads(localizer);
|
|
|
+ FileContext fs = FileContext.getLocalFSFileContext();
|
|
|
+ spylfs = spy(fs.getDefaultFileSystem());
|
|
|
+ ContainerLocalizer localizer = setupContainerLocalizerForTest();
|
|
|
doNothing().when(localizer).localizeFiles(any(LocalizationProtocol.class),
|
|
|
any(CompletionService.class), any(UserGroupInformation.class));
|
|
|
localizer.runLocalization(nmAddr);
|
|
@@ -248,10 +237,10 @@ public class TestContainerLocalizer {
|
|
|
public void testContainerLocalizerClosesFilesystems() throws Exception {
|
|
|
|
|
|
// verify filesystems are closed when localizer doesn't fail
|
|
|
- ContainerLocalizerWrapper wrapper = new ContainerLocalizerWrapper();
|
|
|
+ FileContext fs = FileContext.getLocalFSFileContext();
|
|
|
+ spylfs = spy(fs.getDefaultFileSystem());
|
|
|
|
|
|
- ContainerLocalizer localizer = wrapper.setupContainerLocalizerForTest();
|
|
|
- mockOutDownloads(localizer);
|
|
|
+ ContainerLocalizer localizer = setupContainerLocalizerForTest();
|
|
|
doNothing().when(localizer).localizeFiles(any(LocalizationProtocol.class),
|
|
|
any(CompletionService.class), any(UserGroupInformation.class));
|
|
|
verify(localizer, never()).closeFileSystems(
|
|
@@ -260,8 +249,10 @@ public class TestContainerLocalizer {
|
|
|
localizer.runLocalization(nmAddr);
|
|
|
verify(localizer).closeFileSystems(any(UserGroupInformation.class));
|
|
|
|
|
|
+ spylfs = spy(fs.getDefaultFileSystem());
|
|
|
+
|
|
|
// verify filesystems are closed when localizer fails
|
|
|
- localizer = wrapper.setupContainerLocalizerForTest();
|
|
|
+ localizer = setupContainerLocalizerForTest();
|
|
|
doThrow(new YarnRuntimeException("Forced Failure")).when(localizer).localizeFiles(
|
|
|
any(LocalizationProtocol.class), any(CompletionService.class),
|
|
|
any(UserGroupInformation.class));
|
|
@@ -275,102 +266,41 @@ public class TestContainerLocalizer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
- public void testMultipleLocalizers() throws Exception {
|
|
|
- FakeContainerLocalizerWrapper testA = new FakeContainerLocalizerWrapper();
|
|
|
- FakeContainerLocalizerWrapper testB = new FakeContainerLocalizerWrapper();
|
|
|
-
|
|
|
- final FakeContainerLocalizer localizerA = testA.init();
|
|
|
- final FakeContainerLocalizer localizerB = testB.init();
|
|
|
-
|
|
|
- // run localization
|
|
|
- Thread threadA = new Thread() {
|
|
|
- @Override
|
|
|
- public void run() {
|
|
|
- try {
|
|
|
- localizerA.runLocalization(nmAddr);
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.warn(e);
|
|
|
- }
|
|
|
- }
|
|
|
- };
|
|
|
- Thread threadB = new Thread() {
|
|
|
- @Override
|
|
|
- public void run() {
|
|
|
- try {
|
|
|
- localizerB.runLocalization(nmAddr);
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.warn(e);
|
|
|
- }
|
|
|
- }
|
|
|
- };
|
|
|
- ShellCommandExecutor shexcA = null;
|
|
|
- ShellCommandExecutor shexcB = null;
|
|
|
- try {
|
|
|
- threadA.start();
|
|
|
- threadB.start();
|
|
|
-
|
|
|
- GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
- @Override
|
|
|
- public Boolean get() {
|
|
|
- FakeContainerLocalizer.FakeLongDownload downloader =
|
|
|
- localizerA.getDownloader();
|
|
|
- return downloader != null && downloader.getShexc() != null &&
|
|
|
- downloader.getShexc().getProcess() != null;
|
|
|
- }
|
|
|
- }, 10, 30000);
|
|
|
-
|
|
|
- GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
- @Override
|
|
|
- public Boolean get() {
|
|
|
- FakeContainerLocalizer.FakeLongDownload downloader =
|
|
|
- localizerB.getDownloader();
|
|
|
- return downloader != null && downloader.getShexc() != null &&
|
|
|
- downloader.getShexc().getProcess() != null;
|
|
|
- }
|
|
|
- }, 10, 30000);
|
|
|
-
|
|
|
- shexcA = localizerA.getDownloader().getShexc();
|
|
|
- shexcB = localizerB.getDownloader().getShexc();
|
|
|
-
|
|
|
- assertTrue("Localizer A process not running, but should be",
|
|
|
- shexcA.getProcess().isAlive());
|
|
|
- assertTrue("Localizer B process not running, but should be",
|
|
|
- shexcB.getProcess().isAlive());
|
|
|
-
|
|
|
- // Stop heartbeat from giving anymore resources to download
|
|
|
- testA.heartbeatResponse++;
|
|
|
- testB.heartbeatResponse++;
|
|
|
-
|
|
|
- // Send DIE to localizerA. This should kill its subprocesses
|
|
|
- testA.heartbeatResponse++;
|
|
|
-
|
|
|
- threadA.join();
|
|
|
- shexcA.getProcess().waitFor(10000, TimeUnit.MILLISECONDS);
|
|
|
-
|
|
|
- assertFalse("Localizer A process is still running, but shouldn't be",
|
|
|
- shexcA.getProcess().isAlive());
|
|
|
- assertTrue("Localizer B process not running, but should be",
|
|
|
- shexcB.getProcess().isAlive());
|
|
|
-
|
|
|
- } finally {
|
|
|
- // Make sure everything gets cleaned up
|
|
|
- // Process A should already be dead
|
|
|
- shexcA.getProcess().destroy();
|
|
|
- shexcB.getProcess().destroy();
|
|
|
- shexcA.getProcess().waitFor(10000, TimeUnit.MILLISECONDS);
|
|
|
- shexcB.getProcess().waitFor(10000, TimeUnit.MILLISECONDS);
|
|
|
-
|
|
|
- threadA.join();
|
|
|
- // Send DIE to localizer B
|
|
|
- testB.heartbeatResponse++;
|
|
|
- threadB.join();
|
|
|
+ @SuppressWarnings("unchecked") // mocked generics
|
|
|
+ private ContainerLocalizer setupContainerLocalizerForTest()
|
|
|
+ throws Exception {
|
|
|
+ // don't actually create dirs
|
|
|
+ doNothing().when(spylfs).mkdir(
|
|
|
+ isA(Path.class), isA(FsPermission.class), anyBoolean());
|
|
|
+
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ FileContext lfs = FileContext.getFileContext(spylfs, conf);
|
|
|
+ localDirs = new ArrayList<Path>();
|
|
|
+ for (int i = 0; i < 4; ++i) {
|
|
|
+ localDirs.add(lfs.makeQualified(new Path(basedir, i + "")));
|
|
|
}
|
|
|
- }
|
|
|
+ RecordFactory mockRF = getMockLocalizerRecordFactory();
|
|
|
+ ContainerLocalizer concreteLoc = new ContainerLocalizer(lfs, appUser,
|
|
|
+ appId, containerId, localDirs, mockRF);
|
|
|
+ ContainerLocalizer localizer = spy(concreteLoc);
|
|
|
+
|
|
|
+ // return credential stream instead of opening local file
|
|
|
+ random = new Random();
|
|
|
+ long seed = random.nextLong();
|
|
|
+ System.out.println("SEED: " + seed);
|
|
|
+ random.setSeed(seed);
|
|
|
+ DataInputBuffer appTokens = createFakeCredentials(random, 10);
|
|
|
+ tokenPath =
|
|
|
+ lfs.makeQualified(new Path(
|
|
|
+ String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT,
|
|
|
+ containerId)));
|
|
|
+ doReturn(new FSDataInputStream(new FakeFSDataInputStream(appTokens))
|
|
|
+ ).when(spylfs).open(tokenPath);
|
|
|
+ nmProxy = mock(LocalizationProtocol.class);
|
|
|
+ doReturn(nmProxy).when(localizer).getProxy(nmAddr);
|
|
|
+ doNothing().when(localizer).sleep(anyInt());
|
|
|
+
|
|
|
|
|
|
-
|
|
|
-
|
|
|
- private void mockOutDownloads(ContainerLocalizer localizer) {
|
|
|
// return result instantly for deterministic test
|
|
|
ExecutorService syncExec = mock(ExecutorService.class);
|
|
|
CompletionService<Path> cs = mock(CompletionService.class);
|
|
@@ -388,6 +318,8 @@ public class TestContainerLocalizer {
|
|
|
});
|
|
|
doReturn(syncExec).when(localizer).createDownloadThreadPool();
|
|
|
doReturn(cs).when(localizer).createCompletionService(syncExec);
|
|
|
+
|
|
|
+ return localizer;
|
|
|
}
|
|
|
|
|
|
static class HBMatches extends ArgumentMatcher<LocalizerStatus> {
|
|
@@ -431,141 +363,6 @@ public class TestContainerLocalizer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- class FakeContainerLocalizer extends ContainerLocalizer {
|
|
|
- private FakeLongDownload downloader;
|
|
|
-
|
|
|
- FakeContainerLocalizer(FileContext lfs, String user, String appId,
|
|
|
- String localizerId, List<Path> localDirs,
|
|
|
- RecordFactory recordFactory) throws IOException {
|
|
|
- super(lfs, user, appId, localizerId, localDirs, recordFactory);
|
|
|
- }
|
|
|
-
|
|
|
- FakeLongDownload getDownloader() {
|
|
|
- return downloader;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- Callable<Path> download(Path path, LocalResource rsrc,
|
|
|
- UserGroupInformation ugi) throws IOException {
|
|
|
- downloader = new FakeLongDownload(Mockito.mock(FileContext.class), ugi,
|
|
|
- new Configuration(), path, rsrc);
|
|
|
- return downloader;
|
|
|
- }
|
|
|
-
|
|
|
- class FakeLongDownload extends ContainerLocalizer.FSDownloadWrapper {
|
|
|
- private final Path localPath;
|
|
|
- private Shell.ShellCommandExecutor shexc;
|
|
|
- FakeLongDownload(FileContext files, UserGroupInformation ugi,
|
|
|
- Configuration conf, Path destDirPath, LocalResource resource) {
|
|
|
- super(files, ugi, conf, destDirPath, resource);
|
|
|
- this.localPath = new Path("file:///localcache");
|
|
|
- }
|
|
|
-
|
|
|
- Shell.ShellCommandExecutor getShexc() {
|
|
|
- return shexc;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public Path doDownloadCall() throws IOException {
|
|
|
- String sleepCommand = "sleep 30";
|
|
|
- String[] shellCmd = {"bash", "-c", sleepCommand};
|
|
|
- shexc = new Shell.ShellCommandExecutor(shellCmd);
|
|
|
- shexc.execute();
|
|
|
-
|
|
|
- return localPath;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- class ContainerLocalizerWrapper {
|
|
|
- AbstractFileSystem spylfs;
|
|
|
- Random random;
|
|
|
- List<Path> localDirs;
|
|
|
- Path tokenPath;
|
|
|
- LocalizationProtocol nmProxy;
|
|
|
-
|
|
|
- @SuppressWarnings("unchecked") // mocked generics
|
|
|
- FakeContainerLocalizer setupContainerLocalizerForTest()
|
|
|
- throws Exception {
|
|
|
-
|
|
|
- FileContext fs = FileContext.getLocalFSFileContext();
|
|
|
- spylfs = spy(fs.getDefaultFileSystem());
|
|
|
- // don't actually create dirs
|
|
|
- doNothing().when(spylfs).mkdir(
|
|
|
- isA(Path.class), isA(FsPermission.class), anyBoolean());
|
|
|
-
|
|
|
- Configuration conf = new Configuration();
|
|
|
- FileContext lfs = FileContext.getFileContext(spylfs, conf);
|
|
|
- localDirs = new ArrayList<Path>();
|
|
|
- for (int i = 0; i < 4; ++i) {
|
|
|
- localDirs.add(lfs.makeQualified(new Path(basedir, i + "")));
|
|
|
- }
|
|
|
- RecordFactory mockRF = getMockLocalizerRecordFactory();
|
|
|
- FakeContainerLocalizer concreteLoc = new FakeContainerLocalizer(lfs,
|
|
|
- appUser, appId, containerId, localDirs, mockRF);
|
|
|
- FakeContainerLocalizer localizer = spy(concreteLoc);
|
|
|
-
|
|
|
- // return credential stream instead of opening local file
|
|
|
- random = new Random();
|
|
|
- long seed = random.nextLong();
|
|
|
- System.out.println("SEED: " + seed);
|
|
|
- random.setSeed(seed);
|
|
|
- DataInputBuffer appTokens = createFakeCredentials(random, 10);
|
|
|
- tokenPath =
|
|
|
- lfs.makeQualified(new Path(
|
|
|
- String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT,
|
|
|
- containerId)));
|
|
|
- doReturn(new FSDataInputStream(new FakeFSDataInputStream(appTokens))
|
|
|
- ).when(spylfs).open(tokenPath);
|
|
|
- nmProxy = mock(LocalizationProtocol.class);
|
|
|
- doReturn(nmProxy).when(localizer).getProxy(nmAddr);
|
|
|
- doNothing().when(localizer).sleep(anyInt());
|
|
|
-
|
|
|
- return localizer;
|
|
|
- }
|
|
|
-
|
|
|
- }
|
|
|
-
|
|
|
- class FakeContainerLocalizerWrapper extends ContainerLocalizerWrapper{
|
|
|
- private int heartbeatResponse = 0;
|
|
|
- public FakeContainerLocalizer init() throws Exception {
|
|
|
- FileContext fs = FileContext.getLocalFSFileContext();
|
|
|
- FakeContainerLocalizer localizer = setupContainerLocalizerForTest();
|
|
|
-
|
|
|
- // verify created cache
|
|
|
- List<Path> privCacheList = new ArrayList<Path>();
|
|
|
- for (Path p : localDirs) {
|
|
|
- Path base = new Path(new Path(p, ContainerLocalizer.USERCACHE),
|
|
|
- appUser);
|
|
|
- Path privcache = new Path(base, ContainerLocalizer.FILECACHE);
|
|
|
- privCacheList.add(privcache);
|
|
|
- }
|
|
|
-
|
|
|
- final ResourceLocalizationSpec rsrc = getMockRsrc(random,
|
|
|
- LocalResourceVisibility.PRIVATE, privCacheList.get(0));
|
|
|
-
|
|
|
- // mock heartbeat responses from NM
|
|
|
- doAnswer(new Answer<MockLocalizerHeartbeatResponse>() {
|
|
|
- @Override
|
|
|
- public MockLocalizerHeartbeatResponse answer(
|
|
|
- InvocationOnMock invocationOnMock) throws Throwable {
|
|
|
- if(heartbeatResponse == 0) {
|
|
|
- return new MockLocalizerHeartbeatResponse(LocalizerAction.LIVE,
|
|
|
- Collections.singletonList(rsrc));
|
|
|
- } else if (heartbeatResponse < 2) {
|
|
|
- return new MockLocalizerHeartbeatResponse(LocalizerAction.LIVE,
|
|
|
- Collections.<ResourceLocalizationSpec>emptyList());
|
|
|
- } else {
|
|
|
- return new MockLocalizerHeartbeatResponse(LocalizerAction.DIE,
|
|
|
- null);
|
|
|
- }
|
|
|
- }
|
|
|
- }).when(nmProxy).heartbeat(isA(LocalizerStatus.class));
|
|
|
-
|
|
|
- return localizer;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
static RecordFactory getMockLocalizerRecordFactory() {
|
|
|
RecordFactory mockRF = mock(RecordFactory.class);
|
|
|
when(mockRF.newRecordInstance(same(LocalResourceStatus.class)))
|