diff --git a/codestyle/druid-forbidden-apis.txt b/codestyle/druid-forbidden-apis.txt index be5c40ffe7ad..25735d010e8c 100644 --- a/codestyle/druid-forbidden-apis.txt +++ b/codestyle/druid-forbidden-apis.txt @@ -20,6 +20,9 @@ com.google.common.collect.Sets#newHashSet() @ Create java.util.HashSet directly com.google.common.collect.Sets#newLinkedHashSet() @ Create java.util.LinkedHashSet directly com.google.common.collect.Sets#newTreeSet() @ Create java.util.TreeSet directly com.google.common.collect.Sets#newTreeSet(java.util.Comparator) @ Create java.util.TreeSet directly +com.google.common.util.concurrent.MoreExecutors#sameThreadExecutor() @ Use org.apache.druid.java.util.common.concurrent.Execs#directExecutor() +com.google.common.util.concurrent.MoreExecutors#newDirectExecutorService() @ Use org.apache.druid.java.util.common.concurrent.Execs#directExecutor() +com.google.common.util.concurrent.MoreExecutors#directExecutor() @ Use org.apache.druid.java.util.common.concurrent.Execs#directExecutor() com.google.common.util.concurrent.Futures#transform(com.google.common.util.concurrent.ListenableFuture, com.google.common.util.concurrent.AsyncFunction) @ Use org.apache.druid.java.util.common.concurrent.ListenableFutures#transformAsync java.io.File#toURL() @ Use java.io.File#toURI() and java.net.URI#toURL() instead java.lang.String#matches(java.lang.String) @ Use startsWith(), endsWith(), contains(), or compile and cache a Pattern explicitly diff --git a/core/src/main/java/org/apache/druid/java/util/common/concurrent/DirectExecutorService.java b/core/src/main/java/org/apache/druid/java/util/common/concurrent/DirectExecutorService.java new file mode 100644 index 000000000000..c6924456a5b1 --- /dev/null +++ b/core/src/main/java/org/apache/druid/java/util/common/concurrent/DirectExecutorService.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.java.util.common.concurrent; + + +import com.google.common.util.concurrent.AbstractListeningExecutorService; + +import javax.annotation.concurrent.GuardedBy; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.TimeUnit; + +// Copy of Guava's Apache 2.0 licensed https://github.com/google/guava/blob/a5cafa67da64a12444037bd4f4c30c39a0c184aa/guava/src/com/google/common/util/concurrent/MoreExecutors.java#L240-L339 + +/** + * Creates an executor service that runs each task in the thread that invokes {@code + * execute/submit}, as in {@link java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy} This applies both to individually submitted + * tasks and to collections of tasks submitted via {@code invokeAll} or {@code invokeAny}. In the + * latter case, tasks will run serially on the calling thread. Tasks are run to completion before + * a {@code Future} is returned to the caller (unless the executor has been shutdown). + * + *

Although all tasks are immediately executed in the thread that submitted the task, this + * {@code ExecutorService} imposes a small locking overhead on each task submission in order to + * implement shutdown and termination behavior. + * + *

