|
@@ -42,6 +42,7 @@ import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.URI;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
|
import java.util.Collections;
|
|
|
import java.util.EnumSet;
|
|
@@ -72,6 +73,7 @@ import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
|
import org.apache.hadoop.util.Progressable;
|
|
|
+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;
|
|
@@ -87,6 +89,7 @@ import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
|
|
@@ -491,7 +494,7 @@ public class TestResourceLocalizationService {
|
|
|
Thread.sleep(1000);
|
|
|
dispatcher.await();
|
|
|
String appStr = ConverterUtils.toString(appId);
|
|
|
- String ctnrStr = c.getContainerID().toString();
|
|
|
+ String ctnrStr = c.getContainer().getId().toString();
|
|
|
ArgumentCaptor<Path> tokenPathCaptor = ArgumentCaptor.forClass(Path.class);
|
|
|
verify(exec).startLocalizer(tokenPathCaptor.capture(),
|
|
|
isA(InetSocketAddress.class), eq("user0"), eq(appStr), eq(ctnrStr),
|
|
@@ -567,7 +570,7 @@ public class TestResourceLocalizationService {
|
|
|
public boolean matches(Object o) {
|
|
|
ContainerEvent evt = (ContainerEvent) o;
|
|
|
return evt.getType() == ContainerEventType.RESOURCE_LOCALIZED
|
|
|
- && c.getContainerID() == evt.getContainerID();
|
|
|
+ && c.getContainer().getId() == evt.getContainerID();
|
|
|
}
|
|
|
};
|
|
|
// total 2 resource localzation calls. one for each resource.
|
|
@@ -756,11 +759,11 @@ public class TestResourceLocalizationService {
|
|
|
|
|
|
// Container - 1
|
|
|
ContainerImpl container1 = createMockContainer(user, 1);
|
|
|
- String localizerId1 = container1.getContainerID().toString();
|
|
|
+ String localizerId1 = container1.getContainer().getId().toString();
|
|
|
rls.getPrivateLocalizers().put(
|
|
|
localizerId1,
|
|
|
rls.new LocalizerRunner(new LocalizerContext(user, container1
|
|
|
- .getContainerID(), null), localizerId1));
|
|
|
+ .getContainer().getId(), null), localizerId1));
|
|
|
LocalizerRunner localizerRunner1 = rls.getLocalizerRunner(localizerId1);
|
|
|
|
|
|
dispatcher1.getEventHandler().handle(
|
|
@@ -771,11 +774,11 @@ public class TestResourceLocalizationService {
|
|
|
|
|
|
// Container - 2 now makes the request.
|
|
|
ContainerImpl container2 = createMockContainer(user, 2);
|
|
|
- String localizerId2 = container2.getContainerID().toString();
|
|
|
+ String localizerId2 = container2.getContainer().getId().toString();
|
|
|
rls.getPrivateLocalizers().put(
|
|
|
localizerId2,
|
|
|
rls.new LocalizerRunner(new LocalizerContext(user, container2
|
|
|
- .getContainerID(), null), localizerId2));
|
|
|
+ .getContainer().getId(), null), localizerId2));
|
|
|
LocalizerRunner localizerRunner2 = rls.getLocalizerRunner(localizerId2);
|
|
|
dispatcher1.getEventHandler().handle(
|
|
|
createContainerLocalizationEvent(container2,
|
|
@@ -848,6 +851,163 @@ public class TestResourceLocalizationService {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 10000)
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ public void testLocalResourcePath() throws Exception {
|
|
|
+
|
|
|
+ // test the local path where application and user cache files will be
|
|
|
+ // localized.
|
|
|
+
|
|
|
+ DrainDispatcher dispatcher1 = null;
|
|
|
+ try {
|
|
|
+ dispatcher1 = new DrainDispatcher();
|
|
|
+ String user = "testuser";
|
|
|
+ ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
|
|
|
+
|
|
|
+ // mocked Resource Localization Service
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ AbstractFileSystem spylfs =
|
|
|
+ spy(FileContext.getLocalFSFileContext().getDefaultFileSystem());
|
|
|
+ final FileContext lfs = FileContext.getFileContext(spylfs, conf);
|
|
|
+ // We don't want files to be created
|
|
|
+ doNothing().when(spylfs).mkdir(isA(Path.class), isA(FsPermission.class),
|
|
|
+ anyBoolean());
|
|
|
+
|
|
|
+ // creating one local directory
|
|
|
+ List<Path> localDirs = new ArrayList<Path>();
|
|
|
+ String[] sDirs = new String[1];
|
|
|
+ for (int i = 0; i < 1; ++i) {
|
|
|
+ localDirs.add(lfs.makeQualified(new Path(basedir, i + "")));
|
|
|
+ sDirs[i] = localDirs.get(i).toString();
|
|
|
+ }
|
|
|
+ conf.setStrings(YarnConfiguration.NM_LOCAL_DIRS, sDirs);
|
|
|
+ // setting log directory.
|
|
|
+ String logDir =
|
|
|
+ lfs.makeQualified(new Path(basedir, "logdir ")).toString();
|
|
|
+ conf.set(YarnConfiguration.NM_LOG_DIRS, logDir);
|
|
|
+
|
|
|
+ LocalDirsHandlerService localDirHandler = new LocalDirsHandlerService();
|
|
|
+ localDirHandler.init(conf);
|
|
|
+ // Registering event handlers
|
|
|
+ EventHandler<ApplicationEvent> applicationBus = mock(EventHandler.class);
|
|
|
+ dispatcher1.register(ApplicationEventType.class, applicationBus);
|
|
|
+ EventHandler<ContainerEvent> containerBus = mock(EventHandler.class);
|
|
|
+ dispatcher1.register(ContainerEventType.class, containerBus);
|
|
|
+
|
|
|
+ ContainerExecutor exec = mock(ContainerExecutor.class);
|
|
|
+ DeletionService delService = mock(DeletionService.class);
|
|
|
+ LocalDirsHandlerService dirsHandler = new LocalDirsHandlerService();
|
|
|
+ // initializing directory handler.
|
|
|
+ dirsHandler.init(conf);
|
|
|
+
|
|
|
+ dispatcher1.init(conf);
|
|
|
+ dispatcher1.start();
|
|
|
+
|
|
|
+ ResourceLocalizationService rls =
|
|
|
+ new ResourceLocalizationService(dispatcher1, exec, delService,
|
|
|
+ localDirHandler);
|
|
|
+ dispatcher1.register(LocalizationEventType.class, rls);
|
|
|
+ rls.init(conf);
|
|
|
+
|
|
|
+ rls.handle(createApplicationLocalizationEvent(user, appId));
|
|
|
+
|
|
|
+ // We need to pre-populate the LocalizerRunner as the
|
|
|
+ // Resource Localization Service code internally starts them which
|
|
|
+ // definitely we don't want.
|
|
|
+
|
|
|
+ // creating new container and populating corresponding localizer runner
|
|
|
+
|
|
|
+ // Container - 1
|
|
|
+ Container container1 = createMockContainer(user, 1);
|
|
|
+ String localizerId1 = container1.getContainer().getId().toString();
|
|
|
+ rls.getPrivateLocalizers().put(
|
|
|
+ localizerId1,
|
|
|
+ rls.new LocalizerRunner(new LocalizerContext(user, container1
|
|
|
+ .getContainer().getId(), null), localizerId1));
|
|
|
+
|
|
|
+ // Creating two requests for container
|
|
|
+ // 1) Private resource
|
|
|
+ // 2) Application resource
|
|
|
+ LocalResourceRequest reqPriv =
|
|
|
+ new LocalResourceRequest(new Path("file:///tmp1"), 123L,
|
|
|
+ LocalResourceType.FILE, LocalResourceVisibility.PRIVATE, "");
|
|
|
+ List<LocalResourceRequest> privList =
|
|
|
+ new ArrayList<LocalResourceRequest>();
|
|
|
+ privList.add(reqPriv);
|
|
|
+
|
|
|
+ LocalResourceRequest reqApp =
|
|
|
+ new LocalResourceRequest(new Path("file:///tmp2"), 123L,
|
|
|
+ LocalResourceType.FILE, LocalResourceVisibility.APPLICATION, "");
|
|
|
+ List<LocalResourceRequest> appList =
|
|
|
+ new ArrayList<LocalResourceRequest>();
|
|
|
+ appList.add(reqApp);
|
|
|
+
|
|
|
+ Map<LocalResourceVisibility, Collection<LocalResourceRequest>> rsrcs =
|
|
|
+ new HashMap<LocalResourceVisibility, Collection<LocalResourceRequest>>();
|
|
|
+ rsrcs.put(LocalResourceVisibility.APPLICATION, appList);
|
|
|
+ rsrcs.put(LocalResourceVisibility.PRIVATE, privList);
|
|
|
+
|
|
|
+ dispatcher1.getEventHandler().handle(
|
|
|
+ new ContainerLocalizationRequestEvent(container1, rsrcs));
|
|
|
+
|
|
|
+ // Now waiting for resource download to start. Here actual will not start
|
|
|
+ // Only the resources will be populated into pending list.
|
|
|
+ Assert
|
|
|
+ .assertTrue(waitForPrivateDownloadToStart(rls, localizerId1, 2, 500));
|
|
|
+
|
|
|
+ // Validating user and application cache paths
|
|
|
+
|
|
|
+ String userCachePath =
|
|
|
+ StringUtils.join(Path.SEPARATOR, Arrays.asList(localDirs.get(0)
|
|
|
+ .toUri().getRawPath(), ContainerLocalizer.USERCACHE, user,
|
|
|
+ ContainerLocalizer.FILECACHE));
|
|
|
+ String userAppCachePath =
|
|
|
+ StringUtils.join(Path.SEPARATOR, Arrays.asList(localDirs.get(0)
|
|
|
+ .toUri().getRawPath(), ContainerLocalizer.USERCACHE, user,
|
|
|
+ ContainerLocalizer.APPCACHE, appId.toString(),
|
|
|
+ ContainerLocalizer.FILECACHE));
|
|
|
+
|
|
|
+ // Now the Application and private resources may come in any order
|
|
|
+ // for download.
|
|
|
+ // For User cahce :
|
|
|
+ // returned destinationPath = user cache path + random number
|
|
|
+ // For App cache :
|
|
|
+ // returned destinationPath = user app cache path + random number
|
|
|
+
|
|
|
+ int returnedResources = 0;
|
|
|
+ boolean appRsrc = false, privRsrc = false;
|
|
|
+ while (returnedResources < 2) {
|
|
|
+ LocalizerHeartbeatResponse response =
|
|
|
+ rls.heartbeat(createLocalizerStatus(localizerId1));
|
|
|
+ for (ResourceLocalizationSpec resourceSpec : response
|
|
|
+ .getResourceSpecs()) {
|
|
|
+ returnedResources++;
|
|
|
+ Path destinationDirectory =
|
|
|
+ new Path(resourceSpec.getDestinationDirectory().getFile());
|
|
|
+ if (resourceSpec.getResource().getVisibility() ==
|
|
|
+ LocalResourceVisibility.APPLICATION) {
|
|
|
+ appRsrc = true;
|
|
|
+ Assert.assertEquals(userAppCachePath, destinationDirectory
|
|
|
+ .getParent().toUri().toString());
|
|
|
+ } else if (resourceSpec.getResource().getVisibility() ==
|
|
|
+ LocalResourceVisibility.PRIVATE) {
|
|
|
+ privRsrc = true;
|
|
|
+ Assert.assertEquals(userCachePath, destinationDirectory.getParent()
|
|
|
+ .toUri().toString());
|
|
|
+ } else {
|
|
|
+ throw new Exception("Unexpected resource recevied.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ // We should receive both the resources (Application and Private)
|
|
|
+ Assert.assertTrue(appRsrc && privRsrc);
|
|
|
+ } finally {
|
|
|
+ if (dispatcher1 != null) {
|
|
|
+ dispatcher1.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private LocalizerStatus createLocalizerStatusForFailedResource(
|
|
|
String localizerId, LocalResourceRequest req) {
|
|
|
LocalizerStatus status = createLocalizerStatus(localizerId);
|
|
@@ -1154,7 +1314,10 @@ public class TestResourceLocalizationService {
|
|
|
|
|
|
private ContainerImpl createMockContainer(String user, int containerId) {
|
|
|
ContainerImpl container = mock(ContainerImpl.class);
|
|
|
- when(container.getContainerID()).thenReturn(
|
|
|
+ org.apache.hadoop.yarn.api.records.Container c =
|
|
|
+ mock(org.apache.hadoop.yarn.api.records.Container.class);
|
|
|
+ when(container.getContainer()).thenReturn(c);
|
|
|
+ when(container.getContainer().getId()).thenReturn(
|
|
|
BuilderUtils.newContainerId(1, 1, 1, containerId));
|
|
|
when(container.getUser()).thenReturn(user);
|
|
|
Credentials mockCredentials = mock(Credentials.class);
|
|
@@ -1194,8 +1357,11 @@ public class TestResourceLocalizationService {
|
|
|
ApplicationAttemptId appAttemptId =
|
|
|
BuilderUtils.newApplicationAttemptId(appId, 1);
|
|
|
ContainerId cId = BuilderUtils.newContainerId(appAttemptId, id);
|
|
|
+ org.apache.hadoop.yarn.api.records.Container containerAPI =
|
|
|
+ mock(org.apache.hadoop.yarn.api.records.Container.class);
|
|
|
+ when(c.getContainer()).thenReturn(containerAPI);
|
|
|
when(c.getUser()).thenReturn("user0");
|
|
|
- when(c.getContainerID()).thenReturn(cId);
|
|
|
+ when(c.getContainer().getId()).thenReturn(cId);
|
|
|
Credentials creds = new Credentials();
|
|
|
creds.addToken(new Text("tok" + id), getToken(id));
|
|
|
when(c.getCredentials()).thenReturn(creds);
|