|
@@ -34,9 +34,9 @@ import static org.mockito.Mockito.doReturn;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.spy;
|
|
|
import static org.mockito.Mockito.timeout;
|
|
|
+import static org.mockito.Mockito.times;
|
|
|
import static org.mockito.Mockito.verify;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
-import static org.mockito.Mockito.times;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
@@ -53,6 +53,7 @@ import java.util.Random;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.BrokenBarrierException;
|
|
|
import java.util.concurrent.CyclicBarrier;
|
|
|
+import java.util.concurrent.Future;
|
|
|
|
|
|
import junit.framework.Assert;
|
|
|
|
|
@@ -82,6 +83,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
|
import org.apache.hadoop.yarn.event.DrainDispatcher;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
+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;
|
|
@@ -90,20 +92,28 @@ import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAc
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalizerStatusPBImpl;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerResourceFailedEvent;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.LocalizerRunner;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.LocalizerTracker;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ApplicationLocalizationEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationCleanupEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationRequestEvent;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizerEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizerEventType;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizerResourceRequestEvent;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceFailedLocalizationEvent;
|
|
|
import org.apache.hadoop.yarn.util.BuilderUtils;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
import org.junit.BeforeClass;
|
|
@@ -677,6 +687,481 @@ public class TestResourceLocalizationService {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 100000)
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ public void testParallelDownloadAttemptsForPrivateResource() throws Exception {
|
|
|
+
|
|
|
+ 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));
|
|
|
+
|
|
|
+ LocalResourceRequest req =
|
|
|
+ new LocalResourceRequest(new Path("file:///tmp"), 123L,
|
|
|
+ LocalResourceType.FILE, LocalResourceVisibility.PRIVATE, "");
|
|
|
+
|
|
|
+ // We need to pre-populate the LocalizerRunner as the
|
|
|
+ // Resource Localization Service code internally starts them which
|
|
|
+ // definitely we don't want.
|
|
|
+
|
|
|
+ // creating new containers and populating corresponding localizer runners
|
|
|
+
|
|
|
+ // Container - 1
|
|
|
+ ContainerImpl container1 = createMockContainer(user, 1);
|
|
|
+ String localizerId1 = container1.getContainerID().toString();
|
|
|
+ rls.getPrivateLocalizers().put(
|
|
|
+ localizerId1,
|
|
|
+ rls.new LocalizerRunner(new LocalizerContext(user, container1
|
|
|
+ .getContainerID(), null), localizerId1));
|
|
|
+ LocalizerRunner localizerRunner1 = rls.getLocalizerRunner(localizerId1);
|
|
|
+
|
|
|
+ dispatcher1.getEventHandler().handle(
|
|
|
+ createContainerLocalizationEvent(container1,
|
|
|
+ LocalResourceVisibility.PRIVATE, req));
|
|
|
+ Assert
|
|
|
+ .assertTrue(waitForPrivateDownloadToStart(rls, localizerId1, 1, 200));
|
|
|
+
|
|
|
+ // Container - 2 now makes the request.
|
|
|
+ ContainerImpl container2 = createMockContainer(user, 2);
|
|
|
+ String localizerId2 = container2.getContainerID().toString();
|
|
|
+ rls.getPrivateLocalizers().put(
|
|
|
+ localizerId2,
|
|
|
+ rls.new LocalizerRunner(new LocalizerContext(user, container2
|
|
|
+ .getContainerID(), null), localizerId2));
|
|
|
+ LocalizerRunner localizerRunner2 = rls.getLocalizerRunner(localizerId2);
|
|
|
+ dispatcher1.getEventHandler().handle(
|
|
|
+ createContainerLocalizationEvent(container2,
|
|
|
+ LocalResourceVisibility.PRIVATE, req));
|
|
|
+ Assert
|
|
|
+ .assertTrue(waitForPrivateDownloadToStart(rls, localizerId2, 1, 200));
|
|
|
+
|
|
|
+ // Retrieving localized resource.
|
|
|
+ LocalResourcesTracker tracker =
|
|
|
+ rls.getLocalResourcesTracker(LocalResourceVisibility.PRIVATE, user,
|
|
|
+ appId);
|
|
|
+ LocalizedResource lr = tracker.getLocalizedResource(req);
|
|
|
+ // Resource would now have moved into DOWNLOADING state
|
|
|
+ Assert.assertEquals(ResourceState.DOWNLOADING, lr.getState());
|
|
|
+ // Resource should have one permit
|
|
|
+ Assert.assertEquals(1, lr.sem.availablePermits());
|
|
|
+
|
|
|
+ // Resource Localization Service receives first heart beat from
|
|
|
+ // ContainerLocalizer for container1
|
|
|
+ LocalizerHeartbeatResponse response1 =
|
|
|
+ rls.heartbeat(createLocalizerStatus(localizerId1));
|
|
|
+
|
|
|
+ // Resource must have been added to scheduled map
|
|
|
+ Assert.assertEquals(1, localizerRunner1.scheduled.size());
|
|
|
+ // Checking resource in the response and also available permits for it.
|
|
|
+ Assert.assertEquals(req.getResource(), response1.getResourceSpecs()
|
|
|
+ .get(0).getResource().getResource());
|
|
|
+ Assert.assertEquals(0, lr.sem.availablePermits());
|
|
|
+
|
|
|
+ // Resource Localization Service now receives first heart beat from
|
|
|
+ // ContainerLocalizer for container2
|
|
|
+ LocalizerHeartbeatResponse response2 =
|
|
|
+ rls.heartbeat(createLocalizerStatus(localizerId2));
|
|
|
+
|
|
|
+ // Resource must not have been added to scheduled map
|
|
|
+ Assert.assertEquals(0, localizerRunner2.scheduled.size());
|
|
|
+ // No resource is returned in response
|
|
|
+ Assert.assertEquals(0, response2.getResourceSpecs().size());
|
|
|
+
|
|
|
+ // ContainerLocalizer - 1 now sends failed resource heartbeat.
|
|
|
+ rls.heartbeat(createLocalizerStatusForFailedResource(localizerId1, req));
|
|
|
+
|
|
|
+ // Resource Localization should fail and state is modified accordingly.
|
|
|
+ // Also Local should be release on the LocalizedResource.
|
|
|
+ Assert
|
|
|
+ .assertTrue(waitForResourceState(lr, rls, req,
|
|
|
+ LocalResourceVisibility.PRIVATE, user, appId, ResourceState.FAILED,
|
|
|
+ 200));
|
|
|
+ Assert.assertTrue(lr.getState().equals(ResourceState.FAILED));
|
|
|
+ Assert.assertEquals(0, localizerRunner1.scheduled.size());
|
|
|
+
|
|
|
+ // Now Container-2 once again sends heart beat to resource localization
|
|
|
+ // service
|
|
|
+
|
|
|
+ // Now container-2 again try to download the resource it should still
|
|
|
+ // not get the resource as the resource is now not in DOWNLOADING state.
|
|
|
+ response2 = rls.heartbeat(createLocalizerStatus(localizerId2));
|
|
|
+
|
|
|
+ // Resource must not have been added to scheduled map.
|
|
|
+ // Also as the resource has failed download it will be removed from
|
|
|
+ // pending list.
|
|
|
+ Assert.assertEquals(0, localizerRunner2.scheduled.size());
|
|
|
+ Assert.assertEquals(0, localizerRunner2.pending.size());
|
|
|
+ Assert.assertEquals(0, response2.getResourceSpecs().size());
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ if (dispatcher1 != null) {
|
|
|
+ dispatcher1.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private LocalizerStatus createLocalizerStatusForFailedResource(
|
|
|
+ String localizerId, LocalResourceRequest req) {
|
|
|
+ LocalizerStatus status = createLocalizerStatus(localizerId);
|
|
|
+ LocalResourceStatus resourceStatus = new LocalResourceStatusPBImpl();
|
|
|
+ resourceStatus.setException(new YarnRemoteExceptionPBImpl("test"));
|
|
|
+ resourceStatus.setStatus(ResourceStatusType.FETCH_FAILURE);
|
|
|
+ resourceStatus.setResource(req);
|
|
|
+ status.addResourceStatus(resourceStatus);
|
|
|
+ return status;
|
|
|
+ }
|
|
|
+
|
|
|
+ private LocalizerStatus createLocalizerStatus(String localizerId1) {
|
|
|
+ LocalizerStatus status = new LocalizerStatusPBImpl();
|
|
|
+ status.setLocalizerId(localizerId1);
|
|
|
+ return status;
|
|
|
+ }
|
|
|
+
|
|
|
+ private LocalizationEvent createApplicationLocalizationEvent(String user,
|
|
|
+ ApplicationId appId) {
|
|
|
+ Application app = mock(Application.class);
|
|
|
+ when(app.getUser()).thenReturn(user);
|
|
|
+ when(app.getAppId()).thenReturn(appId);
|
|
|
+ return new ApplicationLocalizationEvent(
|
|
|
+ LocalizationEventType.INIT_APPLICATION_RESOURCES, app);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test(timeout = 100000)
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ public void testParallelDownloadAttemptsForPublicResource() throws Exception {
|
|
|
+
|
|
|
+ DrainDispatcher dispatcher1 = null;
|
|
|
+ String user = "testuser";
|
|
|
+ try {
|
|
|
+ // Setting up ResourceLocalization service.
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ dispatcher1 = new DrainDispatcher();
|
|
|
+ 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);
|
|
|
+
|
|
|
+ // 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();
|
|
|
+
|
|
|
+ // Creating and initializing ResourceLocalizationService but not starting
|
|
|
+ // it as otherwise it will remove requests from pending queue.
|
|
|
+ ResourceLocalizationService rawService =
|
|
|
+ new ResourceLocalizationService(dispatcher1, exec, delService,
|
|
|
+ dirsHandler);
|
|
|
+ ResourceLocalizationService spyService = spy(rawService);
|
|
|
+ dispatcher1.register(LocalizationEventType.class, spyService);
|
|
|
+ spyService.init(conf);
|
|
|
+
|
|
|
+ // Initially pending map should be empty for public localizer
|
|
|
+ Assert.assertEquals(0, spyService.getPublicLocalizer().pending.size());
|
|
|
+
|
|
|
+ LocalResourceRequest req =
|
|
|
+ new LocalResourceRequest(new Path("/tmp"), 123L,
|
|
|
+ LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, "");
|
|
|
+
|
|
|
+ // Initializing application
|
|
|
+ ApplicationImpl app = mock(ApplicationImpl.class);
|
|
|
+ ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
|
|
|
+ when(app.getAppId()).thenReturn(appId);
|
|
|
+ when(app.getUser()).thenReturn(user);
|
|
|
+ dispatcher1.getEventHandler().handle(
|
|
|
+ new ApplicationLocalizationEvent(
|
|
|
+ LocalizationEventType.INIT_APPLICATION_RESOURCES, app));
|
|
|
+
|
|
|
+ // Container - 1
|
|
|
+
|
|
|
+ // container requesting the resource
|
|
|
+ ContainerImpl container1 = createMockContainer(user, 1);
|
|
|
+ dispatcher1.getEventHandler().handle(
|
|
|
+ createContainerLocalizationEvent(container1,
|
|
|
+ LocalResourceVisibility.PUBLIC, req));
|
|
|
+
|
|
|
+ // Waiting for resource to change into DOWNLOADING state.
|
|
|
+ Assert.assertTrue(waitForResourceState(null, spyService, req,
|
|
|
+ LocalResourceVisibility.PUBLIC, user, null, ResourceState.DOWNLOADING,
|
|
|
+ 200));
|
|
|
+
|
|
|
+ // Waiting for download to start.
|
|
|
+ Assert.assertTrue(waitForPublicDownloadToStart(spyService, 1, 200));
|
|
|
+
|
|
|
+ LocalizedResource lr =
|
|
|
+ getLocalizedResource(spyService, req, LocalResourceVisibility.PUBLIC,
|
|
|
+ user, null);
|
|
|
+ // Resource would now have moved into DOWNLOADING state
|
|
|
+ Assert.assertEquals(ResourceState.DOWNLOADING, lr.getState());
|
|
|
+
|
|
|
+ // pending should have this resource now.
|
|
|
+ Assert.assertEquals(1, spyService.getPublicLocalizer().pending.size());
|
|
|
+ // Now resource should have 0 permit.
|
|
|
+ Assert.assertEquals(0, lr.sem.availablePermits());
|
|
|
+
|
|
|
+ // Container - 2
|
|
|
+
|
|
|
+ // Container requesting the same resource.
|
|
|
+ ContainerImpl container2 = createMockContainer(user, 2);
|
|
|
+ dispatcher1.getEventHandler().handle(
|
|
|
+ createContainerLocalizationEvent(container2,
|
|
|
+ LocalResourceVisibility.PUBLIC, req));
|
|
|
+
|
|
|
+ // Waiting for download to start. This should return false as new download
|
|
|
+ // will not start
|
|
|
+ Assert.assertFalse(waitForPublicDownloadToStart(spyService, 2, 100));
|
|
|
+
|
|
|
+ // Now Failing the resource download. As a part of it
|
|
|
+ // resource state is changed and then lock is released.
|
|
|
+ ResourceFailedLocalizationEvent locFailedEvent =
|
|
|
+ new ResourceFailedLocalizationEvent(req, new Exception("test"));
|
|
|
+ spyService.getLocalResourcesTracker(LocalResourceVisibility.PUBLIC, user,
|
|
|
+ null).handle(locFailedEvent);
|
|
|
+
|
|
|
+ // Waiting for resource to change into FAILED state.
|
|
|
+ Assert.assertTrue(waitForResourceState(lr, spyService, req,
|
|
|
+ LocalResourceVisibility.PUBLIC, user, null, ResourceState.FAILED, 200));
|
|
|
+ // releasing lock as a part of download failed process.
|
|
|
+ lr.unlock();
|
|
|
+ // removing pending download request.
|
|
|
+ spyService.getPublicLocalizer().pending.clear();
|
|
|
+
|
|
|
+ // Now I need to simulate a race condition wherein Event is added to
|
|
|
+ // dispatcher before resource state changes to either FAILED or LOCALIZED
|
|
|
+ // Hence sending event directly to dispatcher.
|
|
|
+ LocalizerResourceRequestEvent localizerEvent =
|
|
|
+ new LocalizerResourceRequestEvent(lr, null,
|
|
|
+ mock(LocalizerContext.class), null);
|
|
|
+
|
|
|
+ dispatcher1.getEventHandler().handle(localizerEvent);
|
|
|
+ // Waiting for download to start. This should return false as new download
|
|
|
+ // will not start
|
|
|
+ Assert.assertFalse(waitForPublicDownloadToStart(spyService, 1, 100));
|
|
|
+ // Checking available permits now.
|
|
|
+ Assert.assertEquals(1, lr.sem.availablePermits());
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ if (dispatcher1 != null) {
|
|
|
+ dispatcher1.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean waitForPrivateDownloadToStart(
|
|
|
+ ResourceLocalizationService service, String localizerId, int size,
|
|
|
+ int maxWaitTime) {
|
|
|
+ List<LocalizerResourceRequestEvent> pending = null;
|
|
|
+ // Waiting for localizer to be created.
|
|
|
+ do {
|
|
|
+ if (service.getPrivateLocalizers().get(localizerId) != null) {
|
|
|
+ pending = service.getPrivateLocalizers().get(localizerId).pending;
|
|
|
+ }
|
|
|
+ if (pending == null) {
|
|
|
+ try {
|
|
|
+ maxWaitTime -= 20;
|
|
|
+ Thread.sleep(20);
|
|
|
+ } catch (Exception e) {
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ } while (maxWaitTime > 0);
|
|
|
+ if (pending == null) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ do {
|
|
|
+ if (pending.size() == size) {
|
|
|
+ return true;
|
|
|
+ } else {
|
|
|
+ try {
|
|
|
+ maxWaitTime -= 20;
|
|
|
+ Thread.sleep(20);
|
|
|
+ } catch (Exception e) {
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } while (maxWaitTime > 0);
|
|
|
+ return pending.size() == size;
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean waitForPublicDownloadToStart(
|
|
|
+ ResourceLocalizationService service, int size, int maxWaitTime) {
|
|
|
+ Map<Future<Path>, LocalizerResourceRequestEvent> pending = null;
|
|
|
+ // Waiting for localizer to be created.
|
|
|
+ do {
|
|
|
+ if (service.getPublicLocalizer() != null) {
|
|
|
+ pending = service.getPublicLocalizer().pending;
|
|
|
+ }
|
|
|
+ if (pending == null) {
|
|
|
+ try {
|
|
|
+ maxWaitTime -= 20;
|
|
|
+ Thread.sleep(20);
|
|
|
+ } catch (Exception e) {
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ } while (maxWaitTime > 0);
|
|
|
+ if (pending == null) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ do {
|
|
|
+ if (pending.size() == size) {
|
|
|
+ return true;
|
|
|
+ } else {
|
|
|
+ try {
|
|
|
+ maxWaitTime -= 20;
|
|
|
+ Thread.sleep(20);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } while (maxWaitTime > 0);
|
|
|
+ return pending.size() == size;
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ private LocalizedResource getLocalizedResource(
|
|
|
+ ResourceLocalizationService service, LocalResourceRequest req,
|
|
|
+ LocalResourceVisibility vis, String user, ApplicationId appId) {
|
|
|
+ return service.getLocalResourcesTracker(vis, user, appId)
|
|
|
+ .getLocalizedResource(req);
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean waitForResourceState(LocalizedResource lr,
|
|
|
+ ResourceLocalizationService service, LocalResourceRequest req,
|
|
|
+ LocalResourceVisibility vis, String user, ApplicationId appId,
|
|
|
+ ResourceState resourceState, long maxWaitTime) {
|
|
|
+ LocalResourcesTracker tracker = null;
|
|
|
+ // checking tracker is created
|
|
|
+ do {
|
|
|
+ if (tracker == null) {
|
|
|
+ tracker = service.getLocalResourcesTracker(vis, user, appId);
|
|
|
+ }
|
|
|
+ if (tracker != null && lr == null) {
|
|
|
+ lr = tracker.getLocalizedResource(req);
|
|
|
+ }
|
|
|
+ if (lr != null) {
|
|
|
+ break;
|
|
|
+ } else {
|
|
|
+ try {
|
|
|
+ maxWaitTime -= 20;
|
|
|
+ Thread.sleep(20);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } while (maxWaitTime > 0);
|
|
|
+ // this will wait till resource state is changed to (resourceState).
|
|
|
+ if (lr == null) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ do {
|
|
|
+ if (!lr.getState().equals(resourceState)) {
|
|
|
+ try {
|
|
|
+ maxWaitTime -= 50;
|
|
|
+ Thread.sleep(50);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ } while (maxWaitTime > 0);
|
|
|
+ return lr.getState().equals(resourceState);
|
|
|
+ }
|
|
|
+
|
|
|
+ private ContainerLocalizationRequestEvent createContainerLocalizationEvent(
|
|
|
+ ContainerImpl container, LocalResourceVisibility vis,
|
|
|
+ LocalResourceRequest req) {
|
|
|
+ Map<LocalResourceVisibility, Collection<LocalResourceRequest>> reqs =
|
|
|
+ new HashMap<LocalResourceVisibility, Collection<LocalResourceRequest>>();
|
|
|
+ List<LocalResourceRequest> resourceList =
|
|
|
+ new ArrayList<LocalResourceRequest>();
|
|
|
+ resourceList.add(req);
|
|
|
+ reqs.put(vis, resourceList);
|
|
|
+ return new ContainerLocalizationRequestEvent(container, reqs);
|
|
|
+ }
|
|
|
+
|
|
|
+ private ContainerImpl createMockContainer(String user, int containerId) {
|
|
|
+ ContainerImpl container = mock(ContainerImpl.class);
|
|
|
+ when(container.getContainerID()).thenReturn(
|
|
|
+ BuilderUtils.newContainerId(1, 1, 1, containerId));
|
|
|
+ when(container.getUser()).thenReturn(user);
|
|
|
+ Credentials mockCredentials = mock(Credentials.class);
|
|
|
+ when(container.getCredentials()).thenReturn(mockCredentials);
|
|
|
+ return container;
|
|
|
+ }
|
|
|
+
|
|
|
private static URL getPath(String path) {
|
|
|
URL url = BuilderUtils.newURL("file", null, 0, path);
|
|
|
return url;
|