The implementation deviates from the {@code ExecutorService} specification with regards to + * the {@code shutdownNow} method. First, "best-effort" with regards to canceling running tasks is + * implemented as "no-effort". No interrupts or other attempts are made to stop threads executing + * tasks. Second, the returned list will always be empty, as any submitted task is considered to + * have started execution. This applies also to tasks given to {@code invokeAll} or {@code + * invokeAny} which are pending serial execution, even the subset of the tasks that have not yet + * started execution. It is unclear from the {@code ExecutorService} specification if these should + * be included, and it's much easier to implement the interpretation that they not be. Finally, a + * call to {@code shutdown} or {@code shutdownNow} may result in concurrent calls to {@code + * invokeAll/invokeAny} throwing RejectedExecutionException, although a subset of the tasks may + * already have been executed. + */ +public class DirectExecutorService extends AbstractListeningExecutorService +{ + + /** + * Lock used whenever accessing the state variables (runningTasks, shutdown) of the executor + */ + private final Object lock = new Object(); + + /* + * Conceptually, these two variables describe the executor being in + * one of three states: + * - Active: shutdown == false + * - Shutdown: runningTasks > 0 and shutdown == true + * - Terminated: runningTasks == 0 and shutdown == true + */ + @GuardedBy("lock") + private int runningTasks = 0; + + @GuardedBy("lock") + private boolean shutdown = false; + + @Override + public void execute(Runnable command) + { + startTask(); + try { + command.run(); + } + finally { + endTask(); + } + } + + @Override + public boolean isShutdown() + { + synchronized (lock) { + return shutdown; + } + } + + @Override + public void shutdown() + { + synchronized (lock) { + shutdown = true; + if (runningTasks == 0) { + lock.notifyAll(); + } + } + } + + // See newDirectExecutorService javadoc for unusual behavior of this method. + @Override + public List shutdownNow() + { + shutdown(); + return Collections.emptyList(); + } + + @Override + public boolean isTerminated() + { + synchronized (lock) { + return shutdown && runningTasks == 0; + } + } + + @Override + public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException + { + long nanos = unit.toNanos(timeout); + synchronized (lock) { + while (true) { + if (shutdown && runningTasks == 0) { + return true; + } else if (nanos <= 0) { + return false; + } else { + long now = System.nanoTime(); + TimeUnit.NANOSECONDS.timedWait(lock, nanos); + nanos -= System.nanoTime() - now; // subtract the actual time we waited + } + } + } + } + + /** + * Checks if the executor has been shut down and increments the running task count. + * + * @throws RejectedExecutionException if the executor has been previously shutdown + */ + private void startTask() + { + synchronized (lock) { + if (shutdown) { + throw new RejectedExecutionException("Executor already shutdown"); + } + runningTasks++; + } + } + + /** + * Decrements the running task count. + */ + private void endTask() + { + synchronized (lock) { + int numRunning = --runningTasks; + if (numRunning == 0) { + lock.notifyAll(); + } + } + } +} diff --git a/core/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java b/core/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java index 58cffb33c16f..a310e5675600 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java +++ b/core/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java @@ -21,6 +21,7 @@ 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.ThreadFactoryBuilder; import javax.annotation.Nullable; @@ -38,9 +39,11 @@ import java.util.concurrent.TimeUnit; /** + * */ public class Execs { + /** * Returns an ExecutorService which is terminated and shutdown from the beginning and not able to accept any tasks. */ @@ -152,4 +155,9 @@ public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) } ); } + + public static ListeningExecutorService directExecutor() + { + return new DirectExecutorService(); + } } diff --git a/core/src/test/java/org/apache/druid/concurrent/ExecsTest.java b/core/src/test/java/org/apache/druid/concurrent/ExecsTest.java index cedd99dcdd74..863441184bd6 100644 --- a/core/src/test/java/org/apache/druid/concurrent/ExecsTest.java +++ b/core/src/test/java/org/apache/druid/concurrent/ExecsTest.java @@ -118,4 +118,10 @@ public void run() blockingExecutor.shutdown(); producer.shutdown(); } + + @Test + public void testDirectExecutorFactory() + { + Assert.assertNotNull(Execs.directExecutor()); + } } diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/WithEffectSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/WithEffectSequenceTest.java index d90e6a138823..72182f5cdc7f 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/WithEffectSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/WithEffectSequenceTest.java @@ -19,7 +19,7 @@ package org.apache.druid.java.util.common.guava; -import com.google.common.util.concurrent.MoreExecutors; +import org.apache.druid.java.util.common.concurrent.Execs; import org.junit.Assert; import org.junit.Test; @@ -40,11 +40,11 @@ public void testConsistentEffectApplicationOrder() .simple(Arrays.asList(1, 2, 3)) .withEffect( () -> effect1.set(counter.incrementAndGet()), - MoreExecutors.sameThreadExecutor() + Execs.directExecutor() ) .withEffect( () -> effect2.set(counter.incrementAndGet()), - MoreExecutors.sameThreadExecutor() + Execs.directExecutor() ); // Run sequence via accumulate sequence.toList(); @@ -70,7 +70,7 @@ public void testEffectExecutedIfWrappedSequenceThrowsExceptionFromClose() }); final AtomicBoolean effectExecuted = new AtomicBoolean(); Sequence seqWithEffect = - throwingSeq.withEffect(() -> effectExecuted.set(true), MoreExecutors.sameThreadExecutor()); + throwingSeq.withEffect(() -> effectExecuted.set(true), Execs.directExecutor()); try { seqWithEffect.toList(); Assert.fail("expected RuntimeException"); diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/org/apache/druid/query/lookup/KafkaLookupExtractorFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/org/apache/druid/query/lookup/KafkaLookupExtractorFactory.java index 4fb1a2401116..e65c7a382184 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/org/apache/druid/query/lookup/KafkaLookupExtractorFactory.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/org/apache/druid/query/lookup/KafkaLookupExtractorFactory.java @@ -261,7 +261,7 @@ public void onFailure(Throwable t) } } }, - MoreExecutors.sameThreadExecutor() + Execs.directExecutor() ); this.future = future; final Stopwatch stopwatch = Stopwatch.createStarted(); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index f44877771fbd..0b6e7c9e0002 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -2452,7 +2452,7 @@ public List getLocations() EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, this::makeTimeseriesOnlyConglomerate, - MoreExecutors.sameThreadExecutor(), // queryExecutorService + Execs.directExecutor(), // queryExecutorService EasyMock.createMock(MonitorScheduler.class), new SegmentLoaderFactory( new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index cef97955db2e..31bedd9ae892 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -2765,7 +2765,7 @@ public List getLocations() EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, this::makeTimeseriesOnlyConglomerate, - MoreExecutors.sameThreadExecutor(), // queryExecutorService + Execs.directExecutor(), // queryExecutorService EasyMock.createMock(MonitorScheduler.class), new SegmentLoaderFactory( new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java index bd19c74d0f19..1b0ef259552b 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java @@ -20,8 +20,8 @@ package org.apache.druid.query.aggregation.variance; import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerTestHelper; @@ -47,6 +47,7 @@ import java.util.List; /** + * */ @RunWith(Parameterized.class) public class VarianceGroupByQueryTest @@ -72,7 +73,7 @@ public VarianceGroupByQueryTest( this.testName = testName; this.config = config; this.factory = factory; - this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(runner)); + this.runner = factory.mergeRunners(Execs.directExecutor(), ImmutableList.of(runner)); } @Test diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java index 9448206e116c..02c5c500153e 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java @@ -97,6 +97,7 @@ import java.util.concurrent.TimeoutException; /** + * */ public class IndexGeneratorJob implements Jobby { @@ -240,7 +241,8 @@ public Map getStats() Map metrics = TaskMetricsUtils.makeIngestionRowMetrics( jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_COUNTER).getValue(), - jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_WITH_ERRORS_COUNTER).getValue(), + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_WITH_ERRORS_COUNTER) + .getValue(), jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_UNPARSEABLE_COUNTER).getValue(), jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_THROWN_AWAY_COUNTER).getValue() ); @@ -345,7 +347,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( @@ -359,17 +363,17 @@ protected void innerMap( // and they contain the columns as they show up in the segment after ingestion, not what you would see in raw // data InputRowSerde.SerializeResult serializeResult = inputRow instanceof SegmentInputRow ? - InputRowSerde.toBytes( - typeHelperMap, - inputRow, - aggsForSerializingSegmentInputRow - ) - : - InputRowSerde.toBytes( - typeHelperMap, - inputRow, - aggregators - ); + InputRowSerde.toBytes( + typeHelperMap, + inputRow, + aggsForSerializingSegmentInputRow + ) + : + InputRowSerde.toBytes( + typeHelperMap, + inputRow, + aggregators + ); context.write( new SortableBytes( @@ -678,7 +682,7 @@ public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) ); persistExecutor = MoreExecutors.listeningDecorator(executorService); } else { - persistExecutor = MoreExecutors.sameThreadExecutor(); + persistExecutor = Execs.directExecutor(); } for (final BytesWritable bw : values) { @@ -786,7 +790,10 @@ public void doRun() // 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/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index ddd854d3bdad..25250ac0487c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1051,7 +1051,7 @@ public void statusChanged(String taskId, TaskStatus status) { notices.add(new RunNotice()); } - }, MoreExecutors.sameThreadExecutor() + }, Execs.directExecutor() ); listenerRegistered = true; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java index 53d14951b5d6..7e7c09893b9d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import org.apache.curator.framework.CuratorFramework; import org.apache.druid.curator.CuratorUtils; @@ -30,6 +29,7 @@ import org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.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.directExecutor()); 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/org/apache/druid/indexing/worker/WorkerTaskManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java index 561a6e1befad..0ec7b41089ff 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java @@ -28,7 +28,6 @@ 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 org.apache.druid.client.indexing.IndexingService; @@ -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.directExecutor() ); } @@ -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() //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/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 5c2824210608..59899feb8151 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1607,7 +1607,7 @@ public List getLocations() EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, () -> conglomerate, - MoreExecutors.sameThreadExecutor(), // queryExecutorService + Execs.directExecutor(), // queryExecutorService EasyMock.createMock(MonitorScheduler.class), new SegmentLoaderFactory( new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 73b5a2ce7079..5317b66e4a46 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -1076,7 +1076,7 @@ public List getLocations() EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, () -> conglomerate, - MoreExecutors.sameThreadExecutor(), // queryExecutorService + Execs.directExecutor(), // queryExecutorService EasyMock.createMock(MonitorScheduler.class), new SegmentLoaderFactory( new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 0f1a2f60cfc2..73c79b4960cf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/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 org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.MapCache; import org.apache.druid.data.input.Firehose; @@ -81,6 +80,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -611,7 +611,7 @@ public void unannounceSegments(Iterable segments) EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, () -> queryRunnerFactoryConglomerate, // query runner factory conglomerate corporation unionized collective - MoreExecutors.sameThreadExecutor(), // query executor service + Execs.directExecutor(), // query executor service monitorScheduler, // monitor scheduler new SegmentLoaderFactory( new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, new DefaultObjectMapper()) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java index 25e172ae141e..814a20f5e8cd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java @@ -27,7 +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 org.apache.curator.framework.CuratorFramework; import org.apache.druid.common.guava.DSuppliers; import org.apache.druid.discovery.DiscoveryDruidNode; @@ -49,6 +48,7 @@ import org.apache.druid.indexing.worker.TaskAnnouncement; import org.apache.druid.indexing.worker.Worker; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.DruidNode; @@ -73,6 +73,7 @@ import java.util.concurrent.atomic.AtomicReference; /** + * */ public class HttpRemoteTaskRunnerTest { @@ -91,7 +92,8 @@ public void testFreshStart() throws Exception HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( TestHelper.makeJsonMapper(), - new HttpRemoteTaskRunnerConfig() { + new HttpRemoteTaskRunnerConfig() + { @Override public int getPendingTasksRunnerNumThreads() { @@ -105,7 +107,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, @@ -184,7 +187,8 @@ public void testOneStuckTaskAssignmentDoesntBlockOthers() throws Exception HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( TestHelper.makeJsonMapper(), - new HttpRemoteTaskRunnerConfig() { + new HttpRemoteTaskRunnerConfig() + { @Override public int getPendingTasksRunnerNumThreads() { @@ -198,7 +202,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, @@ -284,7 +289,8 @@ public void testTaskRunnerRestart() throws Exception HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( TestHelper.makeJsonMapper(), - new HttpRemoteTaskRunnerConfig() { + new HttpRemoteTaskRunnerConfig() + { @Override public int getPendingTasksRunnerNumThreads() { @@ -298,7 +304,8 @@ public int getPendingTasksRunnerNumThreads() taskStorageMock, EasyMock.createNiceMock(CuratorFramework.class), new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null) - ) { + ) + { @Override protected WorkerHolder createWorkerHolder( ObjectMapper smileMapper, @@ -316,7 +323,8 @@ protected WorkerHolder createWorkerHolder( config, workersSyncExec, listener, - worker); + worker + ); } else { throw new ISE("No WorkerHolder for [%s].", worker.getHost()); } @@ -420,7 +428,8 @@ public void testWorkerDisapperAndReappearBeforeItsCleanup() throws Exception HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( TestHelper.makeJsonMapper(), - new HttpRemoteTaskRunnerConfig() { + new HttpRemoteTaskRunnerConfig() + { @Override public int getPendingTasksRunnerNumThreads() { @@ -434,7 +443,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, @@ -452,7 +462,8 @@ protected WorkerHolder createWorkerHolder( config, workersSyncExec, listener, - worker); + worker + ); } else { throw new ISE("No WorkerHolder for [%s].", worker.getHost()); } @@ -593,7 +604,8 @@ public void testWorkerDisapperAndReappearAfterItsCleanup() throws Exception HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( TestHelper.makeJsonMapper(), - new HttpRemoteTaskRunnerConfig() { + new HttpRemoteTaskRunnerConfig() + { @Override public Period getTaskCleanupTimeout() { @@ -607,7 +619,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, @@ -625,7 +638,8 @@ protected WorkerHolder createWorkerHolder( config, workersSyncExec, listener, - worker); + worker + ); } else { throw new ISE("No WorkerHolder for [%s].", worker.getHost()); } @@ -801,7 +815,8 @@ protected WorkerHolder createWorkerHolder( config, workersSyncExec, listener, - worker); + worker + ); } else { throw new ISE("No WorkerHolder for [%s].", worker.getHost()); } @@ -914,9 +929,10 @@ WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip1", 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() ); } @@ -969,7 +985,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( @@ -982,7 +1000,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, @@ -1003,7 +1023,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( @@ -1016,7 +1038,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, @@ -1228,7 +1252,7 @@ public void statusChanged(String taskId, TaskStatus status) listenerNotificationsAccumulator.add(ImmutableList.of(taskId, status)); } }, - MoreExecutors.sameThreadExecutor() + Execs.directExecutor() ); } diff --git a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java index cfc18e85b9f0..68a91d61b05c 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java @@ -24,9 +24,9 @@ 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 org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.MergeSequence; @@ -76,11 +76,13 @@ import java.util.Map; /** + * */ 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"; @@ -506,7 +508,7 @@ public Sequence run(QueryPlus queryPlus, Map responseConte public static IntervalChunkingQueryRunnerDecorator sameThreadIntervalChunkingQueryRunnerDecorator() { return new IntervalChunkingQueryRunnerDecorator( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), QueryRunnerTestHelper.NOOP_QUERYWATCHER, new ServiceEmitter("dummy", "dummy", new NoopEmitter()) ); diff --git a/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java b/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java index 2a5fc3da440f..7f8ea4c5af1e 100644 --- a/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java +++ b/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java @@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.io.Closeables; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -32,6 +31,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.FunctionalIterable; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.aggregation.CountAggregatorFactory; @@ -106,7 +106,7 @@ public static > List runQuery( final Sequence results = new FinalizeResultsQueryRunner<>( factory.getToolchest().mergeResults( factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), FunctionalIterable .create(indexes) .transform( diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java index 27137b9d4c1e..bd2c13fdd77e 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java @@ -32,7 +32,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.io.Closeables; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.commons.io.IOUtils; import org.apache.commons.io.LineIterator; import org.apache.druid.collections.CloseableStupidPool; @@ -42,6 +41,7 @@ import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.CloseQuietly; import org.apache.druid.java.util.common.guava.Sequence; @@ -607,7 +607,7 @@ public Sequence runQueryOnSegmentsObjs(final List segments, final toolChest.mergeResults( toolChest.preMergeQueryDecoration( factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), Lists.transform( segments, new Function() diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java index e65aaa295688..19f3a5862677 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -25,11 +25,11 @@ 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 org.apache.druid.collections.CloseableDefaultBlockingPool; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.QueryContexts; @@ -222,7 +222,7 @@ public static Collection constructorFeeder() public GroupByQueryMergeBufferTest(QueryRunner runner) { - this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(runner)); + this.runner = factory.mergeRunners(Execs.directExecutor(), ImmutableList.of(runner)); } @Before diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java index a54a210bc32c..9222ba5c7ace 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java @@ -24,11 +24,11 @@ import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.collections.CloseableDefaultBlockingPool; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.InsufficientResourcesException; @@ -183,7 +183,7 @@ public static Collection constructorFeeder() public GroupByQueryRunnerFailureTest(QueryRunner runner) { - this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(runner)); + this.runner = factory.mergeRunners(Execs.directExecutor(), ImmutableList.of(runner)); } @Test(timeout = 60_000L) diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index 646e2d33a4e2..9142e4d21da0 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java @@ -29,7 +29,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.collections.CloseableDefaultBlockingPool; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.common.config.NullHandling; @@ -40,6 +39,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.DurationGranularity; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.PeriodGranularity; @@ -441,7 +441,7 @@ public GroupByQueryRunnerTest( { this.config = config; this.factory = factory; - this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(runner)); + this.runner = factory.mergeRunners(Execs.directExecutor(), ImmutableList.of(runner)); } @Test @@ -452,33 +452,251 @@ public void testGroupBy() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index"), - new FloatSumAggregatorFactory("idxFloat", "indexFloat"), - new DoubleSumAggregatorFactory("idxDouble", "index")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index"), + new FloatSumAggregatorFactory("idxFloat", "indexFloat"), + new DoubleSumAggregatorFactory("idxDouble", "index") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .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); @@ -538,115 +756,295 @@ 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) - ); - - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - TestHelper.assertExpectedObjects(expectedResults, results, "string-postAgg"); - } - - @Test - public void testGroupByWithStringVirtualColumn() - { - GroupByQuery query = GroupByQuery - .builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setVirtualColumns( - new ExpressionVirtualColumn( - "vc", - "quality + 'x'", - ValueType.STRING, - TestExprMacroTable.INSTANCE - ) - ) - .setDimensions(new DefaultDimensionSpec("vc", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) - .setGranularity(QueryRunnerTestHelper.dayGran) - .build(); - - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotivex", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "businessx", "rows", 1L, "idx", 118L), GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "alias", - "entertainmentx", + "travel", + "post", + "travelx", "rows", 1L, "idx", - 158L + 119L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "healthx", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzaninex", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "newsx", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premiumx", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technologyx", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travelx", "rows", 1L, "idx", 119L), - - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotivex", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "businessx", "rows", 1L, "idx", 112L), GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", + "2011-04-01", "alias", - "entertainmentx", + "technology", + "post", + "technologyx", "rows", 1L, "idx", - 166L + 78L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "healthx", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzaninex", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "newsx", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premiumx", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technologyx", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travelx", "rows", 1L, "idx", 126L) - ); - - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - TestHelper.assertExpectedObjects(expectedResults, results, "virtual-column"); - } - - @Test - public void testGroupByWithDurationGranularity() - { - GroupByQuery query = GroupByQuery - .builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) - .setGranularity(new DurationGranularity(86400L, 0L)) - .build(); - - List 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", + "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); + TestHelper.assertExpectedObjects(expectedResults, results, "string-postAgg"); + } + + @Test + public void testGroupByWithStringVirtualColumn() + { + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setVirtualColumns( + new ExpressionVirtualColumn( + "vc", + "quality + 'x'", + ValueType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + .setDimensions(new DefaultDimensionSpec("vc", "alias")) + .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotivex", "rows", 1L, "idx", 135L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "businessx", "rows", 1L, "idx", 118L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "entertainmentx", + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "healthx", "rows", 1L, "idx", 120L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzaninex", "rows", 3L, "idx", 2870L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "newsx", "rows", 1L, "idx", 121L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premiumx", "rows", 3L, "idx", 2900L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technologyx", "rows", 1L, "idx", 78L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travelx", "rows", 1L, "idx", 119L), + + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotivex", "rows", 1L, "idx", 147L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "businessx", "rows", 1L, "idx", 112L), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "entertainmentx", + "rows", + 1L, + "idx", + 166L + ), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "healthx", "rows", 1L, "idx", 113L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzaninex", "rows", 3L, "idx", 2447L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "newsx", "rows", 1L, "idx", 114L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premiumx", "rows", 3L, "idx", 2505L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technologyx", "rows", 1L, "idx", 97L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travelx", "rows", 1L, "idx", 126L) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "virtual-column"); + } + + @Test + public void testGroupByWithDurationGranularity() + { + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(new DefaultDimensionSpec("quality", "alias")) + .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) + .setGranularity(new DurationGranularity(86400L, 0L)) + .build(); + + List 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-02", "alias", "automotive", "rows", 1L, "idx", 147L), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), @@ -838,29 +1236,69 @@ 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) - ); - - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - TestHelper.assertExpectedObjects(expectedResults, results, "two-multi-value-dims"); - } - - @Test - public void testMultipleDimensionsOneOfWhichIsMultiValue1() - { - GroupByQuery query = GroupByQuery - .builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions( - new DefaultDimensionSpec("placementish", "alias"), - new DefaultDimensionSpec("quality", "quality") - ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) - .setGranularity(QueryRunnerTestHelper.allGran) - .build(); + 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); + TestHelper.assertExpectedObjects(expectedResults, results, "two-multi-value-dims"); + } + + @Test + public void testMultipleDimensionsOneOfWhichIsMultiValue1() + { + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + new DefaultDimensionSpec("placementish", "alias"), + new DefaultDimensionSpec("quality", "quality") + ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow( @@ -1304,25 +1742,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) ); } @@ -1390,7 +1844,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), @@ -1400,7 +1862,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), @@ -1435,7 +1905,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), @@ -1445,7 +1923,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), @@ -1485,7 +1971,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), @@ -1495,7 +1989,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), @@ -1981,23 +2483,73 @@ public void testGroupByWithFirstLast() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval) .setDimensions(new DefaultDimensionSpec("market", "market")) - .setAggregatorSpecs(new LongFirstAggregatorFactory("first", "index"), - new LongLastAggregatorFactory("last", "index")) + .setAggregatorSpecs( + new LongFirstAggregatorFactory("first", "index"), + new LongLastAggregatorFactory("last", "index") + ) .setGranularity(QueryRunnerTestHelper.monthGran) .build(); 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) ); @@ -2013,11 +2565,13 @@ public void testGroupByWithNoResult() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.emptyInterval) .setDimensions(new DefaultDimensionSpec("market", "market")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.indexLongSum, - QueryRunnerTestHelper.qualityCardinality, - new LongFirstAggregatorFactory("first", "index"), - new LongLastAggregatorFactory("last", "index")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + QueryRunnerTestHelper.indexLongSum, + QueryRunnerTestHelper.qualityCardinality, + new LongFirstAggregatorFactory("first", "index"), + new LongLastAggregatorFactory("last", "index") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .build(); @@ -2716,7 +3270,8 @@ public void testGroupByOrderLimit() 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), @@ -2832,8 +3387,16 @@ public void testGroupByOrderLimitNumeric() .setInterval("2011-04-02/2011-04-04") .setDimensions(new DefaultDimensionSpec("quality", "alias")) .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, 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(); @@ -3597,10 +4160,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), @@ -4014,9 +4585,11 @@ public void testIdenticalSubquery() null, JavaScriptConfig.getEnabledInstance() )) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index"), - new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index"), + new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .build(); @@ -4070,9 +4643,11 @@ public void testSubqueryWithMultipleIntervalsInOuterQuery() null, JavaScriptConfig.getEnabledInstance() )) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index"), - new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index"), + new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .build(); @@ -4133,9 +4708,11 @@ public void testSubqueryWithMultipleIntervalsInOuterQueryAndChunkPeriod() null, JavaScriptConfig.getEnabledInstance() )) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index"), - new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index"), + new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .setContext(ImmutableMap.of("chunkPeriod", "P1D")) .build(); @@ -4199,9 +4776,11 @@ public void testSubqueryWithExtractionFnInOuterQuery() null, JavaScriptConfig.getEnabledInstance() )) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index"), - new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index"), + new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .build(); @@ -4238,9 +4817,11 @@ public void testDifferentGroupingSubquery() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index"), - new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index"), + new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .build(); @@ -4248,9 +4829,11 @@ public void testDifferentGroupingSubquery() .builder() .setDataSource(subquery) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new DoubleMaxAggregatorFactory("idx", "idx"), - new DoubleMaxAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new DoubleMaxAggregatorFactory("idx", "idx"), + new DoubleMaxAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .build(); @@ -4269,9 +4852,11 @@ public void testDifferentGroupingSubquery() .setVirtualColumns( new ExpressionVirtualColumn("expr", "-index + 100", ValueType.FLOAT, TestExprMacroTable.INSTANCE) ) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "expr"), - new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "expr"), + new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen") + ) .build(); query = (GroupByQuery) query.withDataSource(new QueryDataSource(subquery)); @@ -4315,10 +4900,12 @@ public void testDifferentGroupingSubqueryMultipleAggregatorsOnSameField() .builder() .setDataSource(subquery) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setAggregatorSpecs(new DoubleMaxAggregatorFactory("idx1", "idx"), - new DoubleMaxAggregatorFactory("idx2", "idx"), - new DoubleMaxAggregatorFactory("idx3", "post_agg"), - new DoubleMaxAggregatorFactory("idx4", "post_agg")) + .setAggregatorSpecs( + new DoubleMaxAggregatorFactory("idx1", "idx"), + new DoubleMaxAggregatorFactory("idx2", "idx"), + new DoubleMaxAggregatorFactory("idx3", "post_agg"), + new DoubleMaxAggregatorFactory("idx4", "post_agg") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .build(); @@ -4532,8 +5119,10 @@ public void testSubqueryWithPostAggregators() .setDataSource(subquery) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("alias", "alias")) - .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"), - new LongSumAggregatorFactory("idx", "idx_subpostagg")) + .setAggregatorSpecs( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx_subpostagg") + ) .setPostAggregatorSpecs( Collections.singletonList( new ArithmeticPostAggregator( @@ -4801,8 +5390,10 @@ public boolean eval(Row row) .setDataSource(subquery) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("alias", "alias")) - .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"), - new LongSumAggregatorFactory("idx", "idx_subpostagg")) + .setAggregatorSpecs( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx_subpostagg") + ) .setPostAggregatorSpecs( Collections.singletonList( new ArithmeticPostAggregator( @@ -5016,16 +5607,18 @@ public void testSubqueryWithMultiColumnAggregators() null, JavaScriptConfig.getEnabledInstance() )) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new DoubleSumAggregatorFactory("idx_subagg", "index"), - new JavaScriptAggregatorFactory( - "js_agg", - Arrays.asList("index", "market"), - "function(current, index, dim){return current + index + dim.length;}", - "function(){return 0;}", - "function(a,b){return a + b;}", - JavaScriptConfig.getEnabledInstance() - )) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new DoubleSumAggregatorFactory("idx_subagg", "index"), + new JavaScriptAggregatorFactory( + "js_agg", + Arrays.asList("index", "market"), + "function(current, index, dim){return current + index + dim.length;}", + "function(){return 0;}", + "function(a,b){return a + b;}", + JavaScriptConfig.getEnabledInstance() + ) + ) .setPostAggregatorSpecs( Collections.singletonList( new ArithmeticPostAggregator( @@ -5058,9 +5651,11 @@ public boolean eval(Row row) .setDataSource(subquery) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("alias", "alias")) - .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"), - new LongSumAggregatorFactory("idx", "idx_subpostagg"), - new DoubleSumAggregatorFactory("js_outer_agg", "js_agg")) + .setAggregatorSpecs( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx_subpostagg"), + new DoubleSumAggregatorFactory("js_outer_agg", "js_agg") + ) .setPostAggregatorSpecs( Collections.singletonList( new ArithmeticPostAggregator( @@ -5200,8 +5795,16 @@ public void testSubqueryWithOuterTimeFilter() .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - final DimFilter fridayFilter = new SelectorDimFilter(ColumnHolder.TIME_COLUMN_NAME, "Friday", new TimeFormatExtractionFn("EEEE", null, null, null, false)); - final DimFilter firstDaysFilter = new InDimFilter(ColumnHolder.TIME_COLUMN_NAME, ImmutableList.of("1", "2", "3"), new TimeFormatExtractionFn("d", null, null, null, false)); + final DimFilter fridayFilter = new SelectorDimFilter( + ColumnHolder.TIME_COLUMN_NAME, + "Friday", + new TimeFormatExtractionFn("EEEE", null, null, null, false) + ); + final DimFilter firstDaysFilter = new InDimFilter( + ColumnHolder.TIME_COLUMN_NAME, + ImmutableList.of("1", "2", "3"), + new TimeFormatExtractionFn("d", null, null, null, false) + ); final GroupByQuery query = GroupByQuery .builder() .setDataSource(subquery) @@ -5472,9 +6075,11 @@ public void testSubqueryWithHyperUniques() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index"), - new HyperUniquesAggregatorFactory("quality_uniques", "quality_uniques")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index"), + new HyperUniquesAggregatorFactory("quality_uniques", "quality_uniques") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .build(); @@ -5483,9 +6088,11 @@ public void testSubqueryWithHyperUniques() .setDataSource(subquery) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("alias", "alias")) - .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"), - new LongSumAggregatorFactory("idx", "idx"), - new HyperUniquesAggregatorFactory("uniq", "quality_uniques")) + .setAggregatorSpecs( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx"), + new HyperUniquesAggregatorFactory("uniq", "quality_uniques") + ) .setGranularity(QueryRunnerTestHelper.allGran) .build(); @@ -5517,313 +6124,1135 @@ public void testSubqueryWithHyperUniques() "alias", "entertainment", "rows", - 2L, + 2L, + "idx", + 324L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "health", + "rows", + 2L, + "idx", + 233L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "mezzanine", + "rows", + 6L, + "idx", + 5317L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "news", + "rows", + 2L, + "idx", + 235L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "premium", + "rows", + 6L, + "idx", + 5405L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "technology", + "rows", + 2L, + "idx", + 175L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "travel", + "rows", + 2L, + "idx", + 245L, + "uniq", + 1.0002442201269182 + ) + ); + + // Subqueries are handled by the ToolChest + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "subquery-hyperunique"); + } + + @Test + public void testSubqueryWithHyperUniquesPostAggregator() + { + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(new ArrayList<>()) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index"), + new HyperUniquesAggregatorFactory("quality_uniques_inner", "quality_uniques") + ) + .setPostAggregatorSpecs( + Collections.singletonList( + new FieldAccessPostAggregator("quality_uniques_inner_post", "quality_uniques_inner") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(new ArrayList<>()) + .setAggregatorSpecs( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx"), + new HyperUniquesAggregatorFactory("quality_uniques_outer", "quality_uniques_inner_post") + ) + .setPostAggregatorSpecs( + Collections.singletonList( + new HyperUniqueFinalizingPostAggregator("quality_uniques_outer_post", "quality_uniques_outer") + ) + ) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + List expectedResults = Collections.singletonList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "rows", + 26L, + "idx", + 12446L, + "quality_uniques_outer", + 9.019833517963864, + "quality_uniques_outer_post", + 9.019833517963864 + ) + ); + + // Subqueries are handled by the ToolChest + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "subquery-hyperunique"); + } + + @Test + public void testSubqueryWithFirstLast() + { + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval) + .setDimensions(new DefaultDimensionSpec("market", "market")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongFirstAggregatorFactory("innerfirst", "index"), + new LongLastAggregatorFactory("innerlast", "index") + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .setContext(ImmutableMap.of("finalize", true)) + .build(); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval) + .setDimensions(new ArrayList<>()) + .setAggregatorSpecs( + new LongFirstAggregatorFactory("first", "innerfirst"), + new LongLastAggregatorFactory("last", "innerlast") + ) + .setGranularity(QueryRunnerTestHelper.monthGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow("2011-01-01", "first", 100L, "last", 943L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-02-01", "first", 132L, "last", 1101L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-03-01", "first", 153L, "last", 1063L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "first", 135L, "last", 780L) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "subquery-firstlast"); + } + + @Test + public void testGroupByWithSubtotalsSpec() + { + if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { + return; + } + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("quality", "alias"), + new DefaultDimensionSpec("market", "market") + )) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index"), + new FloatSumAggregatorFactory("idxFloat", "indexFloat"), + new DoubleSumAggregatorFactory("idxDouble", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .setSubtotalsSpec(ImmutableList.of( + ImmutableList.of("alias"), + ImmutableList.of("market"), + ImmutableList.of() + )) + .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-01T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 643.043177, + "idxFloat", + 643.043212890625, + "rows", + 5L, + "idx", + 640L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "total_market", + "idxDouble", + 1314.839715, + "idxFloat", + 1314.8397, + "rows", + 1L, + "idx", + 1314L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "upfront", + "idxDouble", + 1447.34116, + "idxFloat", + 1447.3412, + "rows", + 1L, + "idx", + 1447L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 266.090949, + "idxFloat", + 266.0909423828125, + "rows", + 2L, + "idx", + 265L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "total_market", + "idxDouble", + 1522.043733, + "idxFloat", + 1522.0437, + "rows", + 1L, + "idx", + 1522L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "upfront", + "idxDouble", + 1234.247546, + "idxFloat", + 1234.2476, + "rows", + 1L, + "idx", + 1234L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 198.545289, + "idxFloat", + 198.5452880859375, + "rows", + 2L, + "idx", + 197L + ), + + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 650.806953, + "idxFloat", + 650.8069458007812, + "rows", + 5L, + "idx", + 648L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "market", + "total_market", + "idxDouble", + 1193.556278, + "idxFloat", + 1193.5563, + "rows", + 1L, + "idx", + 1193L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "market", + "upfront", + "idxDouble", + 1144.342401, + "idxFloat", + 1144.3424, + "rows", + 1L, + "idx", + 1144L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 249.591647, + "idxFloat", + 249.59164428710938, + "rows", + 2L, + "idx", + 249L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "market", + "total_market", + "idxDouble", + 1321.375057, + "idxFloat", + 1321.375, + "rows", + 1L, + "idx", + 1321L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "market", + "upfront", + "idxDouble", + 1049.738585, + "idxFloat", + 1049.7385, + "rows", + 1L, + "idx", + 1049L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 223.798797, + "idxFloat", + 223.79879760742188, + "rows", + 2L, + "idx", + 223L + ), + + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "idxDouble", + 6626.151575318359, + "idxFloat", + 6626.152f, + "rows", + 13L, + "idx", + 6619L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "idxDouble", + 5833.209713, + "idxFloat", + 5833.209f, + "rows", + 13L, + "idx", + 5827L + ) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "subtotal"); + } + + @Test + public void testGroupByWithSubtotalsSpecWithLongDimensionColumn() + { + if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { + return; + } + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("qualityLong", "ql", ValueType.LONG), + new DefaultDimensionSpec("market", "market") + )) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index"), + new FloatSumAggregatorFactory("idxFloat", "indexFloat"), + new DoubleSumAggregatorFactory("idxDouble", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .setSubtotalsSpec(ImmutableList.of( + ImmutableList.of("ql"), + ImmutableList.of("market"), + ImmutableList.of() + )) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "idxDouble", + 135.885094, + "idxFloat", + 135.8851, + "ql", + 1000L, + "rows", + 1L, + "idx", + 135L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "idxDouble", + 118.57034, + "idxFloat", + 118.57034, + "ql", + 1100L, + "rows", + 1L, + "idx", + 118L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "idxDouble", + 158.747224, + "idxFloat", + 158.74722, + "ql", + 1200L, + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "idxDouble", + 120.134704, + "idxFloat", + 120.134705, + "ql", + 1300L, + "rows", + 1L, + "idx", + 120L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "idxDouble", + 2871.8866900000003, + "idxFloat", + 2871.88671875, + "ql", + 1400L, + "rows", + 3L, + "idx", + 2870L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "idxDouble", + 121.583581, + "idxFloat", + 121.58358, + "ql", + 1500L, + "rows", + 1L, + "idx", + 121L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "idxDouble", + 2900.798647, + "idxFloat", + 2900.798583984375, + "ql", + 1600L, + "rows", + 3L, + "idx", + 2900L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "idxDouble", + 78.622547, + "idxFloat", + 78.62254, + "ql", + 1700L, + "rows", + 1L, + "idx", + 78L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "idxDouble", + 119.922742, + "idxFloat", + 119.922745, + "ql", + 1800L, + "rows", + 1L, + "idx", + 119L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "idxDouble", + 147.425935, + "idxFloat", + 147.42593, + "ql", + 1000L, + "rows", + 1L, + "idx", + 147L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "idxDouble", + 112.987027, + "idxFloat", + 112.98703, + "ql", + 1100L, + "rows", + 1L, + "idx", + 112L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "idxDouble", + 166.016049, + "idxFloat", + 166.01605, + "ql", + 1200L, + "rows", + 1L, + "idx", + 166L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "idxDouble", + 113.446008, + "idxFloat", + 113.44601, + "ql", + 1300L, + "rows", + 1L, + "idx", + 113L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "idxDouble", + 2448.830613, + "idxFloat", + 2448.83056640625, + "ql", + 1400L, + "rows", + 3L, + "idx", + 2447L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "idxDouble", + 114.290141, + "idxFloat", + 114.29014, + "ql", + 1500L, + "rows", + 1L, + "idx", + 114L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "idxDouble", + 2506.415148, + "idxFloat", + 2506.4150390625, + "ql", + 1600L, + "rows", + 3L, + "idx", + 2505L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "idxDouble", + 97.387433, + "idxFloat", + 97.387436, + "ql", + 1700L, + "rows", + 1L, + "idx", + 97L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "idxDouble", + 126.411364, + "idxFloat", + 126.41136, + "ql", + 1800L, + "rows", + 1L, + "idx", + 126L + ), + + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 643.043177, + "idxFloat", + 643.043212890625, + "rows", + 5L, + "idx", + 640L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "total_market", + "idxDouble", + 1314.839715, + "idxFloat", + 1314.8397, + "rows", + 1L, + "idx", + 1314L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "upfront", + "idxDouble", + 1447.34116, + "idxFloat", + 1447.3412, + "rows", + 1L, + "idx", + 1447L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 266.090949, + "idxFloat", + 266.0909423828125, + "rows", + 2L, + "idx", + 265L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "total_market", + "idxDouble", + 1522.043733, + "idxFloat", + 1522.0437, + "rows", + 1L, + "idx", + 1522L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "upfront", + "idxDouble", + 1234.247546, + "idxFloat", + 1234.2476, + "rows", + 1L, + "idx", + 1234L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 198.545289, + "idxFloat", + 198.5452880859375, + "rows", + 2L, + "idx", + 197L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 650.806953, + "idxFloat", + 650.8069458007812, + "rows", + 5L, "idx", - 324L, - "uniq", - 1.0002442201269182 + 648L ), GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "health", + "2011-04-02T00:00:00.000Z", + "market", + "total_market", + "idxDouble", + 1193.556278, + "idxFloat", + 1193.5563, "rows", - 2L, + 1L, "idx", - 233L, - "uniq", - 1.0002442201269182 + 1193L ), GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "mezzanine", + "2011-04-02T00:00:00.000Z", + "market", + "upfront", + "idxDouble", + 1144.342401, + "idxFloat", + 1144.3424, "rows", - 6L, + 1L, "idx", - 5317L, - "uniq", - 1.0002442201269182 + 1144L ), GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "news", + "2011-04-02T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 249.591647, + "idxFloat", + 249.59164428710938, "rows", 2L, "idx", - 235L, - "uniq", - 1.0002442201269182 + 249L ), GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "premium", + "2011-04-02T00:00:00.000Z", + "market", + "total_market", + "idxDouble", + 1321.375057, + "idxFloat", + 1321.375, "rows", - 6L, + 1L, "idx", - 5405L, - "uniq", - 1.0002442201269182 + 1321L ), GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "technology", + "2011-04-02T00:00:00.000Z", + "market", + "upfront", + "idxDouble", + 1049.738585, + "idxFloat", + 1049.7385, "rows", - 2L, + 1L, "idx", - 175L, - "uniq", - 1.0002442201269182 + 1049L ), GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "travel", + "2011-04-02T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 223.798797, + "idxFloat", + 223.79879760742188, "rows", 2L, "idx", - 245L, - "uniq", - 1.0002442201269182 - ) - ); - - // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - TestHelper.assertExpectedObjects(expectedResults, results, "subquery-hyperunique"); - } - - @Test - public void testSubqueryWithHyperUniquesPostAggregator() - { - GroupByQuery subquery = GroupByQuery - .builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(new ArrayList<>()) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index"), - new HyperUniquesAggregatorFactory("quality_uniques_inner", "quality_uniques")) - .setPostAggregatorSpecs( - Collections.singletonList( - new FieldAccessPostAggregator("quality_uniques_inner_post", "quality_uniques_inner") - ) - ) - .setGranularity(QueryRunnerTestHelper.dayGran) - .build(); - - GroupByQuery query = GroupByQuery - .builder() - .setDataSource(subquery) - .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(new ArrayList<>()) - .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"), - new LongSumAggregatorFactory("idx", "idx"), - new HyperUniquesAggregatorFactory("quality_uniques_outer", "quality_uniques_inner_post")) - .setPostAggregatorSpecs( - Collections.singletonList( - new HyperUniqueFinalizingPostAggregator("quality_uniques_outer_post", "quality_uniques_outer") - ) - ) - .setGranularity(QueryRunnerTestHelper.allGran) - .build(); + 223L + ), - List expectedResults = Collections.singletonList( GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", + "2011-04-01T00:00:00.000Z", + "idxDouble", + 6626.151569, + "idxFloat", + 6626.1513671875, "rows", - 26L, + 13L, "idx", - 12446L, - "quality_uniques_outer", - 9.019833517963864, - "quality_uniques_outer_post", - 9.019833517963864 - ) - ); - - // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - TestHelper.assertExpectedObjects(expectedResults, results, "subquery-hyperunique"); - } - - @Test - public void testSubqueryWithFirstLast() - { - GroupByQuery subquery = GroupByQuery - .builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval) - .setDimensions(new DefaultDimensionSpec("market", "market")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongFirstAggregatorFactory("innerfirst", "index"), - new LongLastAggregatorFactory("innerlast", "index")) - .setGranularity(QueryRunnerTestHelper.dayGran) - .setContext(ImmutableMap.of("finalize", true)) - .build(); - - GroupByQuery query = GroupByQuery - .builder() - .setDataSource(subquery) - .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval) - .setDimensions(new ArrayList<>()) - .setAggregatorSpecs(new LongFirstAggregatorFactory("first", "innerfirst"), - new LongLastAggregatorFactory("last", "innerlast")) - .setGranularity(QueryRunnerTestHelper.monthGran) - .build(); - - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-01-01", "first", 100L, "last", 943L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-02-01", "first", 132L, "last", 1101L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-03-01", "first", 153L, "last", 1063L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "first", 135L, "last", 780L) - ); - - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - TestHelper.assertExpectedObjects(expectedResults, results, "subquery-firstlast"); - } - - @Test - public void testGroupByWithSubtotalsSpec() - { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } - - GroupByQuery query = GroupByQuery - .builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"), new DefaultDimensionSpec("market", "market"))) - .setAggregatorSpecs( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index"), - new FloatSumAggregatorFactory("idxFloat", "indexFloat"), - new DoubleSumAggregatorFactory("idxDouble", "index") - ) - ) - .setGranularity(QueryRunnerTestHelper.dayGran) - .setSubtotalsSpec(ImmutableList.of( - ImmutableList.of("alias"), - ImmutableList.of("market"), - ImmutableList.of() - )) - .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-01T00:00:00.000Z", "market", "spot", "idxDouble", 643.043177, "idxFloat", 643.043212890625, "rows", 5L, "idx", 640L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "total_market", "idxDouble", 1314.839715, "idxFloat", 1314.8397, "rows", 1L, "idx", 1314L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "upfront", "idxDouble", 1447.34116, "idxFloat", 1447.3412, "rows", 1L, "idx", 1447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "spot", "idxDouble", 266.090949, "idxFloat", 266.0909423828125, "rows", 2L, "idx", 265L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "total_market", "idxDouble", 1522.043733, "idxFloat", 1522.0437, "rows", 1L, "idx", 1522L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "upfront", "idxDouble", 1234.247546, "idxFloat", 1234.2476, "rows", 1L, "idx", 1234L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "spot", "idxDouble", 198.545289, "idxFloat", 198.5452880859375, "rows", 2L, "idx", 197L), - - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "spot", "idxDouble", 650.806953, "idxFloat", 650.8069458007812, "rows", 5L, "idx", 648L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "total_market", "idxDouble", 1193.556278, "idxFloat", 1193.5563, "rows", 1L, "idx", 1193L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "upfront", "idxDouble", 1144.342401, "idxFloat", 1144.3424, "rows", 1L, "idx", 1144L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "spot", "idxDouble", 249.591647, "idxFloat", 249.59164428710938, "rows", 2L, "idx", 249L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "total_market", "idxDouble", 1321.375057, "idxFloat", 1321.375, "rows", 1L, "idx", 1321L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "upfront", "idxDouble", 1049.738585, "idxFloat", 1049.7385, "rows", 1L, "idx", 1049L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "spot", "idxDouble", 223.798797, "idxFloat", 223.79879760742188, "rows", 2L, "idx", 223L), - - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "idxDouble", 6626.151575318359, "idxFloat", 6626.152f, "rows", 13L, "idx", 6619L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "idxDouble", 5833.209713, "idxFloat", 5833.209f, "rows", 13L, "idx", 5827L) - ); - - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - TestHelper.assertExpectedObjects(expectedResults, results, "subtotal"); - } - - @Test - public void testGroupByWithSubtotalsSpecWithLongDimensionColumn() - { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } - - GroupByQuery query = GroupByQuery - .builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("qualityLong", "ql", ValueType.LONG), new DefaultDimensionSpec("market", "market"))) - .setAggregatorSpecs( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index"), - new FloatSumAggregatorFactory("idxFloat", "indexFloat"), - new DoubleSumAggregatorFactory("idxDouble", "index") - ) + 6619L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02T00:00:00.000Z", + "idxDouble", + 5833.209717999999, + "idxFloat", + 5833.20849609375, + "rows", + 13L, + "idx", + 5827L ) - .setGranularity(QueryRunnerTestHelper.dayGran) - .setSubtotalsSpec(ImmutableList.of( - ImmutableList.of("ql"), - ImmutableList.of("market"), - ImmutableList.of() - )) - .build(); - - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "idxDouble", 135.885094, "idxFloat", 135.8851, "ql", 1000L, "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "idxDouble", 118.57034, "idxFloat", 118.57034, "ql", 1100L, "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "idxDouble", 158.747224, "idxFloat", 158.74722, "ql", 1200L, "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "idxDouble", 120.134704, "idxFloat", 120.134705, "ql", 1300L, "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "idxDouble", 2871.8866900000003, "idxFloat", 2871.88671875, "ql", 1400L, "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "idxDouble", 121.583581, "idxFloat", 121.58358, "ql", 1500L, "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "idxDouble", 2900.798647, "idxFloat", 2900.798583984375, "ql", 1600L, "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "idxDouble", 78.622547, "idxFloat", 78.62254, "ql", 1700L, "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "idxDouble", 119.922742, "idxFloat", 119.922745, "ql", 1800L, "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "idxDouble", 147.425935, "idxFloat", 147.42593, "ql", 1000L, "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "idxDouble", 112.987027, "idxFloat", 112.98703, "ql", 1100L, "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "idxDouble", 166.016049, "idxFloat", 166.01605, "ql", 1200L, "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "idxDouble", 113.446008, "idxFloat", 113.44601, "ql", 1300L, "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "idxDouble", 2448.830613, "idxFloat", 2448.83056640625, "ql", 1400L, "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "idxDouble", 114.290141, "idxFloat", 114.29014, "ql", 1500L, "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "idxDouble", 2506.415148, "idxFloat", 2506.4150390625, "ql", 1600L, "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "idxDouble", 97.387433, "idxFloat", 97.387436, "ql", 1700L, "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "idxDouble", 126.411364, "idxFloat", 126.41136, "ql", 1800L, "rows", 1L, "idx", 126L), - - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "spot", "idxDouble", 643.043177, "idxFloat", 643.043212890625, "rows", 5L, "idx", 640L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "total_market", "idxDouble", 1314.839715, "idxFloat", 1314.8397, "rows", 1L, "idx", 1314L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "upfront", "idxDouble", 1447.34116, "idxFloat", 1447.3412, "rows", 1L, "idx", 1447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "spot", "idxDouble", 266.090949, "idxFloat", 266.0909423828125, "rows", 2L, "idx", 265L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "total_market", "idxDouble", 1522.043733, "idxFloat", 1522.0437, "rows", 1L, "idx", 1522L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "upfront", "idxDouble", 1234.247546, "idxFloat", 1234.2476, "rows", 1L, "idx", 1234L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "spot", "idxDouble", 198.545289, "idxFloat", 198.5452880859375, "rows", 2L, "idx", 197L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "spot", "idxDouble", 650.806953, "idxFloat", 650.8069458007812, "rows", 5L, "idx", 648L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "total_market", "idxDouble", 1193.556278, "idxFloat", 1193.5563, "rows", 1L, "idx", 1193L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "upfront", "idxDouble", 1144.342401, "idxFloat", 1144.3424, "rows", 1L, "idx", 1144L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "spot", "idxDouble", 249.591647, "idxFloat", 249.59164428710938, "rows", 2L, "idx", 249L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "total_market", "idxDouble", 1321.375057, "idxFloat", 1321.375, "rows", 1L, "idx", 1321L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "upfront", "idxDouble", 1049.738585, "idxFloat", 1049.7385, "rows", 1L, "idx", 1049L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "spot", "idxDouble", 223.798797, "idxFloat", 223.79879760742188, "rows", 2L, "idx", 223L), - - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "idxDouble", 6626.151569, "idxFloat", 6626.1513671875, "rows", 13L, "idx", 6619L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "idxDouble", 5833.209717999999, "idxFloat", 5833.20849609375, "rows", 13L, "idx", 5827L) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -5845,7 +7274,10 @@ public void testGroupByWithSubtotalsSpecWithOrderLimit() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"), new DefaultDimensionSpec("market", "market"))) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("quality", "alias"), + new DefaultDimensionSpec("market", "market") + )) .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, @@ -5865,9 +7297,43 @@ public void testGroupByWithSubtotalsSpecWithOrderLimit() .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L, "idxFloat", 78.622547f, "idxDouble", 78.622547d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "spot", "idxDouble", 198.545289, "idxFloat", 198.5452880859375, "rows", 2L, "idx", 197L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "idxDouble", 6626.151575318359, "idxFloat", 6626.152f, "rows", 13L, "idx", 6619L) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "technology", + "rows", + 1L, + "idx", + 78L, + "idxFloat", + 78.622547f, + "idxDouble", + 78.622547d + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "market", + "spot", + "idxDouble", + 198.545289, + "idxFloat", + 198.5452880859375, + "rows", + 2L, + "idx", + 197L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01T00:00:00.000Z", + "idxDouble", + 6626.151575318359, + "idxFloat", + 6626.152f, + "rows", + 13L, + "idx", + 6619L + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -5881,9 +7347,11 @@ public void testGroupByWithTimeColumn() .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.jsCountIfTimeGreaterThan, - QueryRunnerTestHelper.__timeLongSum) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + QueryRunnerTestHelper.jsCountIfTimeGreaterThan, + QueryRunnerTestHelper.__timeLongSum + ) .setGranularity(QueryRunnerTestHelper.allGran) .build(); @@ -6483,8 +7951,12 @@ public void testBySegmentResultsUnOptimizedDimextraction() toolChest ) ); - - TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(QueryPlus.wrap(fullQuery), new HashMap<>()), "bySegment"); + + TestHelper.assertExpectedObjects( + bySegmentResults, + theRunner.run(QueryPlus.wrap(fullQuery), new HashMap<>()), + "bySegment" + ); exec.shutdownNow(); } @@ -6544,7 +8016,11 @@ public void testBySegmentResultsOptimizedDimextraction() ) ); - TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(QueryPlus.wrap(fullQuery), new HashMap<>()), "bySegment-dim-extraction"); + TestHelper.assertExpectedObjects( + bySegmentResults, + theRunner.run(QueryPlus.wrap(fullQuery), new HashMap<>()), + "bySegment-dim-extraction" + ); exec.shutdownNow(); } @@ -6575,8 +8051,10 @@ public void testGroupByWithExtractionDimFilter() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .setDimFilter(new OrDimFilter(dimFilters)) .build(); @@ -6627,8 +8105,10 @@ public void testGroupByWithExtractionDimFilterCaseMappingValueIsNullOrEmpty() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .setDimFilter(new ExtractionDimFilter("quality", "", lookupExtractionFn, null)) .build(); @@ -6665,8 +8145,10 @@ public void testGroupByWithExtractionDimFilterWhenSearchValueNotInTheMap() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .setDimFilter( new ExtractionDimFilter("quality", "NOT_THERE", lookupExtractionFn, null) @@ -6699,8 +8181,10 @@ public void testGroupByWithExtractionDimFilterKeyisNull() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("null_column", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .setDimFilter( new ExtractionDimFilter( @@ -6903,8 +8387,10 @@ public void testGroupByWithExtractionDimFilterOptimazitionManyToOne() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) .setGranularity(QueryRunnerTestHelper.dayGran) .setDimFilter( new ExtractionDimFilter( @@ -6989,7 +8475,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)); @@ -7041,7 +8531,11 @@ public void testBySegmentResultsWithAllFiltersWithExtractionFns() ) ); - TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(QueryPlus.wrap(fullQuery), new HashMap<>()), "bySegment-filter"); + TestHelper.assertExpectedObjects( + bySegmentResults, + theRunner.run(QueryPlus.wrap(fullQuery), new HashMap<>()), + "bySegment-filter" + ); exec.shutdownNow(); } @@ -7066,7 +8560,12 @@ public void testGroupByWithAllFiltersOnNullDimsWithExtractionFns() superFilterList.add( new SearchQueryDimFilter("null_column", 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() @@ -7113,12 +8612,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); @@ -7145,12 +8692,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); @@ -7956,8 +9551,10 @@ public void testGroupByNestedWithInnerQueryNumerics() ) ) ) - .setAggregatorSpecs(new LongSumAggregatorFactory("ql_alias_sum", "ql_alias"), - new DoubleSumAggregatorFactory("qf_alias_sum", "qf_alias")) + .setAggregatorSpecs( + new LongSumAggregatorFactory("ql_alias_sum", "ql_alias"), + new DoubleSumAggregatorFactory("qf_alias_sum", "qf_alias") + ) .setGranularity(QueryRunnerTestHelper.allGran) .build(); @@ -8003,8 +9600,10 @@ public void testGroupByNestedWithInnerQueryNumericsWithLongTime() new DefaultDimensionSpec("alias", "market"), new DefaultDimensionSpec("time_alias", "time_alias2", ValueType.LONG) ) - .setAggregatorSpecs(new LongMaxAggregatorFactory("time_alias_max", "time_alias"), - new DoubleMaxAggregatorFactory("index_alias_max", "index_alias")) + .setAggregatorSpecs( + new LongMaxAggregatorFactory("time_alias_max", "time_alias"), + new DoubleMaxAggregatorFactory("index_alias_max", "index_alias") + ) .setGranularity(QueryRunnerTestHelper.allGran) .build(); @@ -8116,27 +9715,29 @@ public void testGroupByWithAggsOnNumericDimensions() .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("quality", "alias")) .setDimFilter(new SelectorDimFilter("quality", "technology", null)) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("qlLong", "qualityLong"), - new DoubleSumAggregatorFactory("qlFloat", "qualityLong"), - new JavaScriptAggregatorFactory( - "qlJs", - ImmutableList.of("qualityLong"), - "function(a,b) { return a + b; }", - "function() { return 0; }", - "function(a,b) { return a + b }", - JavaScriptConfig.getEnabledInstance() - ), - new DoubleSumAggregatorFactory("qfFloat", "qualityFloat"), - new LongSumAggregatorFactory("qfLong", "qualityFloat"), - new JavaScriptAggregatorFactory( - "qfJs", - ImmutableList.of("qualityFloat"), - "function(a,b) { return a + b; }", - "function() { return 0; }", - "function(a,b) { return a + b }", - JavaScriptConfig.getEnabledInstance() - )) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("qlLong", "qualityLong"), + new DoubleSumAggregatorFactory("qlFloat", "qualityLong"), + new JavaScriptAggregatorFactory( + "qlJs", + ImmutableList.of("qualityLong"), + "function(a,b) { return a + b; }", + "function() { return 0; }", + "function(a,b) { return a + b }", + JavaScriptConfig.getEnabledInstance() + ), + new DoubleSumAggregatorFactory("qfFloat", "qualityFloat"), + new LongSumAggregatorFactory("qfLong", "qualityFloat"), + new JavaScriptAggregatorFactory( + "qfJs", + ImmutableList.of("qualityFloat"), + "function(a,b) { return a + b; }", + "function() { return 0; }", + "function(a,b) { return a + b }", + JavaScriptConfig.getEnabledInstance() + ) + ) .setGranularity(QueryRunnerTestHelper.dayGran) .build(); @@ -8510,11 +10111,61 @@ public Sequence run(QueryPlus queryPlus, Map responseC Map context = new HashMap<>(); 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(); @@ -8593,11 +10244,61 @@ public Sequence run(QueryPlus queryPlus, Map responseC Map context = new HashMap<>(); 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(); @@ -8652,9 +10353,9 @@ public void testEmptySubqueryWithLimitPushDown() .setLimitSpec( new DefaultLimitSpec( Collections.singletonList(new OrderByColumnSpec( - "alias", - OrderByColumnSpec.Direction.DESCENDING - )), + "alias", + OrderByColumnSpec.Direction.DESCENDING + )), 5 ) ) @@ -8794,19 +10495,39 @@ 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.directExecutor(), (query1, future) -> { return; }, ImmutableList.>of(runner, runner) ); - QueryRunner mergingRunner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(ceqr)); + QueryRunner mergingRunner = factory.mergeRunners(Execs.directExecutor(), ImmutableList.of(ceqr)); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, mergingRunner, query); TestHelper.assertExpectedObjects(expectedResults, results, "type-conversion"); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index 06a3e5eec037..0dc2a6367118 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -22,12 +22,12 @@ 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 org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -55,6 +55,7 @@ import java.util.Map; /** + * */ @RunWith(Parameterized.class) public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest @@ -93,7 +94,7 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) { TimeseriesQuery tsQuery = (TimeseriesQuery) queryPlus.getQuery(); QueryRunner newRunner = factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), ImmutableList.of(input) + Execs.directExecutor(), ImmutableList.of(input) ); QueryToolChest toolChest = factory.getToolchest(); diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java index 13e1d3ec4d94..1a225444ec16 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java @@ -24,10 +24,10 @@ 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 org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.BySegmentResultValue; import org.apache.druid.query.BySegmentResultValueClass; @@ -304,7 +304,7 @@ public void testSegmentMetadataQueryWithRollupMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), Lists.newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -372,7 +372,7 @@ public void testSegmentMetadataQueryWithHasMultipleValuesMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), Lists.newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -440,7 +440,7 @@ public void testSegmentMetadataQueryWithComplexColumnMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), Lists.newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -557,7 +557,7 @@ private void testSegmentMetadataQueryWithDefaultAnalysisMerge( QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), Lists.newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -609,7 +609,7 @@ public void testSegmentMetadataQueryWithNoAnalysisTypesMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), Lists.newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -671,7 +671,7 @@ public void testSegmentMetadataQueryWithAggregatorsMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), Lists.newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -729,7 +729,7 @@ public void testSegmentMetadataQueryWithTimestampSpecMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), Lists.newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -787,7 +787,7 @@ public void testSegmentMetadataQueryWithQueryGranularityMerge() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), Lists.newArrayList( toolChest.preMergeQueryDecoration(runner1), toolChest.preMergeQueryDecoration(runner2) @@ -832,7 +832,7 @@ public void testBySegmentResults() QueryRunner myRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( FACTORY.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), //Note: It is essential to have atleast 2 query runners merged to reproduce the regression bug described in //https://github.com/apache/incubator-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/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java index cc6ad9d62a43..9d48b6ae4c3d 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java @@ -22,10 +22,10 @@ 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 org.apache.commons.io.IOUtils; import org.apache.commons.lang.StringUtils; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.MergeSequence; import org.apache.druid.java.util.common.guava.Sequence; @@ -60,6 +60,7 @@ import java.util.Map; /** + * */ @RunWith(Parameterized.class) public class MultiSegmentScanQueryTest @@ -167,7 +168,10 @@ public static void clear() @Parameterized.Parameters(name = "limit={0},batchSize={1}") public static Iterable constructorFeeder() { - 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; @@ -196,8 +200,9 @@ public void testMergeRunnersWithLimit() ScanQuery query = newBuilder().build(); List results = factory .mergeRunners( - MoreExecutors.sameThreadExecutor(), - ImmutableList.of(factory.createRunner(segment0), factory.createRunner(segment1))) + Execs.directExecutor(), + ImmutableList.of(factory.createRunner(segment0), factory.createRunner(segment1)) + ) .run(QueryPlus.wrap(query), new HashMap<>()) .toList(); int totalCount = 0; @@ -215,7 +220,8 @@ public void testMergeRunnersWithLimit() public void testMergeResultsWithLimit() { QueryRunner runner = toolChest.mergeResults( - new QueryRunner() { + new QueryRunner() + { @Override public Sequence run( QueryPlus queryPlus, diff --git a/processing/src/test/java/org/apache/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/spec/SpecificSegmentQueryRunnerTest.java index 930e32175031..af64fe2706f5 100644 --- a/processing/src/test/java/org/apache/druid/query/spec/SpecificSegmentQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/spec/SpecificSegmentQueryRunnerTest.java @@ -21,10 +21,10 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Accumulator; import org.apache.druid.java.util.common.guava.Sequence; @@ -116,6 +116,7 @@ public Yielder toYielder(Object initValue, YieldingAccumulator accumulat new YieldingAccumulator() { final List lists = new ArrayList<>(); + @Override public Object accumulate(Object accumulated, Object in) { @@ -162,7 +163,7 @@ public void run() throw new SegmentMissingException("FAILSAUCE"); } }, - MoreExecutors.sameThreadExecutor() + Execs.directExecutor() ); } }, diff --git a/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java b/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java index fa1a9ea5a0ad..0e78e994fd04 100644 --- a/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java +++ b/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java @@ -27,6 +27,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.logger.Logger; @@ -101,7 +102,7 @@ public void onFailure(Throwable t) exec ); }, - MoreExecutors.sameThreadExecutor() + Execs.directExecutor() ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/org/apache/druid/segment/realtime/RealtimeManager.java index bc07ba820d63..4b6437681179 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/RealtimeManager.java @@ -26,7 +26,6 @@ import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.Firehose; @@ -65,6 +64,7 @@ import java.util.concurrent.TimeUnit; /** + * */ public class RealtimeManager implements QuerySegmentWalker { @@ -175,7 +175,7 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, Itera return partitionChiefs == null ? new NoopQueryRunner() : factory.getToolchest().mergeResults( factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), // Chaining query runners which wait on submitted chain query runners can make executor pools deadlock Iterables.transform( partitionChiefs.values(), new Function>() @@ -202,7 +202,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final ? new NoopQueryRunner() : factory.getToolchest().mergeResults( factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), Iterables.transform( specs, new Function>() diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 53f68f59ef2b..6e88cf1eca4f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -23,7 +23,6 @@ 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 org.apache.druid.client.CachingQueryRunner; import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; @@ -31,6 +30,7 @@ import org.apache.druid.client.cache.ForegroundCachePopulator; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.CloseQuietly; import org.apache.druid.java.util.common.guava.FunctionalIterable; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -206,7 +206,7 @@ public QueryRunner apply(final SegmentDescriptor descriptor) sinkSegmentIdentifier, descriptor.getInterval().getStart(), factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), Iterables.transform( theSink, new Function>() diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index 95f50b4a8997..5bb25f8f7957 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -26,12 +26,12 @@ 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 org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ListenableFutures; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; @@ -209,7 +209,7 @@ public void moveSegmentOut(final String sequenceName, final List * Should be called after all data has been added through {@link #add(InputRow, String, Supplier, boolean, boolean)}. * * @param committer committer representing all data that has been added so far @@ -235,7 +235,7 @@ public Object persist(final Committer committer) throws InterruptedException /** * Persist all data indexed through this driver so far. Returns a future of persisted commitMetadata. - * + *

