From c2ba3b61e2b6ae933a4bb25801790b8e676c4040 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 21 Feb 2018 20:48:59 -0800 Subject: [PATCH 01/12] Guava-free Same Thread Executor The `MoreExecs.sameThreadExecutor()` is deprecated as per https://github.com/google/guava/blob/v18.0/guava/src/com/google/common/util/concurrent/MoreExecutors.java#L266-L267 and the function signature moves around a lot in future guava releases. This creates a new `SameThreadExecutorService` using more modern java features. --- .../lookup/KafkaLookupExtractorFactory.java | 2 +- .../kafka/supervisor/KafkaSupervisor.java | 9 +- .../indexing/kafka/KafkaIndexTaskTest.java | 10 +- .../variance/VarianceGroupByQueryTest.java | 11 +- .../io/druid/indexer/IndexGeneratorJob.java | 14 +- .../worker/WorkerCuratorCoordinator.java | 4 +- .../indexing/worker/WorkerTaskManager.java | 21 +- ...penderatorDriverRealtimeIndexTaskTest.java | 4 +- .../common/task/RealtimeIndexTaskTest.java | 5 +- .../indexing/overlord/TaskLifecycleTest.java | 28 +- .../hrtr/HttpRemoteTaskRunnerTest.java | 71 +- .../java/util/common/concurrent/Execs.java | 6 + .../concurrent/SameThreadExecutorService.java | 71 ++ .../SameThreadExecutorServiceTest.java | 54 + .../common/guava/WithEffectSequenceTest.java | 7 +- .../io/druid/query/QueryRunnerTestHelper.java | 11 +- .../io/druid/query/SchemaEvolutionTest.java | 25 +- .../aggregation/AggregationTestHelper.java | 6 +- .../groupby/GroupByQueryMergeBufferTest.java | 29 +- .../GroupByQueryRunnerFailureTest.java | 22 +- .../query/groupby/GroupByQueryRunnerTest.java | 1105 +++++++++++++++-- .../GroupByTimeseriesQueryRunnerTest.java | 4 +- .../metadata/SegmentMetadataQueryTest.java | 20 +- .../query/scan/MultiSegmentScanQueryTest.java | 15 +- .../spec/SpecificSegmentQueryRunnerTest.java | 5 +- .../druid/client/CachingClusteredClient.java | 18 +- .../io/druid/guice/DruidProcessingModule.java | 4 +- .../segment/realtime/RealtimeManager.java | 9 +- .../appenderator/SinkQuerySegmentWalker.java | 10 +- .../StreamAppenderatorDriver.java | 8 +- .../java/io/druid/server/QueryManager.java | 4 +- .../coordination/broker/DruidBroker.java | 4 +- ...chingClusteredClientFunctionalityTest.java | 4 +- .../client/CachingClusteredClientTest.java | 225 ++-- .../druid/client/CachingQueryRunnerTest.java | 6 +- .../client/HttpServerInventoryViewTest.java | 27 +- .../client/BatchServerInventoryViewTest.java | 14 +- ...torDruidNodeAnnouncerAndDiscoveryTest.java | 4 +- .../segment/realtime/RealtimeManagerTest.java | 4 +- ...inatorBasedSegmentHandoffNotifierTest.java | 5 +- .../plumber/RealtimePlumberSchoolTest.java | 9 +- .../io/druid/server/RequestLogLineTest.java | 49 + .../coordination/ServerManagerTest.java | 8 +- .../BatchDataSegmentAnnouncerTest.java | 8 +- .../main/java/io/druid/cli/DumpSegment.java | 4 +- .../druid/sql/calcite/schema/DruidSchema.java | 4 +- .../SpecificSegmentsQuerySegmentWalker.java | 4 +- 47 files changed, 1579 insertions(+), 412 deletions(-) create mode 100644 java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java create mode 100644 java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java create mode 100644 server/src/test/java/io/druid/server/RequestLogLineTest.java diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java index 08df93a5c87d..147b07135997 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java @@ -260,7 +260,7 @@ public void onFailure(Throwable t) } } }, - MoreExecutors.sameThreadExecutor() + Execs.sameThreadExecutor() ); this.future = future; final Stopwatch stopwatch = Stopwatch.createStarted(); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index aac26b2c4ce4..ff1d2d37d201 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -547,7 +547,7 @@ public void statusChanged(String taskId, TaskStatus status) { notices.add(new RunNotice()); } - }, MoreExecutors.sameThreadExecutor() + }, Execs.sameThreadExecutor() ); listenerRegistered = true; @@ -1771,7 +1771,12 @@ private void createKafkaTasksForGroup(int groupId, int replicas) throws JsonProc { }).writeValueAsString(taskGroups.get(groupId).sequenceOffsets); final Map context = spec.getContext() == null - ? ImmutableMap.of("checkpoints", checkpoints, IS_INCREMENTAL_HANDOFF_SUPPORTED, true) + ? ImmutableMap.of( + "checkpoints", + checkpoints, + IS_INCREMENTAL_HANDOFF_SUPPORTED, + true + ) : ImmutableMap.builder() .put("checkpoints", checkpoints) .put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true) diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index dfc20525513b..e12a5470ce1e 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -38,10 +38,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import io.druid.java.util.emitter.EmittingLogger; -import io.druid.java.util.emitter.core.NoopEmitter; -import io.druid.java.util.emitter.service.ServiceEmitter; -import io.druid.java.util.metrics.MonitorScheduler; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.MapCache; import io.druid.data.input.impl.DimensionsSpec; @@ -84,6 +80,10 @@ import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.JSONPathFieldSpec; import io.druid.java.util.common.parsers.JSONPathSpec; +import io.druid.java.util.emitter.EmittingLogger; +import io.druid.java.util.emitter.core.NoopEmitter; +import io.druid.java.util.emitter.service.ServiceEmitter; +import io.druid.java.util.metrics.MonitorScheduler; import io.druid.math.expr.ExprMacroTable; import io.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import io.druid.metadata.EntryExistsException; @@ -2020,7 +2020,7 @@ public List getLocations() EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, this::makeTimeseriesOnlyConglomerate, - MoreExecutors.sameThreadExecutor(), // queryExecutorService + Execs.sameThreadExecutor(), // queryExecutorService EasyMock.createMock(MonitorScheduler.class), new SegmentLoaderFactory( new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) diff --git a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceGroupByQueryTest.java b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceGroupByQueryTest.java index 703d8483743c..931fde3fe3ee 100644 --- a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceGroupByQueryTest.java +++ b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceGroupByQueryTest.java @@ -21,8 +21,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.data.input.Row; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.PeriodGranularity; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; @@ -68,12 +68,17 @@ public static Collection constructorFeeder() throws IOException return GroupByQueryRunnerTest.constructorFeeder(); } - public VarianceGroupByQueryTest(String testName, GroupByQueryConfig config, GroupByQueryRunnerFactory factory, QueryRunner runner) + public VarianceGroupByQueryTest( + String testName, + GroupByQueryConfig config, + GroupByQueryRunnerFactory factory, + QueryRunner runner + ) { this.testName = testName; this.config = config; this.factory = factory; - this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.>of(runner)); + this.runner = factory.mergeRunners(Execs.sameThreadExecutor(), ImmutableList.>of(runner)); } @Test diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index e094809cf3be..3a9e10a01c8c 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -299,7 +299,9 @@ protected void innerMap( throw new ISE("WTF?! No bucket found for row: %s", inputRow); } - final long truncatedTimestamp = granularitySpec.getQueryGranularity().bucketStart(inputRow.getTimestamp()).getMillis(); + final long truncatedTimestamp = granularitySpec.getQueryGranularity() + .bucketStart(inputRow.getTimestamp()) + .getMillis(); final byte[] hashedDimensions = hashFunction.hashBytes( HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsBytes( Rows.toGroupKey( @@ -472,7 +474,8 @@ public int getPartition(BytesWritable bytesWritable, Writable value, int numPart final ByteBuffer bytes = ByteBuffer.wrap(bytesWritable.getBytes()); bytes.position(4); // Skip length added by SortableBytes int shardNum = bytes.getInt(); - if ("local".equals(config.get("mapreduce.jobtracker.address")) || "local".equals(config.get("mapred.job.tracker"))) { + if ("local".equals(config.get("mapreduce.jobtracker.address")) + || "local".equals(config.get("mapred.job.tracker"))) { return shardNum % numPartitions; } else { if (shardNum >= numPartitions) { @@ -620,7 +623,7 @@ public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) ); persistExecutor = MoreExecutors.listeningDecorator(executorService); } else { - persistExecutor = MoreExecutors.sameThreadExecutor(); + persistExecutor = Execs.sameThreadExecutor(); } for (final BytesWritable bw : values) { @@ -730,7 +733,10 @@ indexes, aggregators, new File(baseFlushFile, "merged"), progressIndicator // ShardSpec to be published. final ShardSpec shardSpecForPublishing; if (config.isForceExtendableShardSpecs()) { - shardSpecForPublishing = new NumberedShardSpec(shardSpecForPartitioning.getPartitionNum(), config.getShardSpecCount(bucket)); + shardSpecForPublishing = new NumberedShardSpec( + shardSpecForPartitioning.getPartitionNum(), + config.getShardSpecCount(bucket) + ); } else { shardSpecForPublishing = shardSpecForPartitioning; } diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java index 3707fa1c5ca9..70cd74cab983 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java @@ -23,13 +23,13 @@ import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import io.druid.curator.CuratorUtils; import io.druid.curator.announcement.Announcer; import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.lifecycle.LifecycleStart; import io.druid.java.util.common.lifecycle.LifecycleStop; import io.druid.java.util.common.logger.Logger; @@ -77,7 +77,7 @@ public WorkerCuratorCoordinator( this.curatorFramework = curatorFramework; this.worker = worker; - this.announcer = new Announcer(curatorFramework, MoreExecutors.sameThreadExecutor()); + this.announcer = new Announcer(curatorFramework, Execs.sameThreadExecutor()); this.baseAnnouncementsPath = getPath(Arrays.asList(indexerZkConfig.getAnnouncementsPath(), worker.getHost())); this.baseTaskPath = getPath(Arrays.asList(indexerZkConfig.getTasksPath(), worker.getHost())); diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskManager.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskManager.java index ceb926906d00..c94944f8ac52 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskManager.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskManager.java @@ -28,11 +28,8 @@ import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; -import io.druid.java.util.emitter.EmittingLogger; -import io.druid.java.util.http.client.response.FullResponseHolder; import io.druid.client.indexing.IndexingService; import io.druid.concurrent.LifecycleLock; import io.druid.discovery.DruidLeaderClient; @@ -47,6 +44,8 @@ import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.lifecycle.LifecycleStart; import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.java.util.emitter.EmittingLogger; +import io.druid.java.util.http.client.response.FullResponseHolder; import io.druid.server.coordination.ChangeRequestHistory; import io.druid.server.coordination.ChangeRequestsSnapshot; import org.jboss.netty.handler.codec.http.HttpHeaders; @@ -69,7 +68,7 @@ /** * This class manages the list of tasks assigned to this worker. - * + *

