HDFS-8915. TestFSNamesystem.testFSLockGetWaiterCount fails intermittently. Contributed by Masatake Iwasaki.
(cherry picked from commit 13fb1b50e608558b2970184908ee5b9fcd7eb7b6) Conflicts: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java Updated CHANGES.txt
This commit is contained in:
parent
8aa18534d9
commit
8593225e85
@ -26,6 +26,9 @@ Release 2.7.4 - UNRELEASED
|
||||
HDFS-9745. TestSecureNNWithQJM#testSecureMode sometimes fails with
|
||||
timeouts (Xiao Chen via jlowe)
|
||||
|
||||
HDFS-8915. TestFSNamesystem.testFSLockGetWaiterCount fails intermittently.
|
||||
(Masatake Iwasaki via kihwal)
|
||||
|
||||
Release 2.7.3 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -27,6 +27,7 @@
|
||||
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.FileUtil;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
@ -48,6 +49,7 @@
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
public class TestFSNamesystem {
|
||||
|
||||
@ -241,8 +243,16 @@ public void run() {
|
||||
}
|
||||
|
||||
latch.await();
|
||||
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");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
Loading…
x
Reference in New Issue
Block a user