Browse Source

HDFS-10641. TestBlockManager#testBlockReportQueueing fails intermittently. (Contributed by Daryn Sharp)

Mingliang Liu 8 years ago
parent
commit
4d4d95fdd5

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java

@@ -1013,6 +1013,7 @@ public class TestBlockManager {
 
       final CyclicBarrier startBarrier = new CyclicBarrier(2);
       final CountDownLatch endLatch = new CountDownLatch(3);
+      final CountDownLatch doneLatch = new CountDownLatch(1);
 
       // create a task intended to block while processing, thus causing
       // the queue to backup.  simulates how a full BR is processed.
@@ -1020,7 +1021,7 @@ public class TestBlockManager {
           new Callable<Void>(){
             @Override
             public Void call() throws IOException {
-              return bm.runBlockOp(new Callable<Void>() {
+              bm.runBlockOp(new Callable<Void>() {
                 @Override
                 public Void call()
                     throws InterruptedException, BrokenBarrierException {
@@ -1030,6 +1031,9 @@ public class TestBlockManager {
                   return null;
                 }
               });
+              // signal that runBlockOp returned
+              doneLatch.countDown();
+              return null;
             }
           });
 
@@ -1074,7 +1078,7 @@ public class TestBlockManager {
       startBarrier.await(1, TimeUnit.SECONDS);
       assertTrue(endLatch.await(1, TimeUnit.SECONDS));
       assertEquals(0, bm.getBlockOpQueueLength());
-      assertTrue(blockingOp.isDone());
+      assertTrue(doneLatch.await(1, TimeUnit.SECONDS));
     } finally {
       cluster.shutdown();
     }