HDFS-8915. TestFSNamesystem.testFSLockGetWaiterCount fails intermittently. Contributed by Masatake Iwasaki.

(cherry picked from commit 13fb1b50e6)
This commit is contained in:
Kihwal Lee 2016-08-26 10:00:36 -05:00
parent 026027166e
commit 2a6497cc7c
1 changed files with 12 additions and 3 deletions

View File

@ -30,6 +30,7 @@ import java.net.InetAddress;
import java.net.URI;
import java.util.Collection;
import com.google.common.base.Supplier;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileUtil;
@ -56,6 +57,7 @@ import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeoutException;
public class TestFSNamesystem {
@ -271,9 +273,16 @@ public class TestFSNamesystem {
}
latch.await();
Thread.sleep(10); // Lets all threads get BLOCKED
Assert.assertEquals("Expected number of blocked thread not found",
threadCount, rwLock.getQueueLength());
try {
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
return (threadCount == rwLock.getQueueLength());
}
}, 10, 1000);
} catch (TimeoutException e) {
fail("Expected number of blocked thread not found");
}
}
/**