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

This commit is contained in:
Mingliang Liu 2016-08-15 20:28:40 -07:00
parent 9336a0495f
commit 4d4d95fdd5

View File

@ -1013,6 +1013,7 @@ public void testBlockReportQueueing() throws Exception {
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 void testBlockReportQueueing() throws Exception {
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 Void call()
return null;
}
});
// signal that runBlockOp returned
doneLatch.countDown();
return null;
}
});
@ -1074,7 +1078,7 @@ public void run() {
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();
}