Browse Source

YARN-4380. TestResourceLocalizationService.testDownloadingResourcesOnContainerKill fails intermittently. Contributed by Varun Saxena.

(cherry picked from commit 0656d2dc83af6a48a8d8d0e37cdf1f813124f366)
(cherry picked from commit d76b523b020fcd76e9abdd661967ca4a931d9863)

Conflicts:

	hadoop-yarn-project/CHANGES.txt
Tsuyoshi Ozawa 9 years ago
parent
commit
4be18edf2f

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -37,6 +37,9 @@ Release 2.6.4 - UNRELEASED
     YARN-4354. Public resource localization fails with NPE. (Jason Lowe via
     YARN-4354. Public resource localization fails with NPE. (Jason Lowe via
     junping_du)
     junping_du)
 
 
+    YARN-4380. TestResourceLocalizationService.testDownloadingResourcesOnContainerKill
+    fails intermittently. (Varun Saxena via ozawa)
+
 Release 2.6.3 - 2015-12-17
 Release 2.6.3 - 2015-12-17
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 17 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java

@@ -61,6 +61,7 @@ import java.util.Set;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.Future;
 import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
 
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -1104,16 +1105,23 @@ public class TestResourceLocalizationService {
 
 
   private static class DummyExecutor extends DefaultContainerExecutor {
   private static class DummyExecutor extends DefaultContainerExecutor {
     private volatile boolean stopLocalization = false;
     private volatile boolean stopLocalization = false;
+    private AtomicInteger numLocalizers = new AtomicInteger(0);
     @Override
     @Override
     public void startLocalizer(Path nmPrivateContainerTokensPath,
     public void startLocalizer(Path nmPrivateContainerTokensPath,
         InetSocketAddress nmAddr, String user, String appId, String locId,
         InetSocketAddress nmAddr, String user, String appId, String locId,
         LocalDirsHandlerService dirsHandler)
         LocalDirsHandlerService dirsHandler)
         throws IOException, InterruptedException {
         throws IOException, InterruptedException {
+      numLocalizers.incrementAndGet();
       while (!stopLocalization) {
       while (!stopLocalization) {
         Thread.yield();
         Thread.yield();
       }
       }
     }
     }
-    void setStopLocalization() {
+    private void waitForLocalizers(int num) {
+      while (numLocalizers.intValue() < num) {
+        Thread.yield();
+      }
+    }
+    private void setStopLocalization() {
       stopLocalization = true;
       stopLocalization = true;
     }
     }
   }
   }
@@ -1256,6 +1264,10 @@ public class TestResourceLocalizationService {
       spyService.handle(new ContainerLocalizationRequestEvent(c2, rsrcs1));
       spyService.handle(new ContainerLocalizationRequestEvent(c2, rsrcs1));
 
 
       dispatcher.await();
       dispatcher.await();
+      // Wait for localizers of both container c1 and c2 to begin.
+      exec.waitForLocalizers(2);
+      LocalizerRunner locC1 =
+          spyService.getLocalizerRunner(c1.getContainerId().toString());
       final String containerIdStr = c1.getContainerId().toString();
       final String containerIdStr = c1.getContainerId().toString();
       // Heartbeats from container localizer
       // Heartbeats from container localizer
       LocalResourceStatus rsrc1success = mock(LocalResourceStatus.class);
       LocalResourceStatus rsrc1success = mock(LocalResourceStatus.class);
@@ -1323,6 +1335,10 @@ public class TestResourceLocalizationService {
       Set<Path> paths =
       Set<Path> paths =
           Sets.newHashSet(new Path(locPath1), new Path(locPath1 + "_tmp"),
           Sets.newHashSet(new Path(locPath1), new Path(locPath1 + "_tmp"),
               new Path(locPath2), new Path(locPath2 + "_tmp"));
               new Path(locPath2), new Path(locPath2 + "_tmp"));
+      // Wait for localizer runner thread for container c1 to finish.
+      while (locC1.getState() != Thread.State.TERMINATED) {
+        Thread.sleep(50);
+      }
       // Verify if downloading resources were submitted for deletion.
       // Verify if downloading resources were submitted for deletion.
       verify(delService).delete(eq(user),
       verify(delService).delete(eq(user),
           (Path) eq(null), argThat(new DownloadingPathsMatcher(paths)));
           (Path) eq(null), argThat(new DownloadingPathsMatcher(paths)));