* Should be called after all data has been added through {@link #add(InputRow, String, Supplier, boolean, boolean)}. * * @param committer committer representing all data that has been added so far @@ -331,7 +331,7 @@ public ListenableFuture registerHandoff(SegmentsAndMetadata segmentIdentifier.getVersion(), segmentIdentifier.getShardSpec().getPartitionNum() ), - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), () -> { log.info("Segment[%s] successfully handed off, dropping.", segmentIdentifier); metrics.incrementHandOffCount(); diff --git a/server/src/main/java/org/apache/druid/server/QueryManager.java b/server/src/main/java/org/apache/druid/server/QueryManager.java index 0684ed45fac8..a90bf4b077d5 100644 --- a/server/src/main/java/org/apache/druid/server/QueryManager.java +++ b/server/src/main/java/org/apache/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 org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.query.Query; import org.apache.druid.query.QueryWatcher; @@ -76,7 +76,7 @@ public void run() } } }, - MoreExecutors.sameThreadExecutor() + Execs.directExecutor() ); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index f65dc3070d8c..361213692c0d 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -62,6 +62,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; @@ -165,6 +166,7 @@ import java.util.concurrent.Executor; /** + * */ @RunWith(Parameterized.class) public class CachingClusteredClientTest @@ -331,7 +333,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.directExecutor() ); @Override @@ -1174,14 +1176,14 @@ public void testSearchCaching() public void testSearchCachingRenamedOutput() { final Druids.SearchQueryBuilder builder = Druids.newSearchQueryBuilder() - .dataSource(DATA_SOURCE) - .filters(DIM_FILTER) - .granularity(GRANULARITY) - .limit(1000) - .intervals(SEG_SPEC) - .dimensions(Collections.singletonList(TOP_DIM)) - .query("how") - .context(CONTEXT); + .dataSource(DATA_SOURCE) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .limit(1000) + .intervals(SEG_SPEC) + .dimensions(Collections.singletonList(TOP_DIM)) + .query("how") + .context(CONTEXT); testQueryCaching( getDefaultQueryRunner(), @@ -1347,14 +1349,14 @@ public void testSelectCachingRenamedOutputName() final Set metrics = Sets.newHashSet("rows"); Druids.SelectQueryBuilder builder = Druids.newSelectQueryBuilder() - .dataSource(DATA_SOURCE) - .intervals(SEG_SPEC) - .filters(DIM_FILTER) - .granularity(GRANULARITY) - .dimensions(Collections.singletonList("a")) - .metrics(Collections.singletonList("rows")) - .pagingSpec(new PagingSpec(null, 3)) - .context(CONTEXT); + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .dimensions(Collections.singletonList("a")) + .metrics(Collections.singletonList("rows")) + .pagingSpec(new PagingSpec(null, 3)) + .context(CONTEXT); testQueryCaching( getDefaultQueryRunner(), @@ -1446,9 +1448,9 @@ public void testSelectCachingRenamedOutputName() public void testGroupByCaching() { 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(); @@ -1650,9 +1652,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( @@ -1690,14 +1694,14 @@ public void testSingleDimensionPruning() ); 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<>(); @@ -1888,7 +1892,12 @@ public void testQueryCachingWithFilter( @Override public Sequence answer() { - return toFilteredQueryableTimeseriesResults((TimeseriesQuery) capture.getValue().getQuery(), segmentIds, queryIntervals, results); + return toFilteredQueryableTimeseriesResults( + (TimeseriesQuery) capture.getValue().getQuery(), + segmentIds, + queryIntervals, + results + ); } }) .times(0, 1); @@ -1946,7 +1955,11 @@ private Sequence> toFilteredQueryableTimeseriesRes MultipleSpecificSegmentSpec spec = (MultipleSpecificSegmentSpec) query.getQuerySegmentSpec(); List> ret = new ArrayList<>(); 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( @@ -2466,13 +2479,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() ) ) ); @@ -2722,7 +2735,8 @@ public int getCacheBulkMergeLimit() return mergeLimit; } }, - new DruidHttpClientConfig() { + new DruidHttpClientConfig() + { @Override public long getMaxQueuedBytes() { @@ -3126,7 +3140,8 @@ public void testIfNoneMatch() @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/org/apache/druid/client/HttpServerInventoryViewTest.java b/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java index be49011860fc..bb43104b04bd 100644 --- a/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java +++ b/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java @@ -24,7 +24,6 @@ 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 org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscovery; @@ -32,6 +31,7 @@ import org.apache.druid.discovery.NodeType; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.HttpResponseHandler; @@ -64,6 +64,7 @@ import java.util.concurrent.atomic.AtomicInteger; /** + * */ public class HttpServerInventoryViewTest { @@ -197,7 +198,7 @@ DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerT ); httpServerInventoryView.registerSegmentCallback( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), new ServerView.SegmentCallback() { @Override @@ -225,7 +226,7 @@ public ServerView.CallbackAction segmentViewInitialized() final CountDownLatch serverRemovedCalled = new CountDownLatch(1); httpServerInventoryView.registerServerRemovedCallback( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), new ServerView.ServerRemovedCallback() { @Override @@ -254,8 +255,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 +323,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, null); return Futures.immediateFailedFuture(new RuntimeException("server error")); diff --git a/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java index c002469624c3..987db9c9a138 100644 --- a/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java +++ b/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java @@ -43,6 +43,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer; @@ -77,6 +78,7 @@ import java.util.concurrent.atomic.AtomicInteger; /** + * */ public class BatchServerInventoryViewTest { @@ -118,7 +120,7 @@ public void setUp() throws Exception announcer = new Announcer( cf, - MoreExecutors.sameThreadExecutor() + Execs.directExecutor() ); announcer.start(); @@ -204,7 +206,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) { @@ -337,7 +340,7 @@ public ServerView.CallbackAction answer() EasyMock.replay(callback); filteredBatchServerInventoryView.registerSegmentCallback( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), callback, new Predicate>() { @@ -407,7 +410,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/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java b/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java index fcf6b6b36278..c2121a74888d 100644 --- a/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java +++ b/server/src/test/java/org/apache/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 org.apache.druid.curator.CuratorTestBase; import org.apache.druid.curator.announcement.Announcer; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.NodeType; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.server.DruidNode; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.initialization.ZkPathsConfig; @@ -42,6 +42,7 @@ import java.util.Set; /** + * */ public class CuratorDruidNodeAnnouncerAndDiscoveryTest extends CuratorTestBase { @@ -68,7 +69,7 @@ public void testAnnouncementAndDiscovery() throws Exception Announcer announcer = new Announcer( curator, - MoreExecutors.sameThreadExecutor() + Execs.directExecutor() ); announcer.start(); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/org/apache/druid/segment/realtime/RealtimeManagerTest.java index 0ab42c88e81f..4e693223ba12 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/RealtimeManagerTest.java @@ -27,7 +27,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; @@ -41,6 +40,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.parsers.ParseException; @@ -100,6 +100,7 @@ import java.util.concurrent.TimeUnit; /** + * */ public class RealtimeManagerTest { @@ -1018,7 +1019,8 @@ public Object startJob() } @Override - public IncrementalIndexAddResult add(InputRow row, Supplier committerSupplier) throws IndexSizeExceededException + public IncrementalIndexAddResult add(InputRow row, Supplier committerSupplier) + throws IndexSizeExceededException { if (row == null) { return Plumber.THROWAWAY; @@ -1055,7 +1057,7 @@ public QueryRunner getQueryRunner(final Query query) return factory.getToolchest() .mergeResults( factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), Iterables.transform( baseQuery.getIntervals(), new Function>() diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java index 228d79578436..3fcac21996d9 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java @@ -19,9 +19,9 @@ package org.apache.druid.segment.realtime.plumber; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.query.SegmentDescriptor; import org.easymock.EasyMock; import org.joda.time.Duration; @@ -62,7 +62,7 @@ public void testHandoffCallbackNotCalled() final AtomicBoolean callbackCalled = new AtomicBoolean(false); notifier.registerSegmentHandoffCallback( descriptor, - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), () -> callbackCalled.set(true) ); notifier.checkForSegmentHandoffs(); @@ -93,7 +93,7 @@ public void testHandoffCallbackCalled() notifier.registerSegmentHandoffCallback( descriptor, - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), () -> callbackCalled.set(true) ); Assert.assertEquals(1, notifier.getHandOffCallbacks().size()); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index a3a9b5ba8c80..4052cc678f4a 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.io.Files; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.commons.io.FileUtils; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.MapCache; @@ -38,6 +37,7 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; @@ -79,6 +79,7 @@ import java.util.concurrent.TimeUnit; /** + * */ @RunWith(Parameterized.class) public class RealtimePlumberSchoolTest @@ -115,7 +116,10 @@ public static Collection constructorFeeder() private FireDepartmentMetrics metrics; private File tmpDir; - public RealtimePlumberSchoolTest(RejectionPolicyFactory rejectionPolicy, SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + public RealtimePlumberSchoolTest( + RejectionPolicyFactory rejectionPolicy, + SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + ) { this.rejectionPolicy = rejectionPolicy; this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; @@ -219,7 +223,7 @@ public void setUp() throws Exception announcer, segmentPublisher, handoffNotifierFactory, - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory), TestHelper.getTestIndexIO(), MapCache.create(0), diff --git a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java index e52c2783face..52006f5880d6 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java @@ -27,7 +27,6 @@ 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 org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; @@ -35,6 +34,7 @@ import org.apache.druid.curator.PotentiallyGzippedCompressionProvider; import org.apache.druid.curator.announcement.Announcer; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer; import org.apache.druid.server.coordination.ChangeRequestHistory; @@ -58,6 +58,7 @@ import java.util.concurrent.atomic.AtomicInteger; /** + * */ public class BatchDataSegmentAnnouncerTest { @@ -97,7 +98,7 @@ public void setUp() throws Exception announcer = new Announcer( cf, - MoreExecutors.sameThreadExecutor() + Execs.directExecutor() ); announcer.start(); diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index 66ebd542d2da..e1e2307aa288 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -27,7 +27,6 @@ import com.google.common.collect.Iterables; 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 org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -485,7 +485,7 @@ private static Sequence executeQuery(final Injector injector, final Query final QueryRunner runner = factory.createRunner(new QueryableIndexSegment("segment", index)); return factory .getToolchest() - .mergeResults(factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(runner))) + .mergeResults(factory.mergeRunners(Execs.directExecutor(), ImmutableList.of(runner))) .run(QueryPlus.wrap(query), new HashMap<>()); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java index 87d97ef29e9c..4e1bb0945686 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java @@ -25,7 +25,6 @@ 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 org.apache.calcite.schema.Table; import org.apache.calcite.schema.impl.AbstractSchema; @@ -34,6 +33,7 @@ import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; @@ -141,7 +141,7 @@ public DruidSchema( this.escalator = escalator; serverView.registerTimelineCallback( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), new TimelineServerView.TimelineCallback() { @Override diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index 55fff600da81..0840803c1241 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -23,9 +23,9 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; import com.google.common.io.Closeables; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.FunctionalIterable; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.FinalizeResultsQueryRunner; @@ -224,7 +224,7 @@ private QueryRunner makeBaseRunner( return new FinalizeResultsQueryRunner<>( toolChest.mergeResults( factory.mergeRunners( - MoreExecutors.sameThreadExecutor(), + Execs.directExecutor(), FunctionalIterable .create(specs) .transformCat(