Преглед изворни кода

YARN-5857. TestLogAggregationService.testFixedSizeThreadPool fails intermittently on trunk. Contributed by Bilwa S T.

bibinchundatt пре 5 година
родитељ
комит
e684b17e6f

+ 6 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java

@@ -58,6 +58,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -1215,6 +1216,8 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
 
     ExecutorService executorService = logAggregationService.threadPool;
 
+    CountDownLatch latch = new CountDownLatch(threadPoolSize);
+
     // used to block threads in the thread pool because main thread always
     // acquires the write lock first.
     final ReadWriteLock rwLock = new ReentrantReadWriteLock();
@@ -1227,8 +1230,9 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         @Override
         public void run() {
           try {
+            latch.countDown();
             // threads in the thread pool running this will be blocked
-            rLock.tryLock(35000, TimeUnit.MILLISECONDS);
+            rLock.tryLock(15000, TimeUnit.MILLISECONDS);
           } catch (InterruptedException e) {
             e.printStackTrace();
           } finally {
@@ -1244,6 +1248,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         executorService.submit(runnable);
       }
 
+      latch.await();
       // count the number of current running LogAggregationService threads
       int runningThread = ((ThreadPoolExecutor) executorService).getActiveCount();
       assertEquals(threadPoolSize, runningThread);