* It persists the list of assigned and completed tasks on disk. assigned task from disk is deleted as soon as it * starts running and completed task on disk is deleted based on a periodic schedule where overlord is asked for * active tasks to see which completed tasks are safe to delete. @@ -226,7 +225,7 @@ public void statusChanged(final String taskId, final TaskStatus status) // do nothing } }, - MoreExecutors.sameThreadExecutor() + Execs.sameThreadExecutor() ); } @@ -456,9 +455,12 @@ private void scheduleCompletedTasksCleanup() ); if (fullResponseHolder.getStatus().getCode() == 200) { String responseContent = fullResponseHolder.getContent(); - taskStatusesFromOverlord = jsonMapper.readValue(responseContent, new TypeReference>() - { - }); + taskStatusesFromOverlord = jsonMapper.readValue( + responseContent, + new TypeReference>() + { + } + ); log.debug("Received completed task status response [%s].", responseContent); } else if (fullResponseHolder.getStatus().getCode() == 404) { // NOTE: this is to support backward compatibility, when overlord doesn't have "activeTasks" endpoint. @@ -516,7 +518,7 @@ private void scheduleCompletedTasksCleanup() TimeUnit.MINUTES ); } - + public void workerEnabled() { Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.SECONDS), "not started"); @@ -717,5 +719,6 @@ public void handle() throws InterruptedException //in Overlord as well as MiddleManagers then WorkerTaskMonitor should be deleted, this class should no longer be abstract //and the methods below should be removed. protected abstract void taskStarted(String taskId); + protected abstract void taskAnnouncementChanged(TaskAnnouncement announcement); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 2f1097c3e30f..a64ba23142c3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -151,6 +151,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; public class AppenderatorDriverRealtimeIndexTaskTest @@ -1229,7 +1230,6 @@ public List getLocations() return Lists.newArrayList(); } }; - taskToolboxFactory = new TaskToolboxFactory( taskConfig, taskActionClientFactory, @@ -1242,7 +1242,7 @@ public List getLocations() EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, () -> conglomerate, - MoreExecutors.sameThreadExecutor(), // queryExecutorService + Execs.sameThreadExecutor(), // queryExecutorService EasyMock.createMock(MonitorScheduler.class), new SegmentLoaderFactory( new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 9f050fdcab47..5325f4862826 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -207,7 +207,8 @@ public InputRow nextRow() @Override public Runnable commit() { - return () -> {}; + return () -> { + }; } @Override @@ -1076,7 +1077,7 @@ public List getLocations() EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, () -> conglomerate, - MoreExecutors.sameThreadExecutor(), // queryExecutorService + Execs.sameThreadExecutor(), // queryExecutorService EasyMock.createMock(MonitorScheduler.class), new SegmentLoaderFactory( new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 5f9ade8ddfe6..dcf6f0bd8341 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -33,7 +33,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.client.cache.MapCache; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; @@ -76,6 +75,7 @@ import io.druid.java.util.common.Pair; import io.druid.java.util.common.RE; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.emitter.EmittingLogger; @@ -527,8 +527,11 @@ private TaskToolboxFactory setUpTaskToolboxFactory( Preconditions.checkNotNull(emitter); taskLockbox = new TaskLockbox(taskStorage); - tac = new LocalTaskActionClientFactory(taskStorage, new TaskActionToolbox(taskLockbox, mdc, emitter, EasyMock.createMock( - SupervisorManager.class))); + tac = new LocalTaskActionClientFactory( + taskStorage, + new TaskActionToolbox(taskLockbox, mdc, emitter, EasyMock.createMock( + SupervisorManager.class)) + ); File tmpDir = temporaryFolder.newFolder(); taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, false, null, null); @@ -598,7 +601,7 @@ public void unannounceSegments(Iterable segments) throws IOExceptio EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, () -> queryRunnerFactoryConglomerate, // query runner factory conglomerate corporation unionized collective - MoreExecutors.sameThreadExecutor(), // query executor service + Execs.sameThreadExecutor(), // query executor service monitorScheduler, // monitor scheduler new SegmentLoaderFactory( new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, new DefaultObjectMapper()) @@ -1094,7 +1097,22 @@ public void testResumeTasks() throws Exception mapper ), new IndexIOConfig(new MockFirehoseFactory(false), false), - new IndexTuningConfig(10000, 10, null, null, null, indexSpec, null, false, null, null, null, null, null, null) + new IndexTuningConfig( + 10000, + 10, + null, + null, + null, + indexSpec, + null, + false, + null, + null, + null, + null, + null, + null + ) ), null ); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java index bab20f42c45a..4bb0bdeea5b7 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java @@ -27,8 +27,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.MoreExecutors; -import io.druid.java.util.http.client.HttpClient; import io.druid.common.guava.DSuppliers; import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscovery; @@ -48,6 +46,8 @@ import io.druid.indexing.worker.TaskAnnouncement; import io.druid.indexing.worker.Worker; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.concurrent.Execs; +import io.druid.java.util.http.client.HttpClient; import io.druid.segment.TestHelper; import io.druid.server.DruidNode; import io.druid.server.initialization.IndexerZkConfig; @@ -90,7 +90,8 @@ public void testFreshStart() throws Exception HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( TestHelper.makeJsonMapper(), - new HttpRemoteTaskRunnerConfig() { + new HttpRemoteTaskRunnerConfig() + { @Override public int getPendingTasksRunnerNumThreads() { @@ -104,7 +105,8 @@ public int getPendingTasksRunnerNumThreads() EasyMock.createNiceMock(TaskStorage.class), EasyMock.createNiceMock(CuratorFramework.class), new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null) - ) { + ) + { @Override protected WorkerHolder createWorkerHolder( ObjectMapper smileMapper, @@ -182,7 +184,8 @@ public void testOneStuckTaskAssignmentDoesntBlockOthers() throws Exception HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( TestHelper.makeJsonMapper(), - new HttpRemoteTaskRunnerConfig() { + new HttpRemoteTaskRunnerConfig() + { @Override public int getPendingTasksRunnerNumThreads() { @@ -196,7 +199,8 @@ public int getPendingTasksRunnerNumThreads() EasyMock.createNiceMock(TaskStorage.class), EasyMock.createNiceMock(CuratorFramework.class), new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null) - ) { + ) + { @Override protected WorkerHolder createWorkerHolder( ObjectMapper smileMapper, @@ -281,7 +285,8 @@ public void testTaskRunnerRestart() throws Exception HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( TestHelper.makeJsonMapper(), - new HttpRemoteTaskRunnerConfig() { + new HttpRemoteTaskRunnerConfig() + { @Override public int getPendingTasksRunnerNumThreads() { @@ -295,7 +300,8 @@ public int getPendingTasksRunnerNumThreads() taskStorageMock, EasyMock.createNiceMock(CuratorFramework.class), new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null) - ) { + ) + { @Override protected WorkerHolder createWorkerHolder( ObjectMapper smileMapper, @@ -313,7 +319,8 @@ protected WorkerHolder createWorkerHolder( config, workersSyncExec, listener, - worker); + worker + ); } else { throw new ISE("No WorkerHolder for [%s].", worker.getHost()); } @@ -412,7 +419,8 @@ public void testWorkerDisapperAndReappearBeforeItsCleanup() throws Exception HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( TestHelper.makeJsonMapper(), - new HttpRemoteTaskRunnerConfig() { + new HttpRemoteTaskRunnerConfig() + { @Override public int getPendingTasksRunnerNumThreads() { @@ -426,7 +434,8 @@ public int getPendingTasksRunnerNumThreads() EasyMock.createNiceMock(TaskStorage.class), EasyMock.createNiceMock(CuratorFramework.class), new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null) - ) { + ) + { @Override protected WorkerHolder createWorkerHolder( ObjectMapper smileMapper, @@ -444,7 +453,8 @@ protected WorkerHolder createWorkerHolder( config, workersSyncExec, listener, - worker); + worker + ); } else { throw new ISE("No WorkerHolder for [%s].", worker.getHost()); } @@ -575,7 +585,8 @@ public void testWorkerDisapperAndReappearAfterItsCleanup() throws Exception HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( TestHelper.makeJsonMapper(), - new HttpRemoteTaskRunnerConfig() { + new HttpRemoteTaskRunnerConfig() + { @Override public Period getTaskCleanupTimeout() { @@ -589,7 +600,8 @@ public Period getTaskCleanupTimeout() EasyMock.createNiceMock(TaskStorage.class), EasyMock.createNiceMock(CuratorFramework.class), new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null) - ) { + ) + { @Override protected WorkerHolder createWorkerHolder( ObjectMapper smileMapper, @@ -607,7 +619,8 @@ protected WorkerHolder createWorkerHolder( config, workersSyncExec, listener, - worker); + worker + ); } else { throw new ISE("No WorkerHolder for [%s].", worker.getHost()); } @@ -775,7 +788,8 @@ protected WorkerHolder createWorkerHolder( config, workersSyncExec, listener, - worker); + worker + ); } else { throw new ISE("No WorkerHolder for [%s].", worker.getHost()); } @@ -891,9 +905,10 @@ WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip2", 1, "0") Assert.assertEquals(task1.getId(), Iterables.getOnlyElement(taskRunner.getRunningTasks()).getTaskId()); Assert.assertEquals(task2.getId(), Iterables.getOnlyElement(taskRunner.getPendingTasks()).getTaskId()); - Assert.assertEquals("host3:8080", - Iterables.getOnlyElement(taskRunner.markWorkersLazy(Predicates.alwaysTrue(), Integer.MAX_VALUE)) - .getHost() + Assert.assertEquals( + "host3:8080", + Iterables.getOnlyElement(taskRunner.markWorkersLazy(Predicates.alwaysTrue(), Integer.MAX_VALUE)) + .getHost() ); } @@ -946,7 +961,9 @@ public void testTaskAddedOrUpdated1() throws Exception // Another "rogue-worker" reports running it, and gets asked to shutdown the task WorkerHolder rogueWorkerHolder = EasyMock.createMock(WorkerHolder.class); - EasyMock.expect(rogueWorkerHolder.getWorker()).andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1")).anyTimes(); + EasyMock.expect(rogueWorkerHolder.getWorker()) + .andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1")) + .anyTimes(); rogueWorkerHolder.shutdownTask(task.getId()); EasyMock.replay(rogueWorkerHolder); taskRunner.taskAddedOrUpdated(TaskAnnouncement.create( @@ -959,7 +976,9 @@ public void testTaskAddedOrUpdated1() throws Exception // "rogue-worker" reports FAILURE for the task, ignored rogueWorkerHolder = EasyMock.createMock(WorkerHolder.class); - EasyMock.expect(rogueWorkerHolder.getWorker()).andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1")).anyTimes(); + EasyMock.expect(rogueWorkerHolder.getWorker()) + .andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1")) + .anyTimes(); EasyMock.replay(rogueWorkerHolder); taskRunner.taskAddedOrUpdated(TaskAnnouncement.create( task, @@ -980,7 +999,9 @@ public void testTaskAddedOrUpdated1() throws Exception // "rogue-worker" reports running it, and gets asked to shutdown the task rogueWorkerHolder = EasyMock.createMock(WorkerHolder.class); - EasyMock.expect(rogueWorkerHolder.getWorker()).andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1")).anyTimes(); + EasyMock.expect(rogueWorkerHolder.getWorker()) + .andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1")) + .anyTimes(); rogueWorkerHolder.shutdownTask(task.getId()); EasyMock.replay(rogueWorkerHolder); taskRunner.taskAddedOrUpdated(TaskAnnouncement.create( @@ -993,7 +1014,9 @@ public void testTaskAddedOrUpdated1() throws Exception // "rogue-worker" reports FAILURE for the tasks, ignored rogueWorkerHolder = EasyMock.createMock(WorkerHolder.class); - EasyMock.expect(rogueWorkerHolder.getWorker()).andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1")).anyTimes(); + EasyMock.expect(rogueWorkerHolder.getWorker()) + .andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1")) + .anyTimes(); EasyMock.replay(rogueWorkerHolder); taskRunner.taskAddedOrUpdated(TaskAnnouncement.create( task, @@ -1206,7 +1229,7 @@ public void statusChanged(String taskId, TaskStatus status) listenerNotificationsAccumulator.add(ImmutableList.of(taskId, status)); } }, - MoreExecutors.sameThreadExecutor() + Execs.sameThreadExecutor() ); } diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index 3c5a63755a66..3adddfe33a35 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -21,6 +21,8 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; import javax.annotation.Nullable; @@ -49,6 +51,10 @@ public static ExecutorService dummy() return DummyExecutorService.INSTANCE; } + public static ListeningExecutorService sameThreadExecutor() { + return MoreExecutors.listeningDecorator(new SameThreadExecutorService()); + } + public static ExecutorService singleThreaded(@NotNull String nameFormat) { return singleThreaded(nameFormat, null); diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java new file mode 100644 index 000000000000..1e69c3d637b1 --- /dev/null +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java @@ -0,0 +1,71 @@ +package io.druid.java.util.common.concurrent; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.AbstractExecutorService; +import java.util.concurrent.Phaser; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; + +public class SameThreadExecutorService extends AbstractExecutorService +{ + private final AtomicBoolean shutdownLeader = new AtomicBoolean(true); + private final Phaser shutdownPhaser = new Phaser(0); + private final int initialPhase = shutdownPhaser.register(); + + @Override + public void shutdown() + { + if (shutdownLeader.getAndSet(false)) { + shutdownPhaser.arriveAndDeregister(); + } + } + + @Override + public List shutdownNow() + { + shutdown(); + return Collections.emptyList(); + } + + @Override + public boolean isShutdown() + { + return !shutdownLeader.get(); + } + + @Override + public boolean isTerminated() + { + return isShutdown() && shutdownPhaser.getRegisteredParties() < 1; + } + + @Override + public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException + { + try { + shutdownPhaser.awaitAdvanceInterruptibly(initialPhase, timeout, unit); + return true; + } + catch (TimeoutException ignored) { + return false; + } + } + + @Override + public void execute(Runnable command) + { + shutdownPhaser.register(); + try { + if (isShutdown()) { + throw new RejectedExecutionException(); + } + command.run(); + } + finally { + shutdownPhaser.arriveAndDeregister(); + } + } +} diff --git a/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java b/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java new file mode 100644 index 000000000000..28e5c1ab99a6 --- /dev/null +++ b/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java @@ -0,0 +1,54 @@ +package io.druid.java.util.common.concurrent; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.TimeUnit; + +public class SameThreadExecutorServiceTest +{ + @Test + public void timeoutAndShutdownTest() throws Exception + { + final SameThreadExecutorService service = new SameThreadExecutorService(); + Assert.assertFalse(service.awaitTermination(10, TimeUnit.MILLISECONDS)); + service.shutdown(); + Assert.assertTrue(service.awaitTermination(10, TimeUnit.MILLISECONDS)); + } + + @Test + public void hangingTaskTest() throws Exception + { + final CountDownLatch latch = new CountDownLatch(1); + final SameThreadExecutorService service = new SameThreadExecutorService(); + // Runnable gets called on submit, use the common pool to do the submit action + ForkJoinPool.commonPool().submit( + () -> service.submit(() -> { + try { + latch.await(); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + }) + ); + Assert.assertFalse(service.awaitTermination(10, TimeUnit.MILLISECONDS)); + service.shutdown(); + Assert.assertFalse(service.awaitTermination(10, TimeUnit.MILLISECONDS)); + latch.countDown(); + Assert.assertTrue(service.awaitTermination(10, TimeUnit.MILLISECONDS)); + Assert.assertTrue(service.isTerminated()); + } + + @Test + public void testMultiShutdownIsFine() { + final SameThreadExecutorService service = new SameThreadExecutorService(); + Assert.assertFalse(service.isShutdown()); + service.shutdown(); + Assert.assertTrue(service.isShutdown()); + service.shutdown(); + Assert.assertTrue(service.isShutdown()); + } +} diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java index da6494c0eaaa..d03ad987563c 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java @@ -20,6 +20,7 @@ package io.druid.java.util.common.guava; import com.google.common.util.concurrent.MoreExecutors; +import io.druid.java.util.common.concurrent.Execs; import org.junit.Assert; import org.junit.Test; @@ -40,11 +41,11 @@ public void testConsistentEffectApplicationOrder() .simple(Arrays.asList(1, 2, 3)) .withEffect( () -> effect1.set(counter.incrementAndGet()), - MoreExecutors.sameThreadExecutor() + Execs.sameThreadExecutor() ) .withEffect( () -> effect2.set(counter.incrementAndGet()), - MoreExecutors.sameThreadExecutor() + Execs.sameThreadExecutor() ); // Run sequence via accumulate sequence.toList(); @@ -70,7 +71,7 @@ public void testEffectExecutedIfWrappedSequenceThrowsExceptionFromClose() }); final AtomicBoolean effectExecuted = new AtomicBoolean(); Sequence seqWithEffect = - throwingSeq.withEffect(() -> effectExecuted.set(true), MoreExecutors.sameThreadExecutor()); + throwingSeq.withEffect(() -> effectExecuted.set(true), Execs.sameThreadExecutor()); try { seqWithEffect.toList(); Assert.fail("expected RuntimeException"); diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index e1d48c74b2e7..3188fb3b3ae9 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -24,16 +24,16 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.MoreExecutors; -import io.druid.java.util.emitter.core.NoopEmitter; -import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.MergeSequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.java.util.emitter.core.NoopEmitter; +import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.js.JavaScriptConfig; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -82,7 +82,8 @@ public class QueryRunnerTestHelper { - public static final QueryWatcher NOOP_QUERYWATCHER = (query, future) -> {}; + public static final QueryWatcher NOOP_QUERYWATCHER = (query, future) -> { + }; public static final String segmentId = "testSegment"; public static final String dataSource = "testing"; @@ -528,7 +529,7 @@ public Sequence run(QueryPlus queryPlus, Map responseConte public static IntervalChunkingQueryRunnerDecorator sameThreadIntervalChunkingQueryRunnerDecorator() { return new IntervalChunkingQueryRunnerDecorator( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), QueryRunnerTestHelper.NOOP_QUERYWATCHER, new ServiceEmitter("dummy", "dummy", new NoopEmitter()) ); diff --git a/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java b/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java index 9a96d28db49c..180824417c03 100644 --- a/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java +++ b/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.io.Closeables; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.data.input.InputRow; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.MapInputRowParser; @@ -32,6 +31,7 @@ import io.druid.data.input.impl.TimestampSpec; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.guava.Sequence; import io.druid.query.aggregation.AggregatorFactory; @@ -87,11 +87,22 @@ public static List inputRowsWithDimensions(final List dimensio ) ); return ImmutableList.of( - parser.parseBatch(ImmutableMap.of("t", "2000-01-01", "c1", "9", "c2", ImmutableList.of("a"))).get(0), - parser.parseBatch(ImmutableMap.of("t", "2000-01-02", "c1", "10.1", "c2", ImmutableList.of())).get(0), - parser.parseBatch(ImmutableMap.of("t", "2000-01-03", "c1", "2", "c2", ImmutableList.of(""))).get(0), - parser.parseBatch(ImmutableMap.of("t", "2001-01-01", "c1", "1", "c2", ImmutableList.of("a", "c"))).get(0), - parser.parseBatch(ImmutableMap.of("t", "2001-01-02", "c1", "4", "c2", ImmutableList.of("abc"))).get(0), + parser.parseBatch(ImmutableMap.of("t", "2000-01-01", "c1", "9", "c2", ImmutableList.of("a"))) + .get(0), + parser.parseBatch(ImmutableMap.of("t", "2000-01-02", "c1", "10.1", "c2", ImmutableList.of())) + .get(0), + parser.parseBatch(ImmutableMap.of("t", "2000-01-03", "c1", "2", "c2", ImmutableList.of(""))) + .get(0), + parser.parseBatch(ImmutableMap.of( + "t", + "2001-01-01", + "c1", + "1", + "c2", + ImmutableList.of("a", "c") + )).get(0), + parser.parseBatch(ImmutableMap.of("t", "2001-01-02", "c1", "4", "c2", ImmutableList.of("abc"))) + .get(0), parser.parseBatch(ImmutableMap.of("t", "2001-01-03", "c1", "5")).get(0) ); } @@ -105,7 +116,7 @@ public static > List runQuery( final Sequence results = new FinalizeResultsQueryRunner<>( factory.getToolchest().mergeResults( factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), FunctionalIterable .create(indexes) .transform( diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java index 06575a13673c..0c4e82aaa41c 100644 --- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java @@ -33,20 +33,19 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.io.Closeables; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.collections.StupidPool; import io.druid.data.input.InputRow; import io.druid.data.input.Row; import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.StringInputRowParser; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Yielder; import io.druid.java.util.common.guava.YieldingAccumulator; -import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryPlus; @@ -79,6 +78,7 @@ import io.druid.segment.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.commons.io.IOUtils; import org.apache.commons.io.LineIterator; import org.junit.rules.TemporaryFolder; @@ -525,7 +525,7 @@ public Sequence runQueryOnSegmentsObjs(final List segments, final toolChest.mergeResults( toolChest.preMergeQueryDecoration( factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), Lists.transform( segments, new Function() diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java index f318463b4318..54988590c5da 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -25,12 +25,12 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.collections.DefaultBlockingPool; import io.druid.collections.NonBlockingPool; import io.druid.collections.ReferenceCountingResourceHolder; import io.druid.collections.StupidPool; import io.druid.data.input.Row; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.DruidProcessingConfig; import io.druid.query.QueryContexts; @@ -61,6 +61,7 @@ public class GroupByQueryMergeBufferTest { private static final long TIMEOUT = 5000; + private static class TestBlockingPool extends DefaultBlockingPool { private int minRemainBufferNum; @@ -214,7 +215,7 @@ public static Collection constructorFeeder() throws IOException public GroupByQueryMergeBufferTest(QueryRunner runner) { - this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.>of(runner)); + this.runner = factory.mergeRunners(Execs.sameThreadExecutor(), ImmutableList.>of(runner)); } @Before @@ -252,7 +253,10 @@ public void testNestedGroupBy() .setDataSource(QueryRunnerTestHelper.dataSource) .setInterval(QueryRunnerTestHelper.firstToThird) .setGranularity(Granularities.ALL) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( + "quality", + "alias" + ))) .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) .build() ) @@ -286,12 +290,16 @@ public void testDoubleNestedGroupBy() new DefaultDimensionSpec("quality", "alias"), new DefaultDimensionSpec("market", null) )) - .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .setAggregatorSpecs(Lists.newArrayList( + QueryRunnerTestHelper.rowsCount)) .build() ) .setInterval(QueryRunnerTestHelper.firstToThird) .setGranularity(Granularities.ALL) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( + "quality", + "alias" + ))) .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) .build() ) @@ -329,7 +337,8 @@ public void testTripleNestedGroupBy() new DefaultDimensionSpec("market", null), new DefaultDimensionSpec("placement", null) )) - .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .setAggregatorSpecs(Lists.newArrayList( + QueryRunnerTestHelper.rowsCount)) .build() ) .setInterval(QueryRunnerTestHelper.firstToThird) @@ -338,12 +347,16 @@ public void testTripleNestedGroupBy() new DefaultDimensionSpec("quality", "alias"), new DefaultDimensionSpec("market", null) )) - .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .setAggregatorSpecs(Lists.newArrayList( + QueryRunnerTestHelper.rowsCount)) .build() ) .setInterval(QueryRunnerTestHelper.firstToThird) .setGranularity(Granularities.ALL) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( + "quality", + "alias" + ))) .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) .build() ) diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java index be8bf42be020..033f7265caa7 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java @@ -25,13 +25,13 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.collections.BlockingPool; import io.druid.collections.DefaultBlockingPool; import io.druid.collections.NonBlockingPool; import io.druid.collections.ReferenceCountingResourceHolder; import io.druid.collections.StupidPool; import io.druid.data.input.Row; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.DruidProcessingConfig; import io.druid.query.InsufficientResourcesException; @@ -178,7 +178,7 @@ public static Collection constructorFeeder() throws IOException public GroupByQueryRunnerFailureTest(QueryRunner runner) { - this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(runner)); + this.runner = factory.mergeRunners(Execs.sameThreadExecutor(), ImmutableList.of(runner)); } @Test(timeout = 10000) @@ -195,7 +195,10 @@ public void testNotEnoughMergeBuffersOnQueryable() throws IOException .setDataSource(QueryRunnerTestHelper.dataSource) .setInterval(QueryRunnerTestHelper.firstToThird) .setGranularity(Granularities.ALL) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( + "quality", + "alias" + ))) .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) .build() ) @@ -228,12 +231,16 @@ public void testResourceLimitExceededOnBroker() new DefaultDimensionSpec("quality", "alias"), new DefaultDimensionSpec("market", null) )) - .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .setAggregatorSpecs(Lists.newArrayList( + QueryRunnerTestHelper.rowsCount)) .build() ) .setInterval(QueryRunnerTestHelper.firstToThird) .setGranularity(Granularities.ALL) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( + "quality", + "alias" + ))) .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) .build() ) @@ -258,7 +265,10 @@ public void testInsufficientResourcesOnBroker() throws IOException .setDataSource(QueryRunnerTestHelper.dataSource) .setInterval(QueryRunnerTestHelper.firstToThird) .setGranularity(Granularities.ALL) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( + "quality", + "alias" + ))) .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) .build() ) diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 0d0615393b77..fcfd7ff79c70 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -41,6 +41,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.DurationGranularity; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.PeriodGranularity; @@ -422,7 +423,7 @@ public GroupByQueryRunnerTest( { this.config = config; this.factory = factory; - this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.>of(runner)); + this.runner = factory.mergeRunners(Execs.sameThreadExecutor(), ImmutableList.>of(runner)); } @Test @@ -445,25 +446,241 @@ public void testGroupBy() .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L, "idxFloat", 135.88510131835938f, "idxDouble", 135.88510131835938d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L, "idxFloat", 118.57034, "idxDouble", 118.57034), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L, "idxFloat", 158.747224, "idxDouble", 158.747224), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L, "idxFloat", 120.134704, "idxDouble", 120.134704), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L, "idxFloat", 2871.8866900000003f, "idxDouble", 2871.8866900000003d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L, "idxFloat", 121.58358f, "idxDouble", 121.58358d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L, "idxFloat", 2900.798647f, "idxDouble", 2900.798647d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L, "idxFloat", 78.622547f, "idxDouble", 78.622547d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L, "idxFloat", 119.922742f, "idxDouble", 119.922742d), - - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L, "idxFloat", 147.42593f, "idxDouble", 147.42593d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L, "idxFloat", 112.987027f, "idxDouble", 112.987027d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L, "idxFloat", 166.016049f, "idxDouble", 166.016049d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L, "idxFloat", 113.446008f, "idxDouble", 113.446008d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L, "idxFloat", 2448.830613f, "idxDouble", 2448.830613d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L, "idxFloat", 114.290141f, "idxDouble", 114.290141d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L, "idxFloat", 2506.415148f, "idxDouble", 2506.415148d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L, "idxFloat", 97.387433f, "idxDouble", 97.387433d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L, "idxFloat", 126.411364f, "idxDouble", 126.411364d) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "automotive", + "rows", + 1L, + "idx", + 135L, + "idxFloat", + 135.88510131835938f, + "idxDouble", + 135.88510131835938d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "business", + "rows", + 1L, + "idx", + 118L, + "idxFloat", + 118.57034, + "idxDouble", + 118.57034 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 158L, + "idxFloat", + 158.747224, + "idxDouble", + 158.747224 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "health", + "rows", + 1L, + "idx", + 120L, + "idxFloat", + 120.134704, + "idxDouble", + 120.134704 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "mezzanine", + "rows", + 3L, + "idx", + 2870L, + "idxFloat", + 2871.8866900000003f, + "idxDouble", + 2871.8866900000003d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "news", + "rows", + 1L, + "idx", + 121L, + "idxFloat", + 121.58358f, + "idxDouble", + 121.58358d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "premium", + "rows", + 3L, + "idx", + 2900L, + "idxFloat", + 2900.798647f, + "idxDouble", + 2900.798647d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "technology", + "rows", + 1L, + "idx", + 78L, + "idxFloat", + 78.622547f, + "idxDouble", + 78.622547d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "travel", + "rows", + 1L, + "idx", + 119L, + "idxFloat", + 119.922742f, + "idxDouble", + 119.922742d + ), + + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "automotive", + "rows", + 1L, + "idx", + 147L, + "idxFloat", + 147.42593f, + "idxDouble", + 147.42593d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "business", + "rows", + 1L, + "idx", + 112L, + "idxFloat", + 112.987027f, + "idxDouble", + 112.987027d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 166L, + "idxFloat", + 166.016049f, + "idxDouble", + 166.016049d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "health", + "rows", + 1L, + "idx", + 113L, + "idxFloat", + 113.446008f, + "idxDouble", + 113.446008d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "mezzanine", + "rows", + 3L, + "idx", + 2447L, + "idxFloat", + 2448.830613f, + "idxDouble", + 2448.830613d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "news", + "rows", + 1L, + "idx", + 114L, + "idxFloat", + 114.290141f, + "idxDouble", + 114.290141d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "premium", + "rows", + 3L, + "idx", + 2505L, + "idxFloat", + 2506.415148f, + "idxDouble", + 2506.415148d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "technology", + "rows", + 1L, + "idx", + 97L, + "idxFloat", + 97.387433f, + "idxDouble", + 97.387433d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "travel", + "rows", + 1L, + "idx", + 126L, + "idxFloat", + 126.411364f, + "idxDouble", + 126.411364d + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -531,25 +748,205 @@ public void testGroupByWithStringPostAggregator() .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "post", "travelx", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "post", "technologyx", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "post", "premiumx", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "post", "newsx", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "post", "mezzaninex", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "post", "healthx", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "post", "entertainmentx", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "post", "businessx", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "post", "automotivex", "rows", 1L, "idx", 135L), - - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "post", "travelx", "rows", 1L, "idx", 126L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "post", "technologyx", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "post", "premiumx", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "post", "newsx", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "post", "mezzaninex", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "post", "healthx", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "post", "entertainmentx", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "post", "businessx", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "post", "automotivex", "rows", 1L, "idx", 147L) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "travel", + "post", + "travelx", + "rows", + 1L, + "idx", + 119L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "technology", + "post", + "technologyx", + "rows", + 1L, + "idx", + 78L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "premium", + "post", + "premiumx", + "rows", + 3L, + "idx", + 2900L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "news", + "post", + "newsx", + "rows", + 1L, + "idx", + 121L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "mezzanine", + "post", + "mezzaninex", + "rows", + 3L, + "idx", + 2870L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "health", + "post", + "healthx", + "rows", + 1L, + "idx", + 120L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "entertainment", + "post", + "entertainmentx", + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "business", + "post", + "businessx", + "rows", + 1L, + "idx", + 118L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "automotive", + "post", + "automotivex", + "rows", + 1L, + "idx", + 135L + ), + + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "travel", + "post", + "travelx", + "rows", + 1L, + "idx", + 126L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "technology", + "post", + "technologyx", + "rows", + 1L, + "idx", + 97L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "premium", + "post", + "premiumx", + "rows", + 3L, + "idx", + 2505L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "news", + "post", + "newsx", + "rows", + 1L, + "idx", + 114L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "mezzanine", + "post", + "mezzaninex", + "rows", + 3L, + "idx", + 2447L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "health", + "post", + "healthx", + "rows", + 1L, + "idx", + 113L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "entertainment", + "post", + "entertainmentx", + "rows", + 1L, + "idx", + 166L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "business", + "post", + "businessx", + "rows", + 1L, + "idx", + 112L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "automotive", + "post", + "automotivex", + "rows", + 1L, + "idx", + 147L + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -866,10 +1263,50 @@ public void testTwoMultiValueDimensions() .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "a", "alias2", "a", "rows", 2L, "idx", 282L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "a", "alias2", "preferred", "rows", 2L, "idx", 282L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "preferred", "alias2", "a", "rows", 2L, "idx", 282L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "preferred", "alias2", "preferred", "rows", 2L, "idx", 282L) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "a", + "alias2", + "a", + "rows", + 2L, + "idx", + 282L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "a", + "alias2", + "preferred", + "rows", + 2L, + "idx", + 282L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "preferred", + "alias2", + "a", + "rows", + 2L, + "idx", + 282L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "preferred", + "alias2", + "preferred", + "rows", + 2L, + "idx", + 282L + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -1349,25 +1786,41 @@ public void testGroupByMaxRowsLimitContextOverride() expectedException.expect(ResourceLimitExceededException.class); } else { expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 166L + ), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); } @@ -1445,7 +1898,15 @@ public void testGroupByMaxOnDiskStorageContextOverride() expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 158L + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), @@ -1455,7 +1916,15 @@ public void testGroupByMaxOnDiskStorageContextOverride() GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 166L + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), @@ -1495,7 +1964,15 @@ public void testNotEnoughDictionarySpaceThroughContextOverride() expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 158L + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), @@ -1505,7 +1982,15 @@ public void testNotEnoughDictionarySpaceThroughContextOverride() GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 166L + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), @@ -1550,7 +2035,15 @@ public void testNotEnoughDiskSpaceThroughContextOverride() expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 158L + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), @@ -1560,7 +2053,15 @@ public void testNotEnoughDiskSpaceThroughContextOverride() GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 166L + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), @@ -2128,16 +2629,64 @@ public void testGroupByWithFirstLast() List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow("2011-01-01", "market", "spot", "first", 100L, "last", 155L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-01-01", "market", "total_market", "first", 1000L, "last", 1127L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-01-01", + "market", + "total_market", + "first", + 1000L, + "last", + 1127L + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-01-01", "market", "upfront", "first", 800L, "last", 943L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-02-01", "market", "spot", "first", 132L, "last", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-02-01", "market", "total_market", "first", 1203L, "last", 1292L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-02-01", "market", "upfront", "first", 1667L, "last", 1101L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-02-01", + "market", + "total_market", + "first", + 1203L, + "last", + 1292L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-02-01", + "market", + "upfront", + "first", + 1667L, + "last", + 1101L + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-03-01", "market", "spot", "first", 153L, "last", 125L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-03-01", "market", "total_market", "first", 1124L, "last", 1366L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-03-01", "market", "upfront", "first", 1166L, "last", 1063L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-03-01", + "market", + "total_market", + "first", + 1124L, + "last", + 1366L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-03-01", + "market", + "upfront", + "first", + 1166L, + "last", + 1063L + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "market", "spot", "first", 135L, "last", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "market", "total_market", "first", 1314L, "last", 1029L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "market", + "total_market", + "first", + 1314L, + "last", + 1029L + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "market", "upfront", "first", 1447L, "last", 780L) ); @@ -2929,7 +3478,8 @@ public void testGroupByOrderLimit() throws Exception TestHelper.assertExpectedObjects( expectedResults, mergeRunner.run(QueryPlus.wrap(builder.build()), context), - "no-limit"); + "no-limit" + ); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), @@ -3060,8 +3610,16 @@ public void testGroupByOrderLimitNumeric() throws Exception new LongSumAggregatorFactory("idx", "index") ) ) - .addOrderByColumn(new OrderByColumnSpec("rows", OrderByColumnSpec.Direction.DESCENDING, StringComparators.NUMERIC)) - .addOrderByColumn(new OrderByColumnSpec("alias", OrderByColumnSpec.Direction.ASCENDING, StringComparators.NUMERIC)) + .addOrderByColumn(new OrderByColumnSpec( + "rows", + OrderByColumnSpec.Direction.DESCENDING, + StringComparators.NUMERIC + )) + .addOrderByColumn(new OrderByColumnSpec( + "alias", + OrderByColumnSpec.Direction.ASCENDING, + StringComparators.NUMERIC + )) .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)); final GroupByQuery query = builder.build(); @@ -3959,10 +4517,18 @@ public void testDimFilterHavingSpec() public void testDimFilterHavingSpecWithExtractionFns() { String extractionJsFn = "function(str) { return 'super-' + str; }"; - ExtractionFn extractionFn = new JavaScriptExtractionFn(extractionJsFn, false, JavaScriptConfig.getEnabledInstance()); + ExtractionFn extractionFn = new JavaScriptExtractionFn( + extractionJsFn, + false, + JavaScriptConfig.getEnabledInstance() + ); String extractionJsFn2 = "function(num) { return num + 10; }"; - ExtractionFn extractionFn2 = new JavaScriptExtractionFn(extractionJsFn2, false, JavaScriptConfig.getEnabledInstance()); + ExtractionFn extractionFn2 = new JavaScriptExtractionFn( + extractionJsFn2, + false, + JavaScriptConfig.getEnabledInstance() + ); List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), @@ -5580,8 +6146,10 @@ public void testSubqueryWithOuterFilterAggregator() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("market", "market"), - new DefaultDimensionSpec("quality", "quality"))) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("market", "market"), + new DefaultDimensionSpec("quality", "quality") + )) .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, @@ -5619,8 +6187,10 @@ public void testSubqueryWithOuterTimeFilter() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("market", "market"), - new DefaultDimensionSpec("quality", "quality"))) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("market", "market"), + new DefaultDimensionSpec("quality", "quality") + )) .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, @@ -5630,8 +6200,16 @@ public void testSubqueryWithOuterTimeFilter() .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - final DimFilter fridayFilter = new SelectorDimFilter(Column.TIME_COLUMN_NAME, "Friday", new TimeFormatExtractionFn("EEEE", null, null, null, false)); - final DimFilter firstDaysFilter = new InDimFilter(Column.TIME_COLUMN_NAME, ImmutableList.of("1", "2", "3"), new TimeFormatExtractionFn("d", null, null, null, false)); + final DimFilter fridayFilter = new SelectorDimFilter( + Column.TIME_COLUMN_NAME, + "Friday", + new TimeFormatExtractionFn("EEEE", null, null, null, false) + ); + final DimFilter firstDaysFilter = new InDimFilter( + Column.TIME_COLUMN_NAME, + ImmutableList.of("1", "2", "3"), + new TimeFormatExtractionFn("d", null, null, null, false) + ); final GroupByQuery query = GroupByQuery .builder() .setDataSource(subquery) @@ -5744,12 +6322,13 @@ public void testSubqueryWithOuterCardinalityAggregator() .setDimensions(Lists.newArrayList()) .setAggregatorSpecs( ImmutableList.of( - new CardinalityAggregatorFactory("car", - ImmutableList.of(new DefaultDimensionSpec( - "quality", - "quality" - )), - false + new CardinalityAggregatorFactory( + "car", + ImmutableList.of(new DefaultDimensionSpec( + "quality", + "quality" + )), + false ) ) ) @@ -5814,8 +6393,10 @@ public void testSubqueryWithOuterDimJavascriptAggregators() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("market", "market"), - new DefaultDimensionSpec("quality", "quality"))) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("market", "market"), + new DefaultDimensionSpec("quality", "quality") + )) .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, @@ -5877,8 +6458,10 @@ public void testSubqueryWithOuterJavascriptAggregators() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("market", "market"), - new DefaultDimensionSpec("quality", "quality"))) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("market", "market"), + new DefaultDimensionSpec("quality", "quality") + )) .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, @@ -6485,7 +7068,7 @@ public ExtractionType getExtractionType() new ExtractionFn[]{ new TimeFormatExtractionFn("EEEE", null, null, null, false), nullWednesdays, - } + } ) ) ) @@ -7276,7 +7859,11 @@ public void testBySegmentResultsWithAllFiltersWithExtractionFns() String extractionJsFn = "function(str) { return 'super-' + str; }"; String jsFn = "function(x) { return(x === 'super-mezzanine') }"; - ExtractionFn extractionFn = new JavaScriptExtractionFn(extractionJsFn, false, JavaScriptConfig.getEnabledInstance()); + ExtractionFn extractionFn = new JavaScriptExtractionFn( + extractionJsFn, + false, + JavaScriptConfig.getEnabledInstance() + ); List superFilterList = new ArrayList<>(); superFilterList.add(new SelectorDimFilter("quality", "super-mezzanine", extractionFn)); @@ -7360,7 +7947,12 @@ public void testGroupByWithAllFiltersOnNullDimsWithExtractionFns() new ContainsSearchQuerySpec("EMPTY", true), extractionFn )); - superFilterList.add(new JavaScriptDimFilter("null_column", jsFn, extractionFn, JavaScriptConfig.getEnabledInstance())); + superFilterList.add(new JavaScriptDimFilter( + "null_column", + jsFn, + extractionFn, + JavaScriptConfig.getEnabledInstance() + )); DimFilter superFilter = new AndDimFilter(superFilterList); GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource) @@ -7416,12 +8008,60 @@ public void testGroupByCardinalityAggWithExtractionFn() .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "spot", "rows", 9L, "numVals", 1.0002442201269182d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "total_market", "rows", 2L, "numVals", 1.0002442201269182d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "upfront", "rows", 2L, "numVals", 1.0002442201269182d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "spot", "rows", 9L, "numVals", 1.0002442201269182d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "total_market", "rows", 2L, "numVals", 1.0002442201269182d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "upfront", "rows", 2L, "numVals", 1.0002442201269182d) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "spot", + "rows", + 9L, + "numVals", + 1.0002442201269182d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "total_market", + "rows", + 2L, + "numVals", + 1.0002442201269182d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "upfront", + "rows", + 2L, + "numVals", + 1.0002442201269182d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "spot", + "rows", + 9L, + "numVals", + 1.0002442201269182d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "total_market", + "rows", + 2L, + "numVals", + 1.0002442201269182d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "upfront", + "rows", + 2L, + "numVals", + 1.0002442201269182d + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -7453,12 +8093,60 @@ public void testGroupByCardinalityAggOnFloat() .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "spot", "rows", 9L, "numVals", 8.015665809687173d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "total_market", "rows", 2L, "numVals", 2.000977198748901d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "upfront", "rows", 2L, "numVals", 2.000977198748901d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "spot", "rows", 9L, "numVals", 9.019833517963864d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "total_market", "rows", 2L, "numVals", 2.000977198748901d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "upfront", "rows", 2L, "numVals", 2.000977198748901d) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "spot", + "rows", + 9L, + "numVals", + 8.015665809687173d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "total_market", + "rows", + 2L, + "numVals", + 2.000977198748901d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "upfront", + "rows", + 2L, + "numVals", + 2.000977198748901d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "spot", + "rows", + 9L, + "numVals", + 9.019833517963864d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "total_market", + "rows", + 2L, + "numVals", + 2.000977198748901d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "upfront", + "rows", + 2L, + "numVals", + 2.000977198748901d + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -7477,7 +8165,11 @@ public void testGroupByLongColumn() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG))) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( + "qualityLong", + "ql_alias", + ValueType.LONG + ))) .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) .setAggregatorSpecs( Arrays.asList( @@ -7540,7 +8232,11 @@ public void testGroupByLongColumnDescending() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG))) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( + "qualityLong", + "ql_alias", + ValueType.LONG + ))) .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null)) .setAggregatorSpecs( Arrays.asList( @@ -7606,7 +8302,11 @@ public void testGroupByLongColumnWithExFn() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec("qualityLong", "ql_alias", jsExtractionFn))) + .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec( + "qualityLong", + "ql_alias", + jsExtractionFn + ))) .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) .setAggregatorSpecs( Arrays.asList( @@ -7653,7 +8353,11 @@ public void testGroupByLongTimeColumn() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG))) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( + "__time", + "time_alias", + ValueType.LONG + ))) .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) .setAggregatorSpecs( Arrays.asList( @@ -7698,7 +8402,11 @@ public void testGroupByLongTimeColumnWithExFn() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec("__time", "time_alias", jsExtractionFn))) + .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec( + "__time", + "time_alias", + jsExtractionFn + ))) .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) .setAggregatorSpecs( Arrays.asList( @@ -7745,7 +8453,11 @@ public void testGroupByFloatColumn() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT))) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( + "index", + "index_alias", + ValueType.FLOAT + ))) .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) .setAggregatorSpecs( Arrays.asList( @@ -7809,7 +8521,11 @@ public void testGroupByFloatColumnDescending() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("qualityFloat", "qf_alias", ValueType.FLOAT))) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( + "qualityFloat", + "qf_alias", + ValueType.FLOAT + ))) .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null)) .setAggregatorSpecs( Arrays.asList( @@ -7938,7 +8654,11 @@ public void testGroupByFloatColumnWithExFn() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec("index", "index_alias", jsExtractionFn))) + .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec( + "index", + "index_alias", + jsExtractionFn + ))) .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) .setAggregatorSpecs( Arrays.asList( @@ -9030,11 +9750,61 @@ public Sequence run( Map context = Maps.newHashMap(); List allGranExpectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "market", "spot", "rows", 2L, "idx", 243L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "market", "spot", "rows", 2L, "idx", 177L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "upfront", "rows", 2L, "idx", 1817L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "total_market", "rows", 2L, "idx", 2342L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "spot", "rows", 2L, "idx", 257L) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "travel", + "market", + "spot", + "rows", + 2L, + "idx", + 243L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "technology", + "market", + "spot", + "rows", + 2L, + "idx", + 177L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "premium", + "market", + "upfront", + "rows", + 2L, + "idx", + 1817L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "premium", + "market", + "total_market", + "rows", + 2L, + "idx", + 2342L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "premium", + "market", + "spot", + "rows", + 2L, + "idx", + 257L + ) ); Iterable results = mergedRunner.run(QueryPlus.wrap(allGranQuery), context).toList(); @@ -9124,11 +9894,61 @@ public Sequence run( Map context = Maps.newHashMap(); List allGranExpectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "market", "spot", "rows", 2L, "idx", 243L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "market", "spot", "rows", 2L, "idx", 177L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "total_market", "rows", 2L, "idx", 2342L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "upfront", "rows", 2L, "idx", 1817L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "spot", "rows", 2L, "idx", 257L) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "travel", + "market", + "spot", + "rows", + 2L, + "idx", + 243L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "technology", + "market", + "spot", + "rows", + 2L, + "idx", + 177L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "premium", + "market", + "total_market", + "rows", + 2L, + "idx", + 2342L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "premium", + "market", + "upfront", + "rows", + 2L, + "idx", + 1817L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "premium", + "market", + "spot", + "rows", + 2L, + "idx", + 257L + ) ); Iterable results = mergedRunner.run(QueryPlus.wrap(allGranQuery), context).toList(); @@ -9402,19 +10222,42 @@ public void testTypeConversionWithMergingChainedExecutionRunner() .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "qualityLen", 10L, "rows", 2L, "idx", 156L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "qualityLen", 10L, "rows", 2L, "idx", 194L) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "technology", + "qualityLen", + 10L, + "rows", + 2L, + "idx", + 156L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "technology", + "qualityLen", + 10L, + "rows", + 2L, + "idx", + 194L + ) ); ChainedExecutionQueryRunner ceqr = new ChainedExecutionQueryRunner( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), (query1, future) -> { return; }, ImmutableList.>of(runner, runner) ); - QueryRunner mergingRunner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.>of(ceqr)); + QueryRunner mergingRunner = factory.mergeRunners( + Execs.sameThreadExecutor(), + ImmutableList.>of(ceqr) + ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, mergingRunner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index 65c162b9d422..30fc7d3c3c1c 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -22,11 +22,11 @@ import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; @@ -81,7 +81,7 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) { TimeseriesQuery tsQuery = (TimeseriesQuery) queryPlus.getQuery(); QueryRunner newRunner = factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), ImmutableList.>of(input) + Execs.sameThreadExecutor(), ImmutableList.>of(input) ); QueryToolChest toolChest = factory.getToolchest(); diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java index 6e688c7968bf..1fbfa6810a0f 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -25,10 +25,10 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.data.input.impl.TimestampSpec; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.BySegmentResultValue; import io.druid.query.BySegmentResultValueClass; @@ -303,7 +303,7 @@ public void testSegmentMetadataQueryWithRollupMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), Lists.>newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -371,7 +371,7 @@ public void testSegmentMetadataQueryWithHasMultipleValuesMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), Lists.>newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -439,7 +439,7 @@ public void testSegmentMetadataQueryWithComplexColumnMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), Lists.>newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -556,7 +556,7 @@ private void testSegmentMetadataQueryWithDefaultAnalysisMerge( QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), Lists.>newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -608,7 +608,7 @@ public void testSegmentMetadataQueryWithNoAnalysisTypesMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), Lists.>newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -670,7 +670,7 @@ public void testSegmentMetadataQueryWithAggregatorsMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), Lists.>newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -728,7 +728,7 @@ public void testSegmentMetadataQueryWithTimestampSpecMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), Lists.>newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -786,7 +786,7 @@ public void testSegmentMetadataQueryWithQueryGranularityMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), Lists.>newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -831,7 +831,7 @@ public void testBySegmentResults() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), //Note: It is essential to have atleast 2 query runners merged to reproduce the regression bug described in //https://github.com/druid-io/druid/pull/1172 //the bug surfaces only when ordering is used which happens only when you have 2 things to compare diff --git a/processing/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java b/processing/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java index 3eacb1faa51f..7108dc3edfcb 100644 --- a/processing/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java +++ b/processing/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java @@ -22,8 +22,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.io.CharSource; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.MergeSequence; import io.druid.java.util.common.guava.Sequence; @@ -166,7 +166,10 @@ public static void clear() @Parameterized.Parameters(name = "limit={0},batchSize={1}") public static Iterable constructorFeeder() throws IOException { - return QueryRunnerTestHelper.cartesian(Arrays.asList(0, 1, 3, 7, 10, 20, 1000), Arrays.asList(0, 1, 3, 6, 7, 10, 123, 2000)); + return QueryRunnerTestHelper.cartesian( + Arrays.asList(0, 1, 3, 7, 10, 20, 1000), + Arrays.asList(0, 1, 3, 6, 7, 10, 123, 2000) + ); } private final int limit; @@ -195,8 +198,9 @@ public void testMergeRunnersWithLimit() ScanQuery query = newBuilder().build(); List results = factory .mergeRunners( - MoreExecutors.sameThreadExecutor(), - ImmutableList.of(factory.createRunner(segment0), factory.createRunner(segment1))) + Execs.sameThreadExecutor(), + ImmutableList.of(factory.createRunner(segment0), factory.createRunner(segment1)) + ) .run(QueryPlus.wrap(query), new HashMap<>()) .toList(); int totalCount = 0; @@ -214,7 +218,8 @@ public void testMergeRunnersWithLimit() public void testMergeResultsWithLimit() { QueryRunner runner = toolChest.mergeResults( - new QueryRunner() { + new QueryRunner() + { @Override public Sequence run( QueryPlus queryPlus, Map responseContext diff --git a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java index 604e3bc60baa..1499e99aced7 100644 --- a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java @@ -23,10 +23,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Accumulator; import io.druid.java.util.common.guava.Sequence; @@ -118,6 +118,7 @@ public Yielder toYielder( null, new YieldingAccumulator() { final List lists = Lists.newArrayList(); + @Override public Object accumulate(Object accumulated, Object in) { @@ -164,7 +165,7 @@ public void run() throw new SegmentMissingException("FAILSAUCE"); } }, - MoreExecutors.sameThreadExecutor() + Execs.sameThreadExecutor() ); } }, diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 0fd9762dc7e9..0a360d0e43c6 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -40,21 +40,21 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; -import io.druid.java.util.emitter.EmittingLogger; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; import io.druid.client.selector.QueryableDruidServer; import io.druid.client.selector.ServerSelector; -import io.druid.java.util.common.concurrent.Execs; import io.druid.guice.annotations.BackgroundCaching; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.guava.BaseSequence; import io.druid.java.util.common.guava.LazySequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.java.util.emitter.EmittingLogger; import io.druid.query.BySegmentResultValueClass; import io.druid.query.CacheStrategy; import io.druid.query.Query; @@ -255,7 +255,8 @@ private ImmutableMap makeDownstreamQueryContext() Sequence run(final UnaryOperator> timelineConverter) { - @Nullable TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); + @Nullable + TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); if (timeline == null) { return Sequences.empty(); } @@ -265,10 +266,13 @@ Sequence run(final UnaryOperator> time } final Set segments = computeSegmentsToQuery(timeline); - @Nullable final byte[] queryCacheKey = computeQueryCacheKey(); + @Nullable + final byte[] queryCacheKey = computeQueryCacheKey(); if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { - @Nullable final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); - @Nullable final String currentEtag = computeCurrentEtag(segments, queryCacheKey); + @Nullable + final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); + @Nullable + final String currentEtag = computeCurrentEtag(segments, queryCacheKey); if (currentEtag != null && currentEtag.equals(prevEtag)) { return Sequences.empty(); } @@ -616,7 +620,7 @@ private Sequence getAndCacheServerResults( toolChest.makePreComputeManipulatorFn(downstreamQuery, MetricManipulatorFns.deserializing())::apply ); if (cachePopulator != null) { - res = res.withEffect(cachePopulator::populate, MoreExecutors.sameThreadExecutor()); + res = res.withEffect(cachePopulator::populate, Execs.sameThreadExecutor()); } return res; }) diff --git a/server/src/main/java/io/druid/guice/DruidProcessingModule.java b/server/src/main/java/io/druid/guice/DruidProcessingModule.java index 5d8f369652d6..a71ca8c8d538 100644 --- a/server/src/main/java/io/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/io/druid/guice/DruidProcessingModule.java @@ -19,7 +19,6 @@ package io.druid.guice; -import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Binder; import com.google.inject.Module; @@ -36,6 +35,7 @@ import io.druid.guice.annotations.Merging; import io.druid.guice.annotations.Processing; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.concurrent.ExecutorServiceConfig; import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; @@ -80,7 +80,7 @@ public ExecutorService getBackgroundExecutorService( .build() ); } else { - return MoreExecutors.sameThreadExecutor(); + return Execs.sameThreadExecutor(); } } diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index f054ad94aaa9..342f336971b3 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -27,18 +27,17 @@ import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; -import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; -import io.druid.java.util.emitter.EmittingLogger; -import io.druid.java.util.common.concurrent.Execs; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseV2; import io.druid.data.input.InputRow; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.lifecycle.LifecycleStart; import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.java.util.emitter.EmittingLogger; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; @@ -175,7 +174,7 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, Itera return partitionChiefs == null ? new NoopQueryRunner() : factory.getToolchest().mergeResults( factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), // Chaining query runners which wait on submitted chain query runners can make executor pools deadlock Iterables.transform( partitionChiefs.values(), new Function>() @@ -202,7 +201,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final ? new NoopQueryRunner() : factory.getToolchest().mergeResults( factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), Iterables.transform( specs, new Function>() diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 9d758553e71d..7ee3c2df6cd7 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -23,16 +23,16 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.MoreExecutors; -import io.druid.java.util.emitter.EmittingLogger; -import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.client.CachingQueryRunner; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.guava.FunctionalIterable; +import io.druid.java.util.emitter.EmittingLogger; +import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.query.BySegmentQueryRunner; import io.druid.query.CPUTimeMetricQueryRunner; import io.druid.query.MetricsEmittingQueryRunner; @@ -201,7 +201,7 @@ public QueryRunner apply(final SegmentDescriptor descriptor) sinkSegmentIdentifier, descriptor.getInterval().getStart(), factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), Iterables.transform( theSink, new Function>() @@ -235,7 +235,7 @@ public QueryRunner apply(final FireHydrant hydrant) cache, toolChest, baseRunner, - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), cacheConfig ); } else { diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index ab2c0262e9f0..f9f968d57766 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -28,11 +28,11 @@ import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.SettableFuture; import io.druid.data.input.Committer; import io.druid.data.input.InputRow; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.logger.Logger; import io.druid.query.SegmentDescriptor; import io.druid.segment.realtime.FireDepartmentMetrics; @@ -54,11 +54,11 @@ /** * This class is specialized for streaming ingestion. In streaming ingestion, the segment lifecycle is like: - * + *

