Extend various test timeouts. (#5978)

False failures on Travis due to spurious timeout (in turn due to noisy
neighbors) is a bigger problem than legitimate failures taking too long
to time out. So it makes sense to extend timeouts.
This commit is contained in:
Gian Merlino 2018-07-10 13:02:15 -07:00 committed by Jihoon Son
parent b3021ec802
commit 948e73da77
48 changed files with 101 additions and 101 deletions

View File

@ -79,7 +79,7 @@ public class BlockingPoolTest
emptyPool.takeBatch(1, 0); emptyPool.takeBatch(1, 0);
} }
@Test(timeout = 5000) @Test(timeout = 60_000L)
public void testTake() public void testTake()
{ {
final ReferenceCountingResourceHolder<Integer> holder = pool.take(100); final ReferenceCountingResourceHolder<Integer> holder = pool.take(100);
@ -89,7 +89,7 @@ public class BlockingPoolTest
assertEquals(10, pool.getPoolSize()); assertEquals(10, pool.getPoolSize());
} }
@Test(timeout = 5000) @Test(timeout = 60_000L)
public void testTakeTimeout() public void testTakeTimeout()
{ {
final List<ReferenceCountingResourceHolder<Integer>> batchHolder = pool.takeBatch(10, 100L); final List<ReferenceCountingResourceHolder<Integer>> batchHolder = pool.takeBatch(10, 100L);
@ -98,7 +98,7 @@ public class BlockingPoolTest
batchHolder.forEach(ReferenceCountingResourceHolder::close); batchHolder.forEach(ReferenceCountingResourceHolder::close);
} }
@Test(timeout = 5000) @Test(timeout = 60_000L)
public void testTakeBatch() public void testTakeBatch()
{ {
final List<ReferenceCountingResourceHolder<Integer>> holder = pool.takeBatch(6, 100L); final List<ReferenceCountingResourceHolder<Integer>> holder = pool.takeBatch(6, 100L);
@ -109,7 +109,7 @@ public class BlockingPoolTest
assertEquals(10, pool.getPoolSize()); assertEquals(10, pool.getPoolSize());
} }
@Test(timeout = 5000) @Test(timeout = 60_000L)
public void testWaitAndTakeBatch() throws InterruptedException, ExecutionException public void testWaitAndTakeBatch() throws InterruptedException, ExecutionException
{ {
List<ReferenceCountingResourceHolder<Integer>> batchHolder = pool.takeBatch(10, 10); List<ReferenceCountingResourceHolder<Integer>> batchHolder = pool.takeBatch(10, 10);
@ -132,14 +132,14 @@ public class BlockingPoolTest
assertEquals(10, pool.getPoolSize()); assertEquals(10, pool.getPoolSize());
} }
@Test(timeout = 5000) @Test(timeout = 60_000L)
public void testTakeBatchTooManyObjects() public void testTakeBatchTooManyObjects()
{ {
final List<ReferenceCountingResourceHolder<Integer>> holder = pool.takeBatch(100, 100L); final List<ReferenceCountingResourceHolder<Integer>> holder = pool.takeBatch(100, 100L);
assertTrue(holder.isEmpty()); assertTrue(holder.isEmpty());
} }
@Test(timeout = 5000) @Test(timeout = 60_000L)
public void testConcurrentTake() throws ExecutionException, InterruptedException public void testConcurrentTake() throws ExecutionException, InterruptedException
{ {
final int limit1 = pool.maxSize() / 2; final int limit1 = pool.maxSize() / 2;
@ -225,7 +225,7 @@ public class BlockingPoolTest
assertEquals(pool.maxSize(), pool.getPoolSize()); assertEquals(pool.maxSize(), pool.getPoolSize());
} }
@Test(timeout = 5000) @Test(timeout = 60_000L)
public void testConcurrentTakeBatch() throws ExecutionException, InterruptedException public void testConcurrentTakeBatch() throws ExecutionException, InterruptedException
{ {
final int batch1 = pool.maxSize() / 2; final int batch1 = pool.maxSize() / 2;
@ -255,7 +255,7 @@ public class BlockingPoolTest
assertEquals(pool.maxSize(), pool.getPoolSize()); assertEquals(pool.maxSize(), pool.getPoolSize());
} }
@Test(timeout = 5000) @Test(timeout = 60_000L)
public void testConcurrentBatchClose() throws ExecutionException, InterruptedException public void testConcurrentBatchClose() throws ExecutionException, InterruptedException
{ {
final int batch1 = pool.maxSize() / 2; final int batch1 = pool.maxSize() / 2;
@ -299,7 +299,7 @@ public class BlockingPoolTest
assertEquals(pool.maxSize(), pool.getPoolSize()); assertEquals(pool.maxSize(), pool.getPoolSize());
} }
@Test(timeout = 5000) @Test(timeout = 60_000L)
public void testConcurrentTakeBatchClose() throws ExecutionException, InterruptedException public void testConcurrentTakeBatchClose() throws ExecutionException, InterruptedException
{ {
final List<ReferenceCountingResourceHolder<Integer>> r1 = pool.takeBatch(1, 10); final List<ReferenceCountingResourceHolder<Integer>> r1 = pool.takeBatch(1, 10);

View File

@ -83,7 +83,7 @@ public class ReferenceCountingResourceHolderTest
}); });
} }
@Test(timeout = 60_000) @Test(timeout = 60_000L)
public void testResourceHandlerClearedByJVM() throws InterruptedException public void testResourceHandlerClearedByJVM() throws InterruptedException
{ {
long initialLeakedResources = ReferenceCountingResourceHolder.leakedResources(); long initialLeakedResources = ReferenceCountingResourceHolder.leakedResources();
@ -92,7 +92,7 @@ public class ReferenceCountingResourceHolderTest
verifyCleanerRun(released, initialLeakedResources); verifyCleanerRun(released, initialLeakedResources);
} }
@Test(timeout = 60_000) @Test(timeout = 60_000L)
public void testResourceHandlerWithReleaserClearedByJVM() throws InterruptedException public void testResourceHandlerWithReleaserClearedByJVM() throws InterruptedException
{ {
long initialLeakedResources = ReferenceCountingResourceHolder.leakedResources(); long initialLeakedResources = ReferenceCountingResourceHolder.leakedResources();

View File

@ -68,7 +68,7 @@ public class StupidPoolTest
resourceHolderObj.get(); resourceHolderObj.get();
} }
@Test(timeout = 60_000) @Test(timeout = 60_000L)
public void testResourceHandlerClearedByJVM() throws InterruptedException public void testResourceHandlerClearedByJVM() throws InterruptedException
{ {
String leakedString = createDanglingObjectHandler(); String leakedString = createDanglingObjectHandler();

View File

@ -133,7 +133,7 @@ public class DatasourceOptimizerTest extends CuratorTestBase
tearDownServerAndCurator(); tearDownServerAndCurator();
} }
@Test(timeout = 10 * 1000) @Test(timeout = 60_000L)
public void testOptimize() throws InterruptedException public void testOptimize() throws InterruptedException
{ {
// insert datasource metadata // insert datasource metadata

View File

@ -1641,7 +1641,7 @@ public class KafkaIndexTaskTest
Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
} }
@Test(timeout = 30_000L) @Test(timeout = 60_000L)
public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception
{ {
final KafkaIndexTask task = createTask( final KafkaIndexTask task = createTask(
@ -1671,7 +1671,7 @@ public class KafkaIndexTaskTest
} }
} }
@Test(timeout = 30_000L) @Test(timeout = 60_000L)
public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAvailable() throws Exception public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAvailable() throws Exception
{ {
resetOffsetAutomatically = true; resetOffsetAutomatically = true;

View File

@ -136,7 +136,7 @@ public class NamespacedExtractorModuleTest
} }
} }
@Test//(timeout = 10_000) @Test//(timeout = 60_000L)
public void testDeleteNamespaces() throws Exception public void testDeleteNamespaces() throws Exception
{ {
final File tmpFile = temporaryFolder.newFile(); final File tmpFile = temporaryFolder.newFile();

View File

@ -507,7 +507,7 @@ public class UriCacheGeneratorTest
Assert.assertNotNull(generator.generateCache(extractionNamespace, null, null, scheduler)); Assert.assertNotNull(generator.generateCache(extractionNamespace, null, null, scheduler));
} }
@Test(timeout = 10_000) @Test(timeout = 60_000L)
public void testDeleteOnScheduleFail() throws Exception public void testDeleteOnScheduleFail() throws Exception
{ {
Assert.assertNull(scheduler.scheduleAndWait( Assert.assertNull(scheduler.scheduleAndWait(

View File

@ -180,7 +180,7 @@ public class CacheSchedulerTest
lifecycle.stop(); lifecycle.stop();
} }
@Test(timeout = 10_000) @Test(timeout = 60_000L)
public void testSimpleSubmission() throws InterruptedException public void testSimpleSubmission() throws InterruptedException
{ {
UriExtractionNamespace namespace = new UriExtractionNamespace( UriExtractionNamespace namespace = new UriExtractionNamespace(
@ -199,7 +199,7 @@ public class CacheSchedulerTest
Assert.assertEquals("val", cache.get("key")); Assert.assertEquals("val", cache.get("key"));
} }
@Test(timeout = 10_000) @Test(timeout = 60_000L)
public void testPeriodicUpdatesScheduled() throws InterruptedException public void testPeriodicUpdatesScheduled() throws InterruptedException
{ {
final int repeatCount = 5; final int repeatCount = 5;
@ -233,7 +233,7 @@ public class CacheSchedulerTest
} }
@Test(timeout = 10_000) // This is very fast when run locally. Speed on Travis completely depends on noisy neighbors. @Test(timeout = 60_000L) // This is very fast when run locally. Speed on Travis completely depends on noisy neighbors.
public void testConcurrentAddDelete() throws InterruptedException public void testConcurrentAddDelete() throws InterruptedException
{ {
final int threads = 10; final int threads = 10;
@ -306,7 +306,7 @@ public class CacheSchedulerTest
checkNoMoreRunning(); checkNoMoreRunning();
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testSimpleDelete() throws InterruptedException public void testSimpleDelete() throws InterruptedException
{ {
testDelete(); testDelete();
@ -355,7 +355,7 @@ public class CacheSchedulerTest
); );
} }
@Test(timeout = 10_000) @Test(timeout = 60_000L)
public void testShutdown() public void testShutdown()
throws InterruptedException throws InterruptedException
{ {
@ -389,7 +389,7 @@ public class CacheSchedulerTest
Assert.assertTrue(cacheManager.scheduledExecutorService().isTerminated()); Assert.assertTrue(cacheManager.scheduledExecutorService().isTerminated());
} }
@Test(timeout = 10_000) @Test(timeout = 60_000L)
public void testRunCount() throws InterruptedException public void testRunCount() throws InterruptedException
{ {
final int numWaits = 5; final int numWaits = 5;
@ -415,7 +415,7 @@ public class CacheSchedulerTest
* Tests that even if entry.close() wasn't called, the scheduled task is cancelled when the entry becomes * Tests that even if entry.close() wasn't called, the scheduled task is cancelled when the entry becomes
* unreachable. * unreachable.
*/ */
@Test(timeout = 60_000) @Test(timeout = 60_000L)
public void testEntryCloseForgotten() throws InterruptedException public void testEntryCloseForgotten() throws InterruptedException
{ {
scheduleDanglingEntry(); scheduleDanglingEntry();

View File

@ -362,7 +362,7 @@ public class JdbcExtractionNamespaceTest
Thread.sleep(2); Thread.sleep(2);
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testMappingWithoutFilter() public void testMappingWithoutFilter()
throws InterruptedException throws InterruptedException
{ {
@ -389,7 +389,7 @@ public class JdbcExtractionNamespaceTest
} }
} }
@Test(timeout = 20_000L) @Test(timeout = 60_000L)
public void testMappingWithFilter() public void testMappingWithFilter()
throws InterruptedException throws InterruptedException
{ {
@ -421,7 +421,7 @@ public class JdbcExtractionNamespaceTest
} }
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testSkipOld() public void testSkipOld()
throws InterruptedException throws InterruptedException
{ {

View File

@ -70,7 +70,7 @@ public class NamespaceExtractionCacheManagersTest
manager = createCacheManager.apply(lifecycle); manager = createCacheManager.apply(lifecycle);
} }
@Test(timeout = 30000L) @Test(timeout = 60_000L)
public void testRacyCreation() throws Exception public void testRacyCreation() throws Exception
{ {
final int concurrentThreads = 10; final int concurrentThreads = 10;
@ -116,7 +116,7 @@ public class NamespaceExtractionCacheManagersTest
/** /**
* Tests that even if CacheHandler.close() wasn't called, the cache is cleaned up when it becomes unreachable. * Tests that even if CacheHandler.close() wasn't called, the cache is cleaned up when it becomes unreachable.
*/ */
@Test(timeout = 60_000) @Test(timeout = 60_000L)
public void testCacheCloseForgotten() throws InterruptedException public void testCacheCloseForgotten() throws InterruptedException
{ {
Assert.assertEquals(0, manager.cacheCount()); Assert.assertEquals(0, manager.cacheCount());

View File

@ -71,7 +71,7 @@ public class LockAcquireActionTest
Assert.assertEquals(expected.getTimeoutMs(), actual.getTimeoutMs()); Assert.assertEquals(expected.getTimeoutMs(), actual.getTimeoutMs());
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testWithLockType() public void testWithLockType()
{ {
final Task task = NoopTask.create(); final Task task = NoopTask.create();
@ -86,7 +86,7 @@ public class LockAcquireActionTest
Assert.assertNotNull(lock); Assert.assertNotNull(lock);
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testWithoutLockType() public void testWithoutLockType()
{ {
final Task task = NoopTask.create(); final Task task = NoopTask.create();

View File

@ -67,7 +67,7 @@ public class LockTryAcquireActionTest
Assert.assertEquals(expected.getInterval(), actual.getInterval()); Assert.assertEquals(expected.getInterval(), actual.getInterval());
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testWithLockType() public void testWithLockType()
{ {
final Task task = NoopTask.create(); final Task task = NoopTask.create();
@ -81,7 +81,7 @@ public class LockTryAcquireActionTest
Assert.assertNotNull(lock); Assert.assertNotNull(lock);
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testWithoutLockType() public void testWithoutLockType()
{ {
final Task task = NoopTask.create(); final Task task = NoopTask.create();

View File

@ -69,7 +69,7 @@ public class OverlordBlinkLeadershipTest
* org.apache.curator.framework.recipes.leader.LeaderSelectorListener#takeLeadership} implementation in * org.apache.curator.framework.recipes.leader.LeaderSelectorListener#takeLeadership} implementation in
* {@link TaskMaster} and start it again. * {@link TaskMaster} and start it again.
*/ */
@Test(timeout = 10_000) @Test(timeout = 60_000L)
public void testOverlordBlinkLeadership() public void testOverlordBlinkLeadership()
{ {
try { try {

View File

@ -57,7 +57,7 @@ public class RemoteTaskRunnerRunPendingTasksConcurrencyTest
} }
// This task reproduces the races described in https://github.com/druid-io/druid/issues/2842 // This task reproduces the races described in https://github.com/druid-io/druid/issues/2842
@Test(timeout = 60_000) @Test(timeout = 60_000L)
public void testConcurrency() throws Exception public void testConcurrency() throws Exception
{ {
rtrTestUtils.makeWorker("worker0", 3); rtrTestUtils.makeWorker("worker0", 3);

View File

@ -82,7 +82,7 @@ public class TaskLockBoxConcurrencyTest
service.shutdownNow(); service.shutdownNow();
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testDoInCriticalSectionWithDifferentTasks() public void testDoInCriticalSectionWithDifferentTasks()
throws ExecutionException, InterruptedException, EntryExistsException throws ExecutionException, InterruptedException, EntryExistsException
{ {
@ -163,7 +163,7 @@ public class TaskLockBoxConcurrencyTest
Assert.assertTrue(result.isRevoked()); Assert.assertTrue(result.isRevoked());
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testDoInCriticalSectionWithOverlappedIntervals() throws Exception public void testDoInCriticalSectionWithOverlappedIntervals() throws Exception
{ {
final List<Interval> intervals = ImmutableList.of( final List<Interval> intervals = ImmutableList.of(

View File

@ -408,7 +408,7 @@ public class TaskLockboxTest
); );
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testAcquireLockAfterRevoked() throws EntryExistsException, InterruptedException public void testAcquireLockAfterRevoked() throws EntryExistsException, InterruptedException
{ {
final Interval interval = Intervals.of("2017-01-01/2017-01-02"); final Interval interval = Intervals.of("2017-01-01/2017-01-02");

View File

@ -79,7 +79,7 @@ public class HttpRemoteTaskRunnerTest
Simulates startup of Overlord and Workers being discovered with no previously known tasks. Fresh tasks are given Simulates startup of Overlord and Workers being discovered with no previously known tasks. Fresh tasks are given
and expected to be completed. and expected to be completed.
*/ */
@Test(timeout = 30000L) @Test(timeout = 60_000L)
public void testFreshStart() throws Exception public void testFreshStart() throws Exception
{ {
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();
@ -167,7 +167,7 @@ public class HttpRemoteTaskRunnerTest
Simulates one task not getting acknowledged to be running after assigning it to a worker. But, other tasks are Simulates one task not getting acknowledged to be running after assigning it to a worker. But, other tasks are
successfully assigned to other worker and get completed. successfully assigned to other worker and get completed.
*/ */
@Test(timeout = 10000L) @Test(timeout = 60_000L)
public void testOneStuckTaskAssignmentDoesntBlockOthers() throws Exception public void testOneStuckTaskAssignmentDoesntBlockOthers() throws Exception
{ {
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();
@ -254,7 +254,7 @@ public class HttpRemoteTaskRunnerTest
/* /*
Simulates restart of the Overlord where taskRunner, on start, discovers workers with prexisting tasks. Simulates restart of the Overlord where taskRunner, on start, discovers workers with prexisting tasks.
*/ */
@Test(timeout = 10000L) @Test(timeout = 60_000L)
public void testTaskRunnerRestart() throws Exception public void testTaskRunnerRestart() throws Exception
{ {
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();
@ -399,7 +399,7 @@ public class HttpRemoteTaskRunnerTest
} }
@Test(timeout = 10000L) @Test(timeout = 60_000L)
public void testWorkerDisapperAndReappearBeforeItsCleanup() throws Exception public void testWorkerDisapperAndReappearBeforeItsCleanup() throws Exception
{ {
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();
@ -562,7 +562,7 @@ public class HttpRemoteTaskRunnerTest
Assert.assertTrue(future2.get().isSuccess()); Assert.assertTrue(future2.get().isSuccess());
} }
@Test(timeout = 10000L) @Test(timeout = 60_000L)
public void testWorkerDisapperAndReappearAfterItsCleanup() throws Exception public void testWorkerDisapperAndReappearAfterItsCleanup() throws Exception
{ {
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();
@ -725,7 +725,7 @@ public class HttpRemoteTaskRunnerTest
Assert.assertTrue(taskRunner.run(task2).get().isFailure()); Assert.assertTrue(taskRunner.run(task2).get().isFailure());
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testMarkWorkersLazy() throws Exception public void testMarkWorkersLazy() throws Exception
{ {
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();

View File

@ -199,7 +199,7 @@ public class OverlordTest
EmittingLogger.registerEmitter(serviceEmitter); EmittingLogger.registerEmitter(serviceEmitter);
} }
@Test(timeout = 2000L) @Test(timeout = 60_000L)
public void testOverlordRun() throws Exception public void testOverlordRun() throws Exception
{ {
// basic task master lifecycle test // basic task master lifecycle test

View File

@ -163,7 +163,7 @@ public class WorkerTaskManagerTest
workerTaskManager.stop(); workerTaskManager.stop();
} }
@Test(timeout = 10000L) @Test(timeout = 60_000L)
public void testTaskRun() throws Exception public void testTaskRun() throws Exception
{ {
Task task1 = createNoopTask("task1-assigned-via-assign-dir"); Task task1 = createNoopTask("task1-assigned-via-assign-dir");

View File

@ -215,7 +215,7 @@ public class WorkerTaskMonitorTest
testingCluster.stop(); testingCluster.stop();
} }
@Test(timeout = 30_000L) @Test(timeout = 60_000L)
public void testRunTask() throws Exception public void testRunTask() throws Exception
{ {
Assert.assertTrue( Assert.assertTrue(
@ -271,7 +271,7 @@ public class WorkerTaskMonitorTest
Assert.assertEquals(TaskState.SUCCESS, taskAnnouncement.getTaskStatus().getStatusCode()); Assert.assertEquals(TaskState.SUCCESS, taskAnnouncement.getTaskStatus().getStatusCode());
} }
@Test(timeout = 30_000L) @Test(timeout = 60_000L)
public void testGetAnnouncements() throws Exception public void testGetAnnouncements() throws Exception
{ {
cf.create() cf.create()
@ -309,7 +309,7 @@ public class WorkerTaskMonitorTest
Assert.assertEquals(DUMMY_NODE.getPlaintextPort(), announcements.get(0).getTaskLocation().getPort()); Assert.assertEquals(DUMMY_NODE.getPlaintextPort(), announcements.get(0).getTaskLocation().getPort());
} }
@Test(timeout = 30_000L) @Test(timeout = 60_000L)
public void testRestartCleansOldStatus() throws Exception public void testRestartCleansOldStatus() throws Exception
{ {
task = TestTasks.unending("test"); task = TestTasks.unending("test");
@ -345,7 +345,7 @@ public class WorkerTaskMonitorTest
Assert.assertEquals(TaskState.FAILED, announcements.get(0).getTaskStatus().getStatusCode()); Assert.assertEquals(TaskState.FAILED, announcements.get(0).getTaskStatus().getStatusCode());
} }
@Test(timeout = 30_000L) @Test(timeout = 60_000L)
public void testStatusAnnouncementsArePersistent() throws Exception public void testStatusAnnouncementsArePersistent() throws Exception
{ {
cf.create() cf.create()

View File

@ -28,7 +28,7 @@ import java.util.concurrent.TimeUnit;
public class ConcurrentAwaitableCounterTest public class ConcurrentAwaitableCounterTest
{ {
@Test(timeout = 1000) @Test(timeout = 60_000L)
public void smokeTest() throws InterruptedException public void smokeTest() throws InterruptedException
{ {
ConcurrentAwaitableCounter counter = new ConcurrentAwaitableCounter(); ConcurrentAwaitableCounter counter = new ConcurrentAwaitableCounter();

View File

@ -367,7 +367,7 @@ public class EmitterTest
Assert.assertTrue("httpClient.succeeded()", httpClient.succeeded()); Assert.assertTrue("httpClient.succeeded()", httpClient.succeeded());
} }
@Test(timeout = 60_000) @Test(timeout = 60_000L)
public void testFailedEmission() throws Exception public void testFailedEmission() throws Exception
{ {
final UnitEvent event1 = new UnitEvent("test", 1); final UnitEvent event1 = new UnitEvent("test", 1);

View File

@ -31,7 +31,7 @@ import java.util.List;
public class JvmMonitorTest public class JvmMonitorTest
{ {
@Test(timeout = 60000) @Test(timeout = 60_000L)
public void testGcCounts() throws InterruptedException public void testGcCounts() throws InterruptedException
{ {
GcTrackingEmitter emitter = new GcTrackingEmitter(); GcTrackingEmitter emitter = new GcTrackingEmitter();

View File

@ -58,7 +58,7 @@ public class ChainedExecutionQueryRunnerTest
neverRelease.lock(); neverRelease.lock();
} }
@Test(timeout = 60000) @Test(timeout = 60_000L)
public void testQueryCancellation() throws Exception public void testQueryCancellation() throws Exception
{ {
ExecutorService exec = PrioritizedExecutorService.create( ExecutorService exec = PrioritizedExecutorService.create(
@ -182,7 +182,7 @@ public class ChainedExecutionQueryRunnerTest
EasyMock.verify(watcher); EasyMock.verify(watcher);
} }
@Test(timeout = 60000) @Test(timeout = 60_000L)
public void testQueryTimeout() throws Exception public void testQueryTimeout() throws Exception
{ {
ExecutorService exec = PrioritizedExecutorService.create( ExecutorService exec = PrioritizedExecutorService.create(

View File

@ -180,7 +180,7 @@ public class GroupByQueryRunnerFailureTest
this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(runner)); this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(runner));
} }
@Test(timeout = 10000) @Test(timeout = 60_000L)
public void testNotEnoughMergeBuffersOnQueryable() public void testNotEnoughMergeBuffersOnQueryable()
{ {
expectedException.expect(QueryInterruptedException.class); expectedException.expect(QueryInterruptedException.class);
@ -209,7 +209,7 @@ public class GroupByQueryRunnerFailureTest
GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
} }
@Test(timeout = 10000) @Test(timeout = 60_000L)
public void testResourceLimitExceededOnBroker() public void testResourceLimitExceededOnBroker()
{ {
expectedException.expect(ResourceLimitExceededException.class); expectedException.expect(ResourceLimitExceededException.class);
@ -247,7 +247,7 @@ public class GroupByQueryRunnerFailureTest
GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
} }
@Test(timeout = 10000, expected = InsufficientResourcesException.class) @Test(timeout = 60_000L, expected = InsufficientResourcesException.class)
public void testInsufficientResourcesOnBroker() public void testInsufficientResourcesOnBroker()
{ {
final GroupByQuery query = GroupByQuery final GroupByQuery query = GroupByQuery

View File

@ -80,7 +80,7 @@ public class StreamingMergeSortedGrouperTest
); );
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testEmptyIterator() public void testEmptyIterator()
{ {
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
@ -91,13 +91,13 @@ public class StreamingMergeSortedGrouperTest
Assert.assertTrue(!grouper.iterator(true).hasNext()); Assert.assertTrue(!grouper.iterator(true).hasNext());
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testStreamingAggregateWithLargeBuffer() throws ExecutionException, InterruptedException public void testStreamingAggregateWithLargeBuffer() throws ExecutionException, InterruptedException
{ {
testStreamingAggregate(1024); testStreamingAggregate(1024);
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testStreamingAggregateWithMinimumBuffer() throws ExecutionException, InterruptedException public void testStreamingAggregateWithMinimumBuffer() throws ExecutionException, InterruptedException
{ {
testStreamingAggregate(60); testStreamingAggregate(60);

View File

@ -119,7 +119,7 @@ public class CompressionStrategyTest
Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray); Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray);
} }
@Test(timeout = 60000) @Test(timeout = 60_000L)
public void testConcurrency() throws Exception public void testConcurrency() throws Exception
{ {
final int numThreads = 20; final int numThreads = 20;

View File

@ -67,7 +67,7 @@ import java.util.concurrent.atomic.AtomicInteger;
*/ */
public class HttpServerInventoryViewTest public class HttpServerInventoryViewTest
{ {
@Test(timeout = 10000) @Test(timeout = 60_000L)
public void testSimple() throws Exception public void testSimple() throws Exception
{ {
ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); ObjectMapper jsonMapper = TestHelper.makeJsonMapper();

View File

@ -40,7 +40,7 @@ public class CuratorUtilsTest extends CuratorTestBase
tearDownServerAndCurator(); tearDownServerAndCurator();
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testCreateIfNotExists() throws Exception public void testCreateIfNotExists() throws Exception
{ {
curator.start(); curator.start();
@ -65,7 +65,7 @@ public class CuratorUtilsTest extends CuratorTestBase
Assert.assertEquals("baz", StringUtils.fromUtf8(curator.getData().forPath("/foo/bar"))); Assert.assertEquals("baz", StringUtils.fromUtf8(curator.getData().forPath("/foo/bar")));
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testCreateIfNotExistsPayloadTooLarge() throws Exception public void testCreateIfNotExistsPayloadTooLarge() throws Exception
{ {
curator.start(); curator.start();
@ -89,7 +89,7 @@ public class CuratorUtilsTest extends CuratorTestBase
Assert.assertNull(curator.checkExists().forPath("/foo/bar")); Assert.assertNull(curator.checkExists().forPath("/foo/bar"));
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testCreateOrSet() throws Exception public void testCreateOrSet() throws Exception
{ {
curator.start(); curator.start();
@ -116,7 +116,7 @@ public class CuratorUtilsTest extends CuratorTestBase
Assert.assertEquals("qux", StringUtils.fromUtf8(curator.getData().forPath("/foo/bar"))); Assert.assertEquals("qux", StringUtils.fromUtf8(curator.getData().forPath("/foo/bar")));
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testCreateOrSetPayloadTooLarge() throws Exception public void testCreateOrSetPayloadTooLarge() throws Exception
{ {
curator.start(); curator.start();

View File

@ -47,7 +47,7 @@ public class CuratorDruidLeaderSelectorTest extends CuratorTestBase
setupServerAndCurator(); setupServerAndCurator();
} }
@Test(timeout = 15000) @Test(timeout = 60_000L)
public void testSimple() throws Exception public void testSimple() throws Exception
{ {
curator.start(); curator.start();

View File

@ -52,7 +52,7 @@ public class CuratorDruidNodeAnnouncerAndDiscoveryTest extends CuratorTestBase
setupServerAndCurator(); setupServerAndCurator();
} }
@Test(timeout = 5000) @Test(timeout = 60_000L)
public void testAnnouncementAndDiscovery() throws Exception public void testAnnouncementAndDiscovery() throws Exception
{ {
ObjectMapper objectMapper = new DefaultObjectMapper(); ObjectMapper objectMapper = new DefaultObjectMapper();

View File

@ -217,7 +217,7 @@ public class SQLMetadataStorageActionHandlerTest
} }
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testRepeatInsert() throws Exception public void testRepeatInsert() throws Exception
{ {
final String entryId = "abcd"; final String entryId = "abcd";

View File

@ -434,7 +434,7 @@ public class LookupReferencesManagerTest
Assert.assertTrue(state.getToDrop().contains("one")); Assert.assertTrue(state.getToDrop().contains("one"));
} }
@Test(timeout = 20000) @Test(timeout = 60_000L)
public void testRealModeWithMainThread() throws Exception public void testRealModeWithMainThread() throws Exception
{ {
LookupReferencesManager lookupReferencesManager = new LookupReferencesManager( LookupReferencesManager lookupReferencesManager = new LookupReferencesManager(

View File

@ -376,7 +376,7 @@ public class RealtimeManagerTest
Assert.assertEquals(0, plumber2.getPersistCount()); Assert.assertEquals(0, plumber2.getPersistCount());
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testNormalStop() throws InterruptedException public void testNormalStop() throws InterruptedException
{ {
final TestFirehose firehose = new TestFirehose(rows.iterator()); final TestFirehose firehose = new TestFirehose(rows.iterator());
@ -421,7 +421,7 @@ public class RealtimeManagerTest
Assert.assertTrue(plumber2.isFinishedJob()); Assert.assertTrue(plumber2.isFinishedJob());
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testStopByInterruption() public void testStopByInterruption()
{ {
final SleepingFirehose firehose = new SleepingFirehose(); final SleepingFirehose firehose = new SleepingFirehose();
@ -454,7 +454,7 @@ public class RealtimeManagerTest
Assert.assertFalse(plumber.isFinishedJob()); Assert.assertFalse(plumber.isFinishedJob());
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testQueryWithInterval() throws InterruptedException public void testQueryWithInterval() throws InterruptedException
{ {
List<Row> expectedResults = Arrays.asList( List<Row> expectedResults = Arrays.asList(
@ -532,7 +532,7 @@ public class RealtimeManagerTest
} }
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testQueryWithSegmentSpec() throws InterruptedException public void testQueryWithSegmentSpec() throws InterruptedException
{ {
List<Row> expectedResults = Arrays.asList( List<Row> expectedResults = Arrays.asList(
@ -611,7 +611,7 @@ public class RealtimeManagerTest
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testQueryWithMultipleSegmentSpec() throws InterruptedException public void testQueryWithMultipleSegmentSpec() throws InterruptedException
{ {

View File

@ -414,7 +414,7 @@ public class AppenderatorTest
} }
} }
@Test(timeout = 10000L) @Test(timeout = 60_000L)
public void testTotalRowCount() throws Exception public void testTotalRowCount() throws Exception
{ {
try (final AppenderatorTester tester = new AppenderatorTester(3, true)) { try (final AppenderatorTester tester = new AppenderatorTester(3, true)) {

View File

@ -126,7 +126,7 @@ public class StreamAppenderatorDriverTest extends EasyMockSupport
driver.close(); driver.close();
} }
@Test(timeout = 2000L) @Test(timeout = 60_000L)
public void testSimple() throws Exception public void testSimple() throws Exception
{ {
final TestCommitterSupplier<Integer> committerSupplier = new TestCommitterSupplier<>(); final TestCommitterSupplier<Integer> committerSupplier = new TestCommitterSupplier<>();
@ -204,7 +204,7 @@ public class StreamAppenderatorDriverTest extends EasyMockSupport
Assert.assertEquals(numSegments * MAX_ROWS_PER_SEGMENT, segmentsAndMetadata.getCommitMetadata()); Assert.assertEquals(numSegments * MAX_ROWS_PER_SEGMENT, segmentsAndMetadata.getCommitMetadata());
} }
@Test(timeout = 5000L, expected = TimeoutException.class) @Test(timeout = 60_000L, expected = TimeoutException.class)
public void testHandoffTimeout() throws Exception public void testHandoffTimeout() throws Exception
{ {
final TestCommitterSupplier<Integer> committerSupplier = new TestCommitterSupplier<>(); final TestCommitterSupplier<Integer> committerSupplier = new TestCommitterSupplier<>();

View File

@ -241,7 +241,7 @@ public class EventReceiverFirehoseTest
); );
} }
@Test(timeout = 40_000L) @Test(timeout = 60_000L)
public void testShutdownWithPrevTime() throws Exception public void testShutdownWithPrevTime() throws Exception
{ {
EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED))
@ -261,7 +261,7 @@ public class EventReceiverFirehoseTest
} }
} }
@Test(timeout = 40_000L) @Test(timeout = 60_000L)
public void testShutdown() throws Exception public void testShutdown() throws Exception
{ {
EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED))

View File

@ -248,13 +248,13 @@ public class RealtimePlumberSchoolTest
FileUtils.deleteDirectory(tmpDir); FileUtils.deleteDirectory(tmpDir);
} }
@Test(timeout = 60000) @Test(timeout = 60_000L)
public void testPersist() throws Exception public void testPersist() throws Exception
{ {
testPersist(null); testPersist(null);
} }
@Test(timeout = 60000) @Test(timeout = 60_000L)
public void testPersistWithCommitMetadata() throws Exception public void testPersistWithCommitMetadata() throws Exception
{ {
final Object commitMetadata = "dummyCommitMetadata"; final Object commitMetadata = "dummyCommitMetadata";
@ -309,7 +309,7 @@ public class RealtimePlumberSchoolTest
plumber.finishJob(); plumber.finishJob();
} }
@Test(timeout = 60000) @Test(timeout = 60_000L)
public void testPersistFails() throws Exception public void testPersistFails() throws Exception
{ {
Sink sink = new Sink( Sink sink = new Sink(
@ -356,7 +356,7 @@ public class RealtimePlumberSchoolTest
Assert.assertEquals(1, metrics.failedPersists()); Assert.assertEquals(1, metrics.failedPersists());
} }
@Test(timeout = 60000) @Test(timeout = 60_000L)
public void testPersistHydrantGaps() throws Exception public void testPersistHydrantGaps() throws Exception
{ {
final Object commitMetadata = "dummyCommitMetadata"; final Object commitMetadata = "dummyCommitMetadata";
@ -461,7 +461,7 @@ public class RealtimePlumberSchoolTest
Assert.assertEquals(0, restoredPlumber2.getSinks().size()); Assert.assertEquals(0, restoredPlumber2.getSinks().size());
} }
@Test(timeout = 60000) @Test(timeout = 60_000L)
public void testDimOrderInheritance() throws Exception public void testDimOrderInheritance() throws Exception
{ {
final Object commitMetadata = "dummyCommitMetadata"; final Object commitMetadata = "dummyCommitMetadata";

View File

@ -164,7 +164,7 @@ public class AsyncQueryForwardingServletTest extends BaseJettyTest
Assert.assertNotEquals("gzip", postNoGzip.getContentEncoding()); Assert.assertNotEquals("gzip", postNoGzip.getContentEncoding());
} }
@Test(timeout = 60_000) @Test(timeout = 60_000L)
public void testDeleteBroadcast() throws Exception public void testDeleteBroadcast() throws Exception
{ {
CountDownLatch latch = new CountDownLatch(2); CountDownLatch latch = new CountDownLatch(2);

View File

@ -64,7 +64,7 @@ public class SQLAuditManagerTest
); );
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testAuditEntrySerde() throws IOException public void testAuditEntrySerde() throws IOException
{ {
AuditEntry entry = new AuditEntry( AuditEntry entry = new AuditEntry(
@ -83,7 +83,7 @@ public class SQLAuditManagerTest
Assert.assertEquals(entry, serde); Assert.assertEquals(entry, serde);
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testCreateAuditEntry() throws IOException public void testCreateAuditEntry() throws IOException
{ {
AuditEntry entry = new AuditEntry( AuditEntry entry = new AuditEntry(
@ -109,7 +109,7 @@ public class SQLAuditManagerTest
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testFetchAuditHistory() public void testFetchAuditHistory()
{ {
AuditEntry entry = new AuditEntry( AuditEntry entry = new AuditEntry(
@ -135,7 +135,7 @@ public class SQLAuditManagerTest
Assert.assertEquals(entry, auditEntries.get(1)); Assert.assertEquals(entry, auditEntries.get(1));
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testFetchAuditHistoryByKeyAndTypeWithLimit() public void testFetchAuditHistoryByKeyAndTypeWithLimit()
{ {
AuditEntry entry1 = new AuditEntry( AuditEntry entry1 = new AuditEntry(
@ -171,7 +171,7 @@ public class SQLAuditManagerTest
Assert.assertEquals(entry1, auditEntries.get(0)); Assert.assertEquals(entry1, auditEntries.get(0));
} }
@Test(timeout = 10_000L) @Test(timeout = 60_000L)
public void testFetchAuditHistoryByTypeWithLimit() public void testFetchAuditHistoryByTypeWithLimit()
{ {
AuditEntry entry1 = new AuditEntry( AuditEntry entry1 = new AuditEntry(

View File

@ -50,7 +50,7 @@ import java.util.concurrent.atomic.AtomicInteger;
*/ */
public class ChangeRequestHttpSyncerTest public class ChangeRequestHttpSyncerTest
{ {
@Test(timeout = 10000) @Test(timeout = 60_000L)
public void testSimple() throws Exception public void testSimple() throws Exception
{ {
ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); ObjectMapper jsonMapper = TestHelper.makeJsonMapper();

View File

@ -449,7 +449,7 @@ public class SegmentLoadDropHandlerTest
Assert.assertTrue(infoDir.delete()); Assert.assertTrue(infoDir.delete());
} }
@Test(timeout = 1000L) @Test(timeout = 60_000L)
public void testProcessBatch() throws Exception public void testProcessBatch() throws Exception
{ {
segmentLoadDropHandler.start(); segmentLoadDropHandler.start();

View File

@ -81,7 +81,7 @@ public class ZkCoordinatorTest extends CuratorTestBase
tearDownServerAndCurator(); tearDownServerAndCurator();
} }
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testLoadDrop() throws Exception public void testLoadDrop() throws Exception
{ {
EmittingLogger.registerEmitter(new NoopServiceEmitter()); EmittingLogger.registerEmitter(new NoopServiceEmitter());

View File

@ -240,7 +240,7 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
tearDownServerAndCurator(); tearDownServerAndCurator();
} }
@Test(timeout = 10_000) @Test(timeout = 60_000L)
public void testMoveSegment() throws Exception public void testMoveSegment() throws Exception
{ {
segmentViewInitLatch = new CountDownLatch(1); segmentViewInitLatch = new CountDownLatch(1);

View File

@ -98,7 +98,7 @@ public class HttpLoadQueuePeonTest
} }
}; };
@Test(timeout = 10000) @Test(timeout = 60_000L)
public void testSimple() throws Exception public void testSimple() throws Exception
{ {
HttpLoadQueuePeon httpLoadQueuePeon = new HttpLoadQueuePeon( HttpLoadQueuePeon httpLoadQueuePeon = new HttpLoadQueuePeon(
@ -135,7 +135,7 @@ public class HttpLoadQueuePeonTest
httpLoadQueuePeon.stop(); httpLoadQueuePeon.stop();
} }
@Test(timeout = 10000) @Test(timeout = 60_000L)
public void testLoadDropAfterStop() throws Exception public void testLoadDropAfterStop() throws Exception
{ {
HttpLoadQueuePeon httpLoadQueuePeon = new HttpLoadQueuePeon( HttpLoadQueuePeon httpLoadQueuePeon = new HttpLoadQueuePeon(

View File

@ -33,7 +33,7 @@ import java.util.concurrent.CountDownLatch;
*/ */
public class LimitRequestsFilterTest public class LimitRequestsFilterTest
{ {
@Test(timeout = 5000L) @Test(timeout = 60_000L)
public void testSimple() throws Exception public void testSimple() throws Exception
{ {
LimitRequestsFilter filter = new LimitRequestsFilter(2); LimitRequestsFilter filter = new LimitRequestsFilter(2);

View File

@ -1028,7 +1028,7 @@ public class LookupCoordinatorManagerTest
} }
@Test(timeout = 2_000) @Test(timeout = 60_000L)
public void testLookupManagementLoop() throws Exception public void testLookupManagementLoop() throws Exception
{ {
Map<String, LookupExtractorFactoryMapContainer> lookup1 = ImmutableMap.of( Map<String, LookupExtractorFactoryMapContainer> lookup1 = ImmutableMap.of(

View File

@ -551,7 +551,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
} }
@Test(timeout = 90000) @Test(timeout = 90_000L)
public void testConcurrentQueries() throws Exception public void testConcurrentQueries() throws Exception
{ {
final List<ListenableFuture<Integer>> futures = new ArrayList<>(); final List<ListenableFuture<Integer>> futures = new ArrayList<>();