HDFS-11630. TestThrottledAsyncCheckerTimeout fails intermittently in Jenkins builds. Contributed by Hanisha Koneru.

This commit is contained in:
Hanisha Koneru 2017-04-11 21:36:24 -07:00 committed by Arpit Agarwal
parent 3a91376707
commit 62e4573efb
2 changed files with 24 additions and 23 deletions

View File

@ -94,7 +94,7 @@ public VolumeCheckResult answer(
return volume;
}
@Test (timeout = 1000)
@Test (timeout = 300000)
public void testDiskCheckTimeout() throws Exception {
LOG.info("Executing {}", testName.getMethodName());
final FsVolumeSpi volume = makeSlowVolume();

View File

@ -17,6 +17,14 @@
*/
package org.apache.hadoop.hdfs.server.datanode.checker;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyObject;
import static org.mockito.Matchers.anySet;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.timeout;
import static org.mockito.Mockito.verify;
import com.google.common.base.Optional;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
@ -35,12 +43,7 @@
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyObject;
import static org.mockito.Matchers.anySet;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import org.junit.rules.Timeout;
import org.slf4j.LoggerFactory;
import java.util.Set;
@ -58,10 +61,10 @@ public class TestThrottledAsyncCheckerTimeout {
@Rule
public TestName testName = new TestName();
@Rule
public Timeout testTimeout = new Timeout(300_000);
Configuration conf;
private static final long DISK_CHECK_TIMEOUT = 10;
private static final long DISK_CHECK_TIME = 100;
private ReentrantLock lock;
private ExecutorService getExecutorService() {
@ -73,7 +76,7 @@ public void initializeLock() {
lock = new ReentrantLock();
}
@Test (timeout = 1000)
@Test
public void testDiskCheckTimeout() throws Exception {
LOG.info("Executing {}", testName.getMethodName());
@ -123,7 +126,7 @@ public void onFailure(Throwable t) {
assertTrue(throwable[0] instanceof TimeoutException);
}
@Test (timeout = 2000)
@Test
public void testDiskCheckTimeoutInvokesOneCallbackOnly() throws Exception {
LOG.info("Executing {}", testName.getMethodName());
@ -143,13 +146,12 @@ public void testDiskCheckTimeoutInvokesOneCallbackOnly() throws Exception {
assertTrue(olf1.isPresent());
Futures.addCallback(olf1.get(), futureCallback);
// Wait for the callback
Thread.sleep(DISK_CHECK_TIMEOUT);
// Verify that timeout results in only 1 onFailure call and 0 onSuccess
// calls.
verify(futureCallback, times(1)).onFailure(any());
verify(futureCallback, times(0)).onSuccess(any());
verify(futureCallback, timeout((int) DISK_CHECK_TIMEOUT*10).times(1))
.onFailure(any());
verify(futureCallback, timeout((int) DISK_CHECK_TIMEOUT*10).times(0))
.onSuccess(any());
// Release lock so that target can acquire it.
lock.unlock();
@ -160,16 +162,15 @@ public void testDiskCheckTimeoutInvokesOneCallbackOnly() throws Exception {
assertTrue(olf2.isPresent());
Futures.addCallback(olf2.get(), futureCallback);
// Wait for the callback
Thread.sleep(DISK_CHECK_TIME);
// Verify that normal check (dummy) results in only 1 onSuccess call.
// Number of times onFailure is invoked should remain the same - 1.
verify(futureCallback, times(1)).onFailure(any());
verify(futureCallback, times(1)).onSuccess(any());
// Number of times onFailure is invoked should remain the same i.e. 1.
verify(futureCallback, timeout((int) DISK_CHECK_TIMEOUT*10).times(1))
.onFailure(any());
verify(futureCallback, timeout((int) DISK_CHECK_TIMEOUT*10).times(1))
.onSuccess(any());
}
@Test (timeout = 1000)
@Test
public void testTimeoutExceptionIsNotThrownForGoodDisk() throws Exception {
LOG.info("Executing {}", testName.getMethodName());