*

  * APPENDING -> APPEND_FINISHED -> PUBLISHED
  * 
- * + *

*

    *
  • APPENDING: Segment is available for appending.
  • *
  • APPEND_FINISHED: Segment cannot be updated (data cannot be added anymore) and is waiting for being published.
  • @@ -331,7 +331,7 @@ public ListenableFuture registerHandoff(SegmentsAndMetadata segmentIdentifier.getVersion(), segmentIdentifier.getShardSpec().getPartitionNum() ), - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), () -> { log.info("Segment[%s] successfully handed off, dropping.", segmentIdentifier); metrics.incrementHandOffCount(); diff --git a/server/src/main/java/io/druid/server/QueryManager.java b/server/src/main/java/io/druid/server/QueryManager.java index 791e2cf2ecd7..b3a945df650c 100644 --- a/server/src/main/java/io/druid/server/QueryManager.java +++ b/server/src/main/java/io/druid/server/QueryManager.java @@ -23,7 +23,7 @@ import com.google.common.collect.Multimaps; import com.google.common.collect.SetMultimap; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.MoreExecutors; +import io.druid.java.util.common.concurrent.Execs; import io.druid.query.Query; import io.druid.query.QueryWatcher; @@ -76,7 +76,7 @@ public void run() } } }, - MoreExecutors.sameThreadExecutor() + Execs.sameThreadExecutor() ); } diff --git a/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java b/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java index 9bb94593f437..8b516215ee3e 100644 --- a/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java +++ b/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java @@ -20,7 +20,6 @@ package io.druid.server.coordination.broker; import com.google.common.base.Predicates; -import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import io.druid.client.FilteredServerInventoryView; import io.druid.client.ServerView; @@ -28,6 +27,7 @@ import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.Self; import io.druid.java.util.common.Pair; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.lifecycle.LifecycleStart; import io.druid.java.util.common.lifecycle.LifecycleStop; import io.druid.server.DruidNode; @@ -53,7 +53,7 @@ public DruidBroker( this.serviceAnnouncer = serviceAnnouncer; serverInventoryView.registerSegmentCallback( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), new ServerView.BaseSegmentCallback() { @Override diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java index 62110511dbd6..3406b9995781 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java @@ -22,7 +22,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.MapCache; @@ -30,6 +29,7 @@ import io.druid.client.selector.ServerSelector; import io.druid.client.selector.TierSelectorStrategy; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.guava.Sequence; import io.druid.query.DataSource; import io.druid.query.Druids; @@ -76,7 +76,7 @@ public void setUp() throws Exception timeline = new VersionedIntervalTimeline<>(Ordering.natural()); serverView = EasyMock.createNiceMock(TimelineServerView.class); cache = MapCache.create(100000); - client = makeClient(MoreExecutors.sameThreadExecutor()); + client = makeClient(Execs.sameThreadExecutor()); } @Test diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 0f4d7ebcfcfe..2658e7ae56f8 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -60,6 +60,7 @@ import io.druid.java.util.common.Intervals; import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.granularity.PeriodGranularity; @@ -103,11 +104,11 @@ import io.druid.query.groupby.GroupByQueryRunnerTest; import io.druid.query.groupby.strategy.GroupByStrategySelector; import io.druid.query.ordering.StringComparators; -import io.druid.query.search.SearchQueryQueryToolChest; -import io.druid.query.search.SearchResultValue; import io.druid.query.search.SearchHit; import io.druid.query.search.SearchQuery; import io.druid.query.search.SearchQueryConfig; +import io.druid.query.search.SearchQueryQueryToolChest; +import io.druid.query.search.SearchResultValue; import io.druid.query.select.EventHolder; import io.druid.query.select.PagingSpec; import io.druid.query.select.SelectQuery; @@ -259,42 +260,43 @@ public class CachingClusteredClientTest private static final Granularity PT1H_TZ_GRANULARITY = new PeriodGranularity(new Period("PT1H"), null, TIMEZONE); private static final String TOP_DIM = "a_dim"; - private static final Supplier selectConfigSupplier = Suppliers.ofInstance(new SelectQueryConfig(true)); + private static final Supplier selectConfigSupplier = Suppliers.ofInstance(new SelectQueryConfig( + true)); static final QueryToolChestWarehouse WAREHOUSE = new MapQueryToolChestWarehouse( ImmutableMap., QueryToolChest>builder() - .put( - TimeseriesQuery.class, - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ) - ) - .put( - TopNQuery.class, new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ) - ) - .put( - SearchQuery.class, new SearchQueryQueryToolChest( - new SearchQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ) - ) - .put( - SelectQuery.class, - new SelectQueryQueryToolChest( - jsonMapper, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), - selectConfigSupplier - ) - ) - .put( - GroupByQuery.class, - GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig()).getToolchest() - ) - .put(TimeBoundaryQuery.class, new TimeBoundaryQueryQueryToolChest()) - .build() + .put( + TimeseriesQuery.class, + new TimeseriesQueryQueryToolChest( + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ) + ) + .put( + TopNQuery.class, new TopNQueryQueryToolChest( + new TopNQueryConfig(), + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ) + ) + .put( + SearchQuery.class, new SearchQueryQueryToolChest( + new SearchQueryConfig(), + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ) + ) + .put( + SelectQuery.class, + new SelectQueryQueryToolChest( + jsonMapper, + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), + selectConfigSupplier + ) + ) + .put( + GroupByQuery.class, + GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig()).getToolchest() + ) + .put(TimeBoundaryQuery.class, new TimeBoundaryQueryQueryToolChest()) + .build() ); private final Random random; public CachingClusteredClient client; @@ -332,7 +334,7 @@ public void setUp() throws Exception timeline = new VersionedIntervalTimeline<>(Ordering.natural()); serverView = EasyMock.createNiceMock(TimelineServerView.class); cache = MapCache.create(100000); - client = makeClient(MoreExecutors.sameThreadExecutor()); + client = makeClient(Execs.sameThreadExecutor()); servers = new DruidServer[]{ new DruidServer("test1", "test1", null, 10, ServerType.HISTORICAL, "bye", 0), @@ -360,7 +362,7 @@ abstract class DrainTask implements Runnable final ListeningExecutorService delegate = MoreExecutors.listeningDecorator( // we need to run everything in the same thread to ensure all callbacks on futures in CachingClusteredClient // are complete before moving on to the next query run. - MoreExecutors.sameThreadExecutor() + Execs.sameThreadExecutor() ); @Override @@ -581,7 +583,7 @@ public void testCachingOverBulkLimitEnforcesLimit() throws Exception .andReturn(ImmutableMap.of()) .once(); EasyMock.replay(cache); - client = makeClient(MoreExecutors.sameThreadExecutor(), cache, limit); + client = makeClient(Execs.sameThreadExecutor(), cache, limit); final DruidServer lastServer = servers[random.nextInt(servers.length)]; final DataSegment dataSegment = EasyMock.createNiceMock(DataSegment.class); EasyMock.expect(dataSegment.getIdentifier()).andReturn(DATA_SOURCE).anyTimes(); @@ -606,7 +608,7 @@ public void testCachingOverBulkLimitEnforcesLimit() throws Exception .andReturn(ImmutableMap.of()) .once(); EasyMock.replay(cache); - client = makeClient(MoreExecutors.sameThreadExecutor(), cache, 0); + client = makeClient(Execs.sameThreadExecutor(), cache, 0); getDefaultQueryRunner().run(QueryPlus.wrap(query), context); EasyMock.verify(cache); EasyMock.verify(dataSegment); @@ -1195,14 +1197,14 @@ public void testSearchCaching() throws Exception public void testSearchCachingRenamedOutput() throws Exception { final Druids.SearchQueryBuilder builder = Druids.newSearchQueryBuilder() - .dataSource(DATA_SOURCE) - .filters(DIM_FILTER) - .granularity(GRANULARITY) - .limit(1000) - .intervals(SEG_SPEC) - .dimensions(Arrays.asList(TOP_DIM)) - .query("how") - .context(CONTEXT); + .dataSource(DATA_SOURCE) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .limit(1000) + .intervals(SEG_SPEC) + .dimensions(Arrays.asList(TOP_DIM)) + .query("how") + .context(CONTEXT); testQueryCaching( getDefaultQueryRunner(), @@ -1310,18 +1312,18 @@ public void testSelectCaching() throws Exception Intervals.of("2011-01-05/2011-01-10"), makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-05"), - DateTimes.of("2011-01-06"), - DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"), - DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8), - DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9) + DateTimes.of("2011-01-06"), + DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"), + DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8), + DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9) ), Intervals.of("2011-01-05/2011-01-10"), makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5), - DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6), - DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7), - DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8), - DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9) + DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6), + DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7), + DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8), + DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9) ) ); @@ -1336,17 +1338,17 @@ public void testSelectCaching() throws Exception HashMap context = new HashMap(); TestHelper.assertExpectedResults( makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-01"), ImmutableMap.of("a", "b", "rows", 1), - DateTimes.of("2011-01-02"), ImmutableMap.of("a", "c", "rows", 5), - DateTimes.of("2011-01-05"), - DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5), - DateTimes.of("2011-01-06"), - DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6), - DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"), - DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7), - DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8), - DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8), - DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9), - DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9) + DateTimes.of("2011-01-02"), ImmutableMap.of("a", "c", "rows", 5), + DateTimes.of("2011-01-05"), + DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5), + DateTimes.of("2011-01-06"), + DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6), + DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"), + DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7), + DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8), + DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8), + DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9), + DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9) ), runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context) ); @@ -1359,14 +1361,14 @@ public void testSelectCachingRenamedOutputName() throws Exception final Set metrics = Sets.newHashSet("rows"); Druids.SelectQueryBuilder builder = Druids.newSelectQueryBuilder() - .dataSource(DATA_SOURCE) - .intervals(SEG_SPEC) - .filters(DIM_FILTER) - .granularity(GRANULARITY) - .dimensions(Arrays.asList("a")) - .metrics(Arrays.asList("rows")) - .pagingSpec(new PagingSpec(null, 3)) - .context(CONTEXT); + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .dimensions(Arrays.asList("a")) + .metrics(Arrays.asList("rows")) + .pagingSpec(new PagingSpec(null, 3)) + .context(CONTEXT); testQueryCaching( getDefaultQueryRunner(), @@ -1454,9 +1456,9 @@ public void testSelectCachingRenamedOutputName() throws Exception public void testGroupByCaching() throws Exception { List aggsWithUniques = ImmutableList.builder() - .addAll(AGGS) - .add(new HyperUniquesAggregatorFactory("uniques", "uniques")) - .build(); + .addAll(AGGS) + .add(new HyperUniquesAggregatorFactory("uniques", "uniques")) + .build(); final HashFunction hashFn = Hashing.murmur3_128(); @@ -1657,9 +1659,11 @@ For dim1 (2011-01-06/2011-01-10), the combined range for the bound filters is {( makeTimeResults(DateTimes.of("2011-01-01"), 50, 5000, DateTimes.of("2011-01-02"), 10, 1252, DateTimes.of("2011-01-03"), 20, 6213, - DateTimes.of("2011-01-04"), 30, 743), + DateTimes.of("2011-01-04"), 30, 743 + ), makeTimeResults(DateTimes.of("2011-01-07"), 60, 6020, - DateTimes.of("2011-01-08"), 70, 250) + DateTimes.of("2011-01-08"), 70, 250 + ) ); testQueryCachingWithFilter( @@ -1697,14 +1701,14 @@ public void testSingleDimensionPruning() throws Exception ); final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() - .dataSource(DATA_SOURCE) - .filters(filter) - .granularity(GRANULARITY) - .intervals(SEG_SPEC) - .context(CONTEXT) - .intervals("2011-01-05/2011-01-10") - .aggregators(RENAMED_AGGS) - .postAggregators(RENAMED_POST_AGGS); + .dataSource(DATA_SOURCE) + .filters(filter) + .granularity(GRANULARITY) + .intervals(SEG_SPEC) + .context(CONTEXT) + .intervals("2011-01-05/2011-01-10") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS); TimeseriesQuery query = builder.build(); Map context = new HashMap<>(); @@ -1760,7 +1764,8 @@ public void testSingleDimensionPruning() throws Exception } private ServerSelector makeMockSingleDimensionSelector( - DruidServer server, String dimension, String start, String end, int partitionNum) + DruidServer server, String dimension, String start, String end, int partitionNum + ) { DataSegment segment = EasyMock.createNiceMock(DataSegment.class); EasyMock.expect(segment.getIdentifier()).andReturn(DATA_SOURCE).anyTimes(); @@ -1890,7 +1895,12 @@ public void testQueryCachingWithFilter( @Override public Sequence answer() throws Throwable { - return toFilteredQueryableTimeseriesResults((TimeseriesQuery) capture.getValue().getQuery(), segmentIds, queryIntervals, results); + return toFilteredQueryableTimeseriesResults( + (TimeseriesQuery) capture.getValue().getQuery(), + segmentIds, + queryIntervals, + results + ); } }) .times(0, 1); @@ -1948,7 +1958,11 @@ private Sequence> toFilteredQueryableTimeseriesRes MultipleSpecificSegmentSpec spec = (MultipleSpecificSegmentSpec) query.getQuerySegmentSpec(); List> ret = Lists.newArrayList(); for (SegmentDescriptor descriptor : spec.getDescriptors()) { - String id = StringUtils.format("%s_%s", queryIntervals.indexOf(descriptor.getInterval()), descriptor.getPartitionNumber()); + String id = StringUtils.format( + "%s_%s", + queryIntervals.indexOf(descriptor.getInterval()), + descriptor.getPartitionNumber() + ); int index = segmentIds.indexOf(id); if (index != -1) { ret.add(new Result( @@ -2460,13 +2474,13 @@ private Iterable> makeTimeResults(Object... object (DateTime) objects[i], new TimeseriesResultValue( ImmutableMap.builder() - .put("rows", objects[i + 1]) - .put("imps", objects[i + 2]) - .put("impers", objects[i + 2]) - .put("avg_imps_per_row", avg_impr) - .put("avg_imps_per_row_half", avg_impr / 2) - .put("avg_imps_per_row_double", avg_impr * 2) - .build() + .put("rows", objects[i + 1]) + .put("imps", objects[i + 2]) + .put("impers", objects[i + 2]) + .put("avg_imps_per_row", avg_impr) + .put("avg_imps_per_row_half", avg_impr / 2) + .put("avg_imps_per_row_double", avg_impr * 2) + .build() ) ) ); @@ -2613,7 +2627,11 @@ private Iterable> makeSearchResults(String dim, Object return retVal; } - private Iterable> makeSelectResults(Set dimensions, Set metrics, Object... objects) + private Iterable> makeSelectResults( + Set dimensions, + Set metrics, + Object... objects + ) { List> retVal = Lists.newArrayList(); int index = 0; @@ -2629,7 +2647,8 @@ private Iterable> makeSelectResults(Set dimens retVal.add(new Result<>( timestamp, new SelectResultValue(ImmutableMap.of(timestamp.toString(), 0), - dimensions, metrics, values) + dimensions, metrics, values + ) )); } return retVal; @@ -3128,7 +3147,10 @@ public void testIfNoneMatch() throws Exception TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder() .dataSource(DATA_SOURCE) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(interval))) - .context(ImmutableMap.of("If-None-Match", "aVJV29CJY93rszVW/QBy0arWZo0=")) + .context(ImmutableMap.of( + "If-None-Match", + "aVJV29CJY93rszVW/QBy0arWZo0=" + )) .build(); @@ -3141,7 +3163,8 @@ public void testIfNoneMatch() throws Exception @SuppressWarnings("unchecked") private QueryRunner getDefaultQueryRunner() { - return new QueryRunner() { + return new QueryRunner() + { @Override public Sequence run(final QueryPlus queryPlus, final Map responseContext) { diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index 4a7338417b09..ef749aeb98fc 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -24,8 +24,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.MoreExecutors; -import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.CacheStats; @@ -34,10 +32,12 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.SequenceWrapper; import io.druid.java.util.common.guava.Sequences; +import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.query.CacheStrategy; import io.druid.query.Druids; import io.druid.query.Query; @@ -106,7 +106,7 @@ public CachingQueryRunnerTest(int numBackgroundThreads) if (numBackgroundThreads > 0) { backgroundExecutorService = Executors.newFixedThreadPool(numBackgroundThreads); } else { - backgroundExecutorService = MoreExecutors.sameThreadExecutor(); + backgroundExecutorService = Execs.sameThreadExecutor(); } } diff --git a/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java b/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java index 7f7f88835187..717361f68e5d 100644 --- a/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java +++ b/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java @@ -24,23 +24,23 @@ import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.MoreExecutors; -import io.druid.java.util.http.client.HttpClient; -import io.druid.java.util.http.client.Request; -import io.druid.java.util.http.client.response.HttpResponseHandler; import io.druid.discovery.DataNodeService; import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscovery; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.RE; +import io.druid.java.util.common.concurrent.Execs; +import io.druid.java.util.http.client.HttpClient; +import io.druid.java.util.http.client.Request; +import io.druid.java.util.http.client.response.HttpResponseHandler; import io.druid.segment.TestHelper; import io.druid.server.DruidNode; +import io.druid.server.coordination.ChangeRequestHistory; +import io.druid.server.coordination.ChangeRequestsSnapshot; import io.druid.server.coordination.DruidServerMetadata; import io.druid.server.coordination.SegmentChangeRequestDrop; -import io.druid.server.coordination.ChangeRequestHistory; import io.druid.server.coordination.SegmentChangeRequestLoad; -import io.druid.server.coordination.ChangeRequestsSnapshot; import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; @@ -197,7 +197,7 @@ DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerT ); httpServerInventoryView.registerSegmentCallback( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), new ServerView.SegmentCallback() { @Override @@ -229,7 +229,7 @@ public ServerView.CallbackAction segmentViewInitialized() final CountDownLatch serverRemovedCalled = new CountDownLatch(1); httpServerInventoryView.registerServerRemovedCallback( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), new ServerView.ServerRemovedCallback() { @Override @@ -258,8 +258,10 @@ public ServerView.CallbackAction serverRemoved(DruidServer server) segmentDropLatches.get(segment2.getIdentifier()).await(); DruidServer druidServer = httpServerInventoryView.getInventoryValue("host:8080"); - Assert.assertEquals(ImmutableMap.of(segment3.getIdentifier(), segment3, segment4.getIdentifier(), segment4), - druidServer.getSegments()); + Assert.assertEquals( + ImmutableMap.of(segment3.getIdentifier(), segment3, segment4.getIdentifier(), segment4), + druidServer.getSegments() + ); druidNodeDiscovery.listener.nodesRemoved(ImmutableList.of(druidNode)); @@ -320,7 +322,10 @@ public ListenableFuture go( if (requestNum.get() == 2) { //fail scenario where request is sent to server but we got an unexpected response. - HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR); + HttpResponse httpResponse = new DefaultHttpResponse( + HttpVersion.HTTP_1_1, + HttpResponseStatus.INTERNAL_SERVER_ERROR + ); httpResponse.setContent(ChannelBuffers.buffer(0)); httpResponseHandler.handleResponse(httpResponse); return Futures.immediateFailedFuture(new RuntimeException("server error")); diff --git a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java index c66aebe13c51..e1dffe2b6a97 100644 --- a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java +++ b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java @@ -38,6 +38,7 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.guava.Comparators; import io.druid.segment.TestHelper; import io.druid.server.coordination.BatchDataSegmentAnnouncer; @@ -118,7 +119,7 @@ public void setUp() throws Exception announcer = new Announcer( cf, - MoreExecutors.sameThreadExecutor() + Execs.sameThreadExecutor() ); announcer.start(); @@ -204,7 +205,8 @@ public boolean apply(@Nullable Pair input) return input.rhs.getInterval().getStart().isBefore(SEGMENT_INTERVAL_START.plusDays(INITIAL_SEGMENTS)); } } - ) { + ) + { @Override protected DruidServer addInnerInventory( DruidServer container, String inventoryKey, Set inventory @@ -339,7 +341,7 @@ public ServerView.CallbackAction answer() throws Throwable EasyMock.replay(callback); filteredBatchServerInventoryView.registerSegmentCallback( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), callback, new Predicate>() { @@ -409,7 +411,11 @@ private void waitForUpdateEvents(int count) while (inventoryUpdateCounter.get() != count) { Thread.sleep(100); if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > forWaitingTiming.milliseconds()) { - throw new ISE("BatchServerInventoryView is not updating counter expected[%d] value[%d]", count, inventoryUpdateCounter.get()); + throw new ISE( + "BatchServerInventoryView is not updating counter expected[%d] value[%d]", + count, + inventoryUpdateCounter.get() + ); } } } diff --git a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java index d3ec923a43b6..587710886c9d 100644 --- a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java +++ b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java @@ -23,13 +23,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.curator.CuratorTestBase; import io.druid.curator.announcement.Announcer; import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscovery; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.concurrent.Execs; import io.druid.server.DruidNode; import io.druid.server.initialization.ServerConfig; import io.druid.server.initialization.ZkPathsConfig; @@ -69,7 +69,7 @@ public void testAnnouncementAndDiscovery() throws Exception Announcer announcer = new Announcer( curator, - MoreExecutors.sameThreadExecutor() + Execs.sameThreadExecutor() ); announcer.start(); diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 59f145555e85..fd078e0f4b96 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -28,7 +28,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; @@ -41,6 +40,7 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.parsers.ParseException; import io.druid.query.BaseQuery; @@ -1087,7 +1087,7 @@ public QueryRunner getQueryRunner(final Query query) return factory.getToolchest() .mergeResults( factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), Iterables.transform( baseQuery.getIntervals(), new Function>() diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java index b328add2de7d..023203395779 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java @@ -25,6 +25,7 @@ import io.druid.client.ImmutableSegmentLoadInfo; import io.druid.client.coordinator.CoordinatorClient; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.concurrent.Execs; import io.druid.query.SegmentDescriptor; import io.druid.server.coordination.DruidServerMetadata; import io.druid.server.coordination.ServerType; @@ -91,7 +92,7 @@ public void testHandoffCallbackNotCalled() throws IOException, InterruptedExcept ); final AtomicBoolean callbackCalled = new AtomicBoolean(false); notifier.registerSegmentHandoffCallback( - descriptor, MoreExecutors.sameThreadExecutor(), new Runnable() + descriptor, Execs.sameThreadExecutor(), new Runnable() { @Override public void run() @@ -148,7 +149,7 @@ public void testHandoffCallbackCalled() throws IOException, InterruptedException ); notifier.registerSegmentHandoffCallback( - descriptor, MoreExecutors.sameThreadExecutor(), new Runnable() + descriptor, Execs.sameThreadExecutor(), new Runnable() { @Override public void run() diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 76c3a1f2b797..d70ed250d010 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -25,7 +25,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.io.Files; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.client.cache.MapCache; import io.druid.data.input.Committer; import io.druid.data.input.InputRow; @@ -37,6 +36,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; @@ -119,7 +119,10 @@ public static Collection constructorFeeder() throws IOException private FireDepartmentMetrics metrics; private File tmpDir; - public RealtimePlumberSchoolTest(RejectionPolicyFactory rejectionPolicy, SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + public RealtimePlumberSchoolTest( + RejectionPolicyFactory rejectionPolicy, + SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + ) { this.rejectionPolicy = rejectionPolicy; this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; @@ -221,7 +224,7 @@ public void setUp() throws Exception announcer, segmentPublisher, handoffNotifierFactory, - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory), TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), MapCache.create(0), diff --git a/server/src/test/java/io/druid/server/RequestLogLineTest.java b/server/src/test/java/io/druid/server/RequestLogLineTest.java new file mode 100644 index 000000000000..81601af9912a --- /dev/null +++ b/server/src/test/java/io/druid/server/RequestLogLineTest.java @@ -0,0 +1,49 @@ +package io.druid.server; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.query.TableDataSource; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.post.ConstantPostAggregator; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.topn.NumericTopNMetricSpec; +import io.druid.query.topn.TopNQuery; +import io.druid.segment.VirtualColumns; +import org.joda.time.DateTime; +import org.junit.Test; + +public class RequestLogLineTest +{ + + @Test + public void testSimple() throws Exception { + final RequestLogLine line = new RequestLogLine( + DateTime.now(), + "127.0.0.1", + new TopNQuery( + new TableDataSource("dummy"), + VirtualColumns.EMPTY, + new DefaultDimensionSpec("test", "test"), + new NumericTopNMetricSpec("metric1"), + 3, + new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2015-01-01/2015-01-02"))), + null, + Granularities.ALL, + ImmutableList.of(new CountAggregatorFactory("metric1")), + ImmutableList.of(new ConstantPostAggregator("post", 10)), + null + ), + new QueryStats( + ImmutableMap.of() + ) + ); + final String print = line.getLine(new DefaultObjectMapper()); + System.out.println(print); + } +} \ No newline at end of file diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 0289d697ac15..d8b31ce66660 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -26,8 +26,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.MoreExecutors; -import io.druid.java.util.emitter.EmittingLogger; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.LocalCacheProvider; import io.druid.jackson.DefaultObjectMapper; @@ -35,6 +33,7 @@ import io.druid.java.util.common.Intervals; import io.druid.java.util.common.MapUtils; import io.druid.java.util.common.Pair; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Sequence; @@ -42,6 +41,7 @@ import io.druid.java.util.common.guava.Yielder; import io.druid.java.util.common.guava.YieldingAccumulator; import io.druid.java.util.common.guava.YieldingSequenceBase; +import io.druid.java.util.emitter.EmittingLogger; import io.druid.query.ConcatQueryRunner; import io.druid.query.DefaultQueryMetrics; import io.druid.query.Druids; @@ -55,8 +55,8 @@ import io.druid.query.QueryToolChest; import io.druid.query.Result; import io.druid.query.aggregation.MetricManipulationFn; -import io.druid.query.search.SearchResultValue; import io.druid.query.search.SearchQuery; +import io.druid.query.search.SearchResultValue; import io.druid.segment.AbstractSegment; import io.druid.segment.IndexIO; import io.druid.segment.QueryableIndex; @@ -152,7 +152,7 @@ public > QueryRunnerFactory findFact }, new NoopServiceEmitter(), serverManagerExec, - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), new DefaultObjectMapper(), new LocalCacheProvider().get(), new CacheConfig(), diff --git a/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java index 6d205cfe4868..956d557852de 100644 --- a/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java +++ b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java @@ -27,16 +27,16 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.curator.PotentiallyGzippedCompressionProvider; import io.druid.curator.announcement.Announcer; import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.concurrent.Execs; import io.druid.segment.TestHelper; import io.druid.server.coordination.BatchDataSegmentAnnouncer; -import io.druid.server.coordination.DataSegmentChangeRequest; -import io.druid.server.coordination.DruidServerMetadata; import io.druid.server.coordination.ChangeRequestHistory; import io.druid.server.coordination.ChangeRequestsSnapshot; +import io.druid.server.coordination.DataSegmentChangeRequest; +import io.druid.server.coordination.DruidServerMetadata; import io.druid.server.coordination.ServerType; import io.druid.server.initialization.BatchDataSegmentAnnouncerConfig; import io.druid.server.initialization.ZkPathsConfig; @@ -96,7 +96,7 @@ public void setUp() throws Exception announcer = new Announcer( cf, - MoreExecutors.sameThreadExecutor() + Execs.sameThreadExecutor() ); announcer.start(); diff --git a/services/src/main/java/io/druid/cli/DumpSegment.java b/services/src/main/java/io/druid/cli/DumpSegment.java index 3d82fbf8d3ce..e5956a746b39 100644 --- a/services/src/main/java/io/druid/cli/DumpSegment.java +++ b/services/src/main/java/io/druid/cli/DumpSegment.java @@ -28,7 +28,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; @@ -48,6 +47,7 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Accumulator; import io.druid.java.util.common.guava.Sequence; @@ -484,7 +484,7 @@ private static Sequence executeQuery(final Injector injector, final Query final QueryRunnerFactory factory = conglomerate.findFactory(query); final QueryRunner runner = factory.createRunner(new QueryableIndexSegment("segment", index)); final Sequence results = factory.getToolchest().mergeResults( - factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(runner)) + factory.mergeRunners(Execs.sameThreadExecutor(), ImmutableList.of(runner)) ).run(QueryPlus.wrap(query), Maps.newHashMap()); return (Sequence) results; } diff --git a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java index b946afea7774..8253d1de91a8 100644 --- a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java @@ -26,13 +26,13 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; -import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import io.druid.client.ServerView; import io.druid.client.TimelineServerView; import io.druid.guice.ManageLifecycle; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.concurrent.ScheduledExecutors; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Yielder; @@ -142,7 +142,7 @@ public DruidSchema( this.escalator = escalator; serverView.registerTimelineCallback( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), new TimelineServerView.TimelineCallback() { @Override diff --git a/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index 5c3f8a97ba16..8b3843a39483 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -25,9 +25,9 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.io.Closeables; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.java.util.common.ISE; import io.druid.java.util.common.UOE; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.guava.Sequence; import io.druid.query.FinalizeResultsQueryRunner; @@ -224,7 +224,7 @@ private QueryRunner makeBaseRunner( return new FinalizeResultsQueryRunner<>( toolChest.mergeResults( factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.sameThreadExecutor(), FunctionalIterable .create(specs) .transformCat( From 37179e6232024f84c2278abcd43b4d106c92df61 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 21 Feb 2018 20:54:10 -0800 Subject: [PATCH 02/12] Add copyright --- .../concurrent/SameThreadExecutorService.java | 19 +++++++++++++++++++ .../SameThreadExecutorServiceTest.java | 19 +++++++++++++++++++ 2 files changed, 38 insertions(+) diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java index 1e69c3d637b1..fb340612800a 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java @@ -1,3 +1,22 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package io.druid.java.util.common.concurrent; import java.util.Collections; diff --git a/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java b/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java index 28e5c1ab99a6..45792ff9823a 100644 --- a/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java @@ -1,3 +1,22 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package io.druid.java.util.common.concurrent; import org.junit.Assert; From 0ac9e1a69259b3d4008ac97f89af4602eb22adc8 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 21 Feb 2018 20:56:19 -0800 Subject: [PATCH 03/12] Linter fixes --- .../main/java/io/druid/java/util/common/concurrent/Execs.java | 3 ++- .../util/common/concurrent/SameThreadExecutorServiceTest.java | 3 ++- .../druid/java/util/common/guava/WithEffectSequenceTest.java | 1 - 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index 3adddfe33a35..ae8b8822d4f8 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -51,7 +51,8 @@ public static ExecutorService dummy() return DummyExecutorService.INSTANCE; } - public static ListeningExecutorService sameThreadExecutor() { + public static ListeningExecutorService sameThreadExecutor() + { return MoreExecutors.listeningDecorator(new SameThreadExecutorService()); } diff --git a/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java b/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java index 45792ff9823a..30b49baf808c 100644 --- a/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java @@ -62,7 +62,8 @@ public void hangingTaskTest() throws Exception } @Test - public void testMultiShutdownIsFine() { + public void testMultiShutdownIsFine() + { final SameThreadExecutorService service = new SameThreadExecutorService(); Assert.assertFalse(service.isShutdown()); service.shutdown(); diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java index d03ad987563c..438daa40159c 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java @@ -19,7 +19,6 @@ package io.druid.java.util.common.guava; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.java.util.common.concurrent.Execs; import org.junit.Assert; import org.junit.Test; From 5784383603499ca7719db89daf096368a3bc00dd Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 21 Feb 2018 21:09:16 -0800 Subject: [PATCH 04/12] More linter --- ...penderatorDriverRealtimeIndexTaskTest.java | 68 +++++++++++++------ .../query/groupby/GroupByQueryRunnerTest.java | 1 - ...inatorBasedSegmentHandoffNotifierTest.java | 1 - .../io/druid/server/RequestLogLineTest.java | 49 ------------- 4 files changed, 46 insertions(+), 73 deletions(-) delete mode 100644 server/src/test/java/io/druid/server/RequestLogLineTest.java diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index a64ba23142c3..6ddaf1aabef9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -151,7 +151,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; public class AppenderatorDriverRealtimeIndexTaskTest @@ -216,7 +215,8 @@ public InputRow nextRow() @Override public Runnable commit() { - return () -> {}; + return () -> { + }; } @Override @@ -362,11 +362,17 @@ public void testBasics() throws Exception for (DataSegment publishedSegment : publishedSegments) { Optional>> optional = handOffCallbacks.entrySet().stream() - .filter(e -> e.getKey().equals(new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ))) + .filter(e -> e.getKey() + .equals( + new SegmentDescriptor( + publishedSegment + .getInterval(), + publishedSegment + .getVersion(), + publishedSegment + .getShardSpec() + .getPartitionNum() + ))) .findFirst(); Assert.assertTrue( @@ -426,11 +432,17 @@ public void testLateData() throws Exception for (DataSegment publishedSegment : publishedSegments) { Optional>> optional = handOffCallbacks.entrySet().stream() - .filter(e -> e.getKey().equals(new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ))) + .filter(e -> e.getKey() + .equals( + new SegmentDescriptor( + publishedSegment + .getInterval(), + publishedSegment + .getVersion(), + publishedSegment + .getShardSpec() + .getPartitionNum() + ))) .findFirst(); Assert.assertTrue( @@ -493,11 +505,17 @@ public void testMaxRowsPerSegment() throws Exception for (DataSegment publishedSegment : publishedSegments) { Optional>> optional = handOffCallbacks.entrySet().stream() - .filter(e -> e.getKey().equals(new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ))) + .filter(e -> e.getKey() + .equals( + new SegmentDescriptor( + publishedSegment + .getInterval(), + publishedSegment + .getVersion(), + publishedSegment + .getShardSpec() + .getPartitionNum() + ))) .findFirst(); Assert.assertTrue( @@ -565,11 +583,17 @@ public void testTransformSpec() throws Exception for (DataSegment publishedSegment : publishedSegments) { Optional>> optional = handOffCallbacks.entrySet().stream() - .filter(e -> e.getKey().equals(new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ))) + .filter(e -> e.getKey() + .equals( + new SegmentDescriptor( + publishedSegment + .getInterval(), + publishedSegment + .getVersion(), + publishedSegment + .getShardSpec() + .getPartitionNum() + ))) .findFirst(); Assert.assertTrue( diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index fcfd7ff79c70..ae8b810cb0b1 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -30,7 +30,6 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.collections.BlockingPool; import io.druid.collections.DefaultBlockingPool; import io.druid.collections.NonBlockingPool; diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java index 023203395779..906ae81c1a52 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.client.ImmutableSegmentLoadInfo; import io.druid.client.coordinator.CoordinatorClient; import io.druid.java.util.common.Intervals; diff --git a/server/src/test/java/io/druid/server/RequestLogLineTest.java b/server/src/test/java/io/druid/server/RequestLogLineTest.java deleted file mode 100644 index 81601af9912a..000000000000 --- a/server/src/test/java/io/druid/server/RequestLogLineTest.java +++ /dev/null @@ -1,49 +0,0 @@ -package io.druid.server; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import io.druid.jackson.DefaultObjectMapper; -import io.druid.java.util.common.Intervals; -import io.druid.java.util.common.granularity.Granularities; -import io.druid.query.TableDataSource; -import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.CountAggregatorFactory; -import io.druid.query.aggregation.PostAggregator; -import io.druid.query.aggregation.post.ConstantPostAggregator; -import io.druid.query.dimension.DefaultDimensionSpec; -import io.druid.query.spec.MultipleIntervalSegmentSpec; -import io.druid.query.topn.NumericTopNMetricSpec; -import io.druid.query.topn.TopNQuery; -import io.druid.segment.VirtualColumns; -import org.joda.time.DateTime; -import org.junit.Test; - -public class RequestLogLineTest -{ - - @Test - public void testSimple() throws Exception { - final RequestLogLine line = new RequestLogLine( - DateTime.now(), - "127.0.0.1", - new TopNQuery( - new TableDataSource("dummy"), - VirtualColumns.EMPTY, - new DefaultDimensionSpec("test", "test"), - new NumericTopNMetricSpec("metric1"), - 3, - new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2015-01-01/2015-01-02"))), - null, - Granularities.ALL, - ImmutableList.of(new CountAggregatorFactory("metric1")), - ImmutableList.of(new ConstantPostAggregator("post", 10)), - null - ), - new QueryStats( - ImmutableMap.of() - ) - ); - final String print = line.getLine(new DefaultObjectMapper()); - System.out.println(print); - } -} \ No newline at end of file From 28d234227013dd6cd996eb58c2f7403c927fcfe0 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 22 Feb 2018 07:46:16 -0800 Subject: [PATCH 05/12] Manual formattting --- .../kafka/supervisor/KafkaSupervisor.java | 27 ++++++++++--------- ...penderatorDriverRealtimeIndexTaskTest.java | 27 ++++++++++--------- .../indexing/overlord/TaskLifecycleTest.java | 8 ++++-- 3 files changed, 35 insertions(+), 27 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index ff1d2d37d201..3fe42e97369e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -1770,18 +1770,21 @@ private void createKafkaTasksForGroup(int groupId, int replicas) throws JsonProc final String checkpoints = sortingMapper.writerWithType(new TypeReference>>() { }).writeValueAsString(taskGroups.get(groupId).sequenceOffsets); - final Map context = spec.getContext() == null - ? ImmutableMap.of( - "checkpoints", - checkpoints, - IS_INCREMENTAL_HANDOFF_SUPPORTED, - true - ) - : ImmutableMap.builder() - .put("checkpoints", checkpoints) - .put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true) - .putAll(spec.getContext()) - .build(); + final Map context; + if (spec.getContext() == null) { + context = ImmutableMap.of( + "checkpoints", + checkpoints, + IS_INCREMENTAL_HANDOFF_SUPPORTED, + true + ); + } else { + context = ImmutableMap.builder() + .put("checkpoints", checkpoints) + .put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true) + .putAll(spec.getContext()) + .build(); + } for (int i = 0; i < replicas; i++) { String taskId = Joiner.on("_").join(sequenceName, getRandomId()); KafkaIndexTask indexTask = new KafkaIndexTask( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 6ddaf1aabef9..00874e406e4d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -361,19 +361,20 @@ public void testBasics() throws Exception awaitHandoffs(); for (DataSegment publishedSegment : publishedSegments) { - Optional>> optional = handOffCallbacks.entrySet().stream() - .filter(e -> e.getKey() - .equals( - new SegmentDescriptor( - publishedSegment - .getInterval(), - publishedSegment - .getVersion(), - publishedSegment - .getShardSpec() - .getPartitionNum() - ))) - .findFirst(); + Optional>> optional = handOffCallbacks + .entrySet().stream() + .filter(e -> e.getKey() + .equals( + new SegmentDescriptor( + publishedSegment + .getInterval(), + publishedSegment + .getVersion(), + publishedSegment + .getShardSpec() + .getPartitionNum() + ))) + .findFirst(); Assert.assertTrue( publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index dcf6f0bd8341..2b12c653b648 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -529,8 +529,12 @@ private TaskToolboxFactory setUpTaskToolboxFactory( taskLockbox = new TaskLockbox(taskStorage); tac = new LocalTaskActionClientFactory( taskStorage, - new TaskActionToolbox(taskLockbox, mdc, emitter, EasyMock.createMock( - SupervisorManager.class)) + new TaskActionToolbox( + taskLockbox, + mdc, + emitter, + EasyMock.createMock(SupervisorManager.class) + ) ); File tmpDir = temporaryFolder.newFolder(); taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, false, null, null); From 8ad1173195c15ab004ca7ae41214ffe74e0185fe Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 22 Feb 2018 07:49:24 -0800 Subject: [PATCH 06/12] Prettier formatting --- ...penderatorDriverRealtimeIndexTaskTest.java | 120 +++++++++--------- 1 file changed, 60 insertions(+), 60 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 00874e406e4d..64cedf0b28e5 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -363,17 +363,16 @@ public void testBasics() throws Exception for (DataSegment publishedSegment : publishedSegments) { Optional>> optional = handOffCallbacks .entrySet().stream() - .filter(e -> e.getKey() - .equals( - new SegmentDescriptor( - publishedSegment - .getInterval(), - publishedSegment - .getVersion(), - publishedSegment - .getShardSpec() - .getPartitionNum() - ))) + .filter(e -> e + .getKey() + .equals( + new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ) + ) + ) .findFirst(); Assert.assertTrue( @@ -432,19 +431,19 @@ public void testLateData() throws Exception awaitHandoffs(); for (DataSegment publishedSegment : publishedSegments) { - Optional>> optional = handOffCallbacks.entrySet().stream() - .filter(e -> e.getKey() - .equals( - new SegmentDescriptor( - publishedSegment - .getInterval(), - publishedSegment - .getVersion(), - publishedSegment - .getShardSpec() - .getPartitionNum() - ))) - .findFirst(); + Optional>> optional = handOffCallbacks + .entrySet().stream() + .filter(e -> e + .getKey() + .equals( + new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ) + ) + ) + .findFirst(); Assert.assertTrue( publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, @@ -505,19 +504,19 @@ public void testMaxRowsPerSegment() throws Exception awaitHandoffs(); for (DataSegment publishedSegment : publishedSegments) { - Optional>> optional = handOffCallbacks.entrySet().stream() - .filter(e -> e.getKey() - .equals( - new SegmentDescriptor( - publishedSegment - .getInterval(), - publishedSegment - .getVersion(), - publishedSegment - .getShardSpec() - .getPartitionNum() - ))) - .findFirst(); + Optional>> optional = handOffCallbacks + .entrySet().stream() + .filter(e -> e + .getKey() + .equals( + new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ) + ) + ) + .findFirst(); Assert.assertTrue( publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, @@ -583,19 +582,19 @@ public void testTransformSpec() throws Exception awaitHandoffs(); for (DataSegment publishedSegment : publishedSegments) { - Optional>> optional = handOffCallbacks.entrySet().stream() - .filter(e -> e.getKey() - .equals( - new SegmentDescriptor( - publishedSegment - .getInterval(), - publishedSegment - .getVersion(), - publishedSegment - .getShardSpec() - .getPartitionNum() - ))) - .findFirst(); + Optional>> optional = handOffCallbacks + .entrySet().stream() + .filter(e -> e + .getKey() + .equals( + new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ) + ) + ) + .findFirst(); Assert.assertTrue( publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, @@ -1287,16 +1286,17 @@ public List getLocations() public long sumMetric(final Task task, final DimFilter filter, final String metric) throws Exception { // Do a query. - TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() - .dataSource("test_ds") - .filters(filter) - .aggregators( - ImmutableList.of( - new LongSumAggregatorFactory(metric, metric) - ) - ).granularity(Granularities.ALL) - .intervals("2000/3000") - .build(); + TimeseriesQuery query = Druids + .newTimeseriesQueryBuilder() + .dataSource("test_ds") + .filters(filter) + .aggregators( + ImmutableList.of( + new LongSumAggregatorFactory(metric, metric) + ) + ).granularity(Granularities.ALL) + .intervals("2000/3000") + .build(); List> results = task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList(); From 9c6f4b5fedfe921d91b77798c51fed9527c4b61f Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 23 Feb 2018 18:57:19 -0800 Subject: [PATCH 07/12] Simply the same thread executor service --- .../concurrent/SameThreadExecutorService.java | 45 +++++------------ .../SameThreadExecutorServiceTest.java | 48 +++++++------------ 2 files changed, 30 insertions(+), 63 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java index fb340612800a..0f66e03a21b5 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java @@ -19,72 +19,53 @@ package io.druid.java.util.common.concurrent; -import java.util.Collections; import java.util.List; import java.util.concurrent.AbstractExecutorService; -import java.util.concurrent.Phaser; -import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; +/** + * A simple class that implements the ExecutorService interface, but runs the code on a call to submit + */ public class SameThreadExecutorService extends AbstractExecutorService { - private final AtomicBoolean shutdownLeader = new AtomicBoolean(true); - private final Phaser shutdownPhaser = new Phaser(0); - private final int initialPhase = shutdownPhaser.register(); @Override public void shutdown() { - if (shutdownLeader.getAndSet(false)) { - shutdownPhaser.arriveAndDeregister(); - } + throw new UnsupportedOperationException(); } @Override public List shutdownNow() { - shutdown(); - return Collections.emptyList(); + throw new UnsupportedOperationException(); } @Override public boolean isShutdown() { - return !shutdownLeader.get(); + return false; } @Override public boolean isTerminated() { - return isShutdown() && shutdownPhaser.getRegisteredParties() < 1; + return false; } @Override public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException { - try { - shutdownPhaser.awaitAdvanceInterruptibly(initialPhase, timeout, unit); - return true; - } - catch (TimeoutException ignored) { - return false; - } + final long nanos = TimeUnit.NANOSECONDS.convert(timeout, unit); + final long millis = TimeUnit.MILLISECONDS.convert(timeout, unit); + final int sleepNanos = (int) (nanos - millis * 1_000_000L); + Thread.sleep(millis, sleepNanos); + return false; } @Override public void execute(Runnable command) { - shutdownPhaser.register(); - try { - if (isShutdown()) { - throw new RejectedExecutionException(); - } - command.run(); - } - finally { - shutdownPhaser.arriveAndDeregister(); - } + command.run(); } } diff --git a/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java b/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java index 30b49baf808c..21d5b658fe58 100644 --- a/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java @@ -23,52 +23,38 @@ import org.junit.Test; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; public class SameThreadExecutorServiceTest { + private final SameThreadExecutorService service = new SameThreadExecutorService(); + @Test public void timeoutAndShutdownTest() throws Exception { - final SameThreadExecutorService service = new SameThreadExecutorService(); Assert.assertFalse(service.awaitTermination(10, TimeUnit.MILLISECONDS)); - service.shutdown(); - Assert.assertTrue(service.awaitTermination(10, TimeUnit.MILLISECONDS)); } @Test - public void hangingTaskTest() throws Exception + public void runsTasks() throws Exception { - final CountDownLatch latch = new CountDownLatch(1); - final SameThreadExecutorService service = new SameThreadExecutorService(); - // Runnable gets called on submit, use the common pool to do the submit action - ForkJoinPool.commonPool().submit( - () -> service.submit(() -> { - try { - latch.await(); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - }) - ); - Assert.assertFalse(service.awaitTermination(10, TimeUnit.MILLISECONDS)); - service.shutdown(); - Assert.assertFalse(service.awaitTermination(10, TimeUnit.MILLISECONDS)); - latch.countDown(); - Assert.assertTrue(service.awaitTermination(10, TimeUnit.MILLISECONDS)); - Assert.assertTrue(service.isTerminated()); + final CountDownLatch finished = new CountDownLatch(1); + service.submit(finished::countDown); + finished.await(); } - @Test - public void testMultiShutdownIsFine() + @Test(expected = UnsupportedOperationException.class) + public void cannotShutdown() throws Exception { - final SameThreadExecutorService service = new SameThreadExecutorService(); - Assert.assertFalse(service.isShutdown()); - service.shutdown(); - Assert.assertTrue(service.isShutdown()); service.shutdown(); - Assert.assertTrue(service.isShutdown()); + } + + @Test(expected = ExecutionException.class) + public void exceptionsCaught() throws Exception + { + service.submit(() -> { + throw new RuntimeException(); + }).get(); } } From d5a023314ac19fc3ce1bc1af62153c34d2113a33 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 23 Feb 2018 19:02:07 -0800 Subject: [PATCH 08/12] Address code comments --- .../io/druid/indexing/kafka/supervisor/KafkaSupervisor.java | 3 ++- .../common/task/AppenderatorDriverRealtimeIndexTaskTest.java | 4 ++-- .../io/druid/indexing/common/task/RealtimeIndexTaskTest.java | 4 ++-- .../java/io/druid/java/util/common/concurrent/Execs.java | 5 ++++- 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 3fe42e97369e..82a5152c8224 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -547,7 +547,8 @@ public void statusChanged(String taskId, TaskStatus status) { notices.add(new RunNotice()); } - }, Execs.sameThreadExecutor() + }, + Execs.sameThreadExecutor() ); listenerRegistered = true; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 64cedf0b28e5..ff216c20cde4 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -120,6 +120,7 @@ import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.NumberedShardSpec; +import io.druid.utils.Runnables; import org.apache.commons.io.FileUtils; import org.easymock.EasyMock; import org.hamcrest.CoreMatchers; @@ -215,8 +216,7 @@ public InputRow nextRow() @Override public Runnable commit() { - return () -> { - }; + return Runnables.getNoopRunnable(); } @Override diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 5325f4862826..1af02c92c12a 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -119,6 +119,7 @@ import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; +import io.druid.utils.Runnables; import org.easymock.EasyMock; import org.hamcrest.CoreMatchers; import org.joda.time.DateTime; @@ -207,8 +208,7 @@ public InputRow nextRow() @Override public Runnable commit() { - return () -> { - }; + return Runnables.getNoopRunnable(); } @Override diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index ae8b8822d4f8..d0a8b013b98f 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -43,6 +43,9 @@ */ public class Execs { + private static final ListeningExecutorService SAME_THREAD_EXECUTOR_SERVICE_INSTANCE = MoreExecutors + .listeningDecorator(new SameThreadExecutorService()); + /** * Returns an ExecutorService which is terminated and shutdown from the beginning and not able to accept any tasks. */ @@ -53,7 +56,7 @@ public static ExecutorService dummy() public static ListeningExecutorService sameThreadExecutor() { - return MoreExecutors.listeningDecorator(new SameThreadExecutorService()); + return SAME_THREAD_EXECUTOR_SERVICE_INSTANCE; } public static ExecutorService singleThreaded(@NotNull String nameFormat) From 40ae3cdda819557494c628111f09739d56cc268e Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 26 Feb 2018 18:05:42 -0800 Subject: [PATCH 09/12] More comments and druid-forbidden-apis.txt * SameTheadExecutorService constructor now package private --- codestyle/druid-forbidden-apis.txt | 3 ++- .../java/io/druid/java/util/common/concurrent/Execs.java | 5 +++++ .../util/common/concurrent/SameThreadExecutorService.java | 5 +++++ 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/codestyle/druid-forbidden-apis.txt b/codestyle/druid-forbidden-apis.txt index abc3d6e00f4d..bf7e46fdad3f 100644 --- a/codestyle/druid-forbidden-apis.txt +++ b/codestyle/druid-forbidden-apis.txt @@ -1,2 +1,3 @@ com.google.common.collect.MapMaker @ Create java.util.concurrent.ConcurrentHashMap directly -com.google.common.collect.Maps#newConcurrentMap() @ Create java.util.concurrent.ConcurrentHashMap directly \ No newline at end of file +com.google.common.collect.Maps#newConcurrentMap() @ Create java.util.concurrent.ConcurrentHashMap directly +com.google.common.util.concurrent.MoreExecutors#sameThreadExecutor() @ Use io.druid.java.util.common.concurrent.Execs#sameThreadExecutor diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index d0a8b013b98f..c61fc8502294 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -54,6 +54,11 @@ public static ExecutorService dummy() return DummyExecutorService.INSTANCE; } + /** + * Guava changes the name from `MoreExecutors.sameThreadExecutor()` to `MoreExecutors.directExecutor()` + * quite abruptly. This serves as a compatability method to work in both environments for the common use cases + * in druid.io + */ public static ListeningExecutorService sameThreadExecutor() { return SAME_THREAD_EXECUTOR_SERVICE_INSTANCE; diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java index 0f66e03a21b5..820de1aa3898 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java @@ -28,6 +28,11 @@ */ public class SameThreadExecutorService extends AbstractExecutorService { + // Use io.druid.java.util.common.concurrent.Execs#sameThreadExecutor() + SameThreadExecutorService() + { + + } @Override public void shutdown() From c559a88b526d7ac68eb47a96dd0dc50e5e7822ea Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 12 Mar 2018 07:26:52 -0700 Subject: [PATCH 10/12] Move around some comments --- .../java/util/common/concurrent/SameThreadExecutorService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java index 820de1aa3898..791ff7ac6bce 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java @@ -25,10 +25,10 @@ /** * A simple class that implements the ExecutorService interface, but runs the code on a call to submit + * Use io.druid.java.util.common.concurrent.Execs#sameThreadExecutor() to get the instance */ public class SameThreadExecutorService extends AbstractExecutorService { - // Use io.druid.java.util.common.concurrent.Execs#sameThreadExecutor() SameThreadExecutorService() { From 730a7e015f529ea34cbf42b9d25fc9b0d755b888 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 19 Mar 2018 10:48:51 -0700 Subject: [PATCH 11/12] Add javadoc link --- .../java/util/common/concurrent/SameThreadExecutorService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java index 791ff7ac6bce..976a528427fa 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java @@ -25,7 +25,7 @@ /** * A simple class that implements the ExecutorService interface, but runs the code on a call to submit - * Use io.druid.java.util.common.concurrent.Execs#sameThreadExecutor() to get the instance + * Use {@link io.druid.java.util.common.concurrent.Execs#sameThreadExecutor() Execs.sameThreadExecutor()} to get the instance */ public class SameThreadExecutorService extends AbstractExecutorService { From 0f4cf1f638e31430918dc6f313b84d48d57904bd Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 20 Mar 2018 09:47:16 -0700 Subject: [PATCH 12/12] Allow instances of SameThreadExecutorService --- .../java/util/common/concurrent/Execs.java | 5 +-- .../concurrent/SameThreadExecutorService.java | 45 ++++++++++++++----- .../SameThreadExecutorServiceTest.java | 13 ++++-- 3 files changed, 45 insertions(+), 18 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index c61fc8502294..dd1a1b13a04e 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -43,9 +43,6 @@ */ public class Execs { - private static final ListeningExecutorService SAME_THREAD_EXECUTOR_SERVICE_INSTANCE = MoreExecutors - .listeningDecorator(new SameThreadExecutorService()); - /** * Returns an ExecutorService which is terminated and shutdown from the beginning and not able to accept any tasks. */ @@ -61,7 +58,7 @@ public static ExecutorService dummy() */ public static ListeningExecutorService sameThreadExecutor() { - return SAME_THREAD_EXECUTOR_SERVICE_INSTANCE; + return MoreExecutors.listeningDecorator(new SameThreadExecutorService()); } public static ExecutorService singleThreaded(@NotNull String nameFormat) diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java index 976a528427fa..99b8b224ffce 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java @@ -19,16 +19,25 @@ package io.druid.java.util.common.concurrent; +import java.util.Collections; import java.util.List; import java.util.concurrent.AbstractExecutorService; +import java.util.concurrent.Phaser; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; /** * A simple class that implements the ExecutorService interface, but runs the code on a call to submit - * Use {@link io.druid.java.util.common.concurrent.Execs#sameThreadExecutor() Execs.sameThreadExecutor()} to get the instance + * Use {@link Execs#sameThreadExecutor() Execs.sameThreadExecutor()} to get the instance */ public class SameThreadExecutorService extends AbstractExecutorService { + private final AtomicBoolean shutdownLeader = new AtomicBoolean(true); + private final Phaser shutdownPhaser = new Phaser(0); + private final int initialPhase = shutdownPhaser.register(); + SameThreadExecutorService() { @@ -37,40 +46,54 @@ public class SameThreadExecutorService extends AbstractExecutorService @Override public void shutdown() { - throw new UnsupportedOperationException(); + if (shutdownLeader.getAndSet(false)) { + shutdownPhaser.arriveAndDeregister(); + } } @Override public List shutdownNow() { - throw new UnsupportedOperationException(); + shutdown(); + return Collections.emptyList(); } @Override public boolean isShutdown() { - return false; + return !shutdownLeader.get(); } @Override public boolean isTerminated() { - return false; + return isShutdown() && shutdownPhaser.getRegisteredParties() < 1; } @Override public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException { - final long nanos = TimeUnit.NANOSECONDS.convert(timeout, unit); - final long millis = TimeUnit.MILLISECONDS.convert(timeout, unit); - final int sleepNanos = (int) (nanos - millis * 1_000_000L); - Thread.sleep(millis, sleepNanos); - return false; + try { + shutdownPhaser.awaitAdvanceInterruptibly(initialPhase, timeout, unit); + return true; + } + catch (TimeoutException e) { + return false; + } } @Override public void execute(Runnable command) { - command.run(); + shutdownPhaser.register(); + try { + if (isShutdown()) { + throw new RejectedExecutionException(); + } + command.run(); + } + finally { + shutdownPhaser.arriveAndDeregister(); + } } } diff --git a/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java b/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java index 21d5b658fe58..8b2622a3ae71 100644 --- a/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java @@ -28,31 +28,38 @@ public class SameThreadExecutorServiceTest { - private final SameThreadExecutorService service = new SameThreadExecutorService(); @Test public void timeoutAndShutdownTest() throws Exception { + final SameThreadExecutorService service = new SameThreadExecutorService(); Assert.assertFalse(service.awaitTermination(10, TimeUnit.MILLISECONDS)); } @Test public void runsTasks() throws Exception { + final SameThreadExecutorService service = new SameThreadExecutorService(); final CountDownLatch finished = new CountDownLatch(1); service.submit(finished::countDown); finished.await(); } - @Test(expected = UnsupportedOperationException.class) - public void cannotShutdown() throws Exception + @Test + public void multiShutdownIsFine() { + final SameThreadExecutorService service = new SameThreadExecutorService(); + Assert.assertFalse(service.isShutdown()); + service.shutdown(); + Assert.assertTrue(service.isShutdown()); service.shutdown(); + Assert.assertTrue(service.isShutdown()); } @Test(expected = ExecutionException.class) public void exceptionsCaught() throws Exception { + final SameThreadExecutorService service = new SameThreadExecutorService(); service.submit(() -> { throw new RuntimeException(); }).get();