You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2019/01/05 01:41:49 UTC

[GitHub] drcrallen closed pull request #5413: Guava-free Same Thread Executor

drcrallen closed pull request #5413: Guava-free Same Thread Executor
URL: https://github.com/apache/incubator-druid/pull/5413
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/codestyle/druid-forbidden-apis.txt b/codestyle/druid-forbidden-apis.txt
index 96db0882600..253f0363bbe 100644
--- a/codestyle/druid-forbidden-apis.txt
+++ b/codestyle/druid-forbidden-apis.txt
@@ -1,4 +1,5 @@
 com.google.common.collect.MapMaker @ Create java.util.concurrent.ConcurrentHashMap directly
 com.google.common.collect.Maps#newConcurrentMap() @ Create java.util.concurrent.ConcurrentHashMap directly
+com.google.common.util.concurrent.MoreExecutors#sameThreadExecutor() @ Use io.druid.java.util.common.concurrent.Execs#sameThreadExecutor
 com.google.common.util.concurrent.Futures#transform(com.google.common.util.concurrent.ListenableFuture, com.google.common.util.concurrent.AsyncFunction) @ Use io.druid.java.util.common.concurrent.ListenableFutures#transformAsync
-com.google.common.collect.Iterators#emptyIterator() @ Use java.util.Collections#emptyIterator()
\ No newline at end of file
+com.google.common.collect.Iterators#emptyIterator() @ Use java.util.Collections#emptyIterator()
diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java
index 08df93a5c87..147b0713599 100644
--- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java
+++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java
@@ -260,7 +260,7 @@ public void onFailure(Throwable t)
               }
             }
           },
-          MoreExecutors.sameThreadExecutor()
+          Execs.sameThreadExecutor()
       );
       this.future = future;
       final Stopwatch stopwatch = Stopwatch.createStarted();
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java
index 13388e8de08..52e7645b317 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java
@@ -547,7 +547,8 @@ public void statusChanged(String taskId, TaskStatus status)
             {
               notices.add(new RunNotice());
             }
-          }, MoreExecutors.sameThreadExecutor()
+          },
+          Execs.sameThreadExecutor()
       );
 
       listenerRegistered = true;
@@ -1770,13 +1771,21 @@ private void createKafkaTasksForGroup(int groupId, int replicas) throws JsonProc
     final String checkpoints = sortingMapper.writerWithType(new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
     {
     }).writeValueAsString(taskGroups.get(groupId).sequenceOffsets);
-    final Map<String, Object> context = spec.getContext() == null
-                                        ? ImmutableMap.of("checkpoints", checkpoints, IS_INCREMENTAL_HANDOFF_SUPPORTED, true)
-                                        : ImmutableMap.<String, Object>builder()
-                                            .put("checkpoints", checkpoints)
-                                            .put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true)
-                                            .putAll(spec.getContext())
-                                            .build();
+    final Map<String, Object> context;
+    if (spec.getContext() == null) {
+      context = ImmutableMap.of(
+          "checkpoints",
+          checkpoints,
+          IS_INCREMENTAL_HANDOFF_SUPPORTED,
+          true
+      );
+    } else {
+      context = ImmutableMap.<String, Object>builder()
+          .put("checkpoints", checkpoints)
+          .put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true)
+          .putAll(spec.getContext())
+          .build();
+    }
     for (int i = 0; i < replicas; i++) {
       String taskId = Joiner.on("_").join(sequenceName, getRandomId());
       KafkaIndexTask indexTask = new KafkaIndexTask(
diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java
index 13452ce4227..1d16f6e2e37 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java
@@ -2019,7 +2019,7 @@ public void close()
         EasyMock.createNiceMock(DataSegmentServerAnnouncer.class),
         handoffNotifierFactory,
         this::makeTimeseriesOnlyConglomerate,
-        MoreExecutors.sameThreadExecutor(), // queryExecutorService
+        Execs.sameThreadExecutor(), // queryExecutorService
         EasyMock.createMock(MonitorScheduler.class),
         new SegmentLoaderFactory(
             new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper())
diff --git a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceGroupByQueryTest.java b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceGroupByQueryTest.java
index deff3189a6d..2694b8ccdbd 100644
--- a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceGroupByQueryTest.java
+++ b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceGroupByQueryTest.java
@@ -21,8 +21,8 @@
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.data.input.Row;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.PeriodGranularity;
 import io.druid.query.QueryRunner;
 import io.druid.query.QueryRunnerTestHelper;
@@ -67,12 +67,17 @@
     return GroupByQueryRunnerTest.constructorFeeder();
   }
 
-  public VarianceGroupByQueryTest(String testName, GroupByQueryConfig config, GroupByQueryRunnerFactory factory, QueryRunner runner)
+  public VarianceGroupByQueryTest(
+      String testName,
+      GroupByQueryConfig config,
+      GroupByQueryRunnerFactory factory,
+      QueryRunner runner
+  )
   {
     this.testName = testName;
     this.config = config;
     this.factory = factory;
-    this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.<QueryRunner<Row>>of(runner));
+    this.runner = factory.mergeRunners(Execs.sameThreadExecutor(), ImmutableList.<QueryRunner<Row>>of(runner));
   }
 
   @Test
diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java
index cd2389f52ea..50384a116b5 100644
--- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java
+++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java
@@ -303,7 +303,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(
@@ -475,7 +477,8 @@ public int getPartition(BytesWritable bytesWritable, Writable value, int numPart
       final ByteBuffer bytes = ByteBuffer.wrap(bytesWritable.getBytes());
       bytes.position(4); // Skip length added by SortableBytes
       int shardNum = bytes.getInt();
-      if ("local".equals(config.get("mapreduce.jobtracker.address")) || "local".equals(config.get("mapred.job.tracker"))) {
+      if ("local".equals(config.get("mapreduce.jobtracker.address"))
+          || "local".equals(config.get("mapred.job.tracker"))) {
         return shardNum % numPartitions;
       } else {
         if (shardNum >= numPartitions) {
@@ -622,7 +625,7 @@ public void rejectedExecution(Runnable r, ThreadPoolExecutor executor)
           );
           persistExecutor = MoreExecutors.listeningDecorator(executorService);
         } else {
-          persistExecutor = MoreExecutors.sameThreadExecutor();
+          persistExecutor = Execs.sameThreadExecutor();
         }
 
         for (final BytesWritable bw : values) {
@@ -732,7 +735,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/io/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java
index b1b9682a57e..ae12da2a5b5 100644
--- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java
+++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java
@@ -23,13 +23,13 @@
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.MoreExecutors;
 import com.google.inject.Inject;
 import io.druid.curator.CuratorUtils;
 import io.druid.curator.announcement.Announcer;
 import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
 import io.druid.java.util.common.DateTimes;
 import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.lifecycle.LifecycleStart;
 import io.druid.java.util.common.lifecycle.LifecycleStop;
 import io.druid.java.util.common.logger.Logger;
@@ -77,7 +77,7 @@ public WorkerCuratorCoordinator(
     this.curatorFramework = curatorFramework;
     this.worker = worker;
 
-    this.announcer = new Announcer(curatorFramework, MoreExecutors.sameThreadExecutor());
+    this.announcer = new Announcer(curatorFramework, Execs.sameThreadExecutor());
 
     this.baseAnnouncementsPath = getPath(Arrays.asList(indexerZkConfig.getAnnouncementsPath(), worker.getHost()));
     this.baseTaskPath = getPath(Arrays.asList(indexerZkConfig.getTasksPath(), worker.getHost()));
diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskManager.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskManager.java
index e8d7b9fb987..cba2e4d8751 100644
--- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskManager.java
+++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskManager.java
@@ -28,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 io.druid.client.indexing.IndexingService;
@@ -69,7 +68,7 @@
 
 /**
  * This class manages the list of tasks assigned to this worker.
- *
+ * <p>
  * It persists the list of assigned and completed tasks on disk. assigned task from disk is deleted as soon as it
  * starts running and completed task on disk is deleted based on a periodic schedule where overlord is asked for
  * active tasks to see which completed tasks are safe to delete.
@@ -226,7 +225,7 @@ public void statusChanged(final String taskId, final TaskStatus status)
             // do nothing
           }
         },
-        MoreExecutors.sameThreadExecutor()
+        Execs.sameThreadExecutor()
     );
   }
 
@@ -456,9 +455,12 @@ private void scheduleCompletedTasksCleanup()
               );
               if (fullResponseHolder.getStatus().getCode() == 200) {
                 String responseContent = fullResponseHolder.getContent();
-                taskStatusesFromOverlord = jsonMapper.readValue(responseContent, new TypeReference<Map<String, TaskStatus>>()
-                {
-                });
+                taskStatusesFromOverlord = jsonMapper.readValue(
+                    responseContent,
+                    new TypeReference<Map<String, TaskStatus>>()
+                    {
+                    }
+                );
                 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/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
index 2761aacbdb6..d1d0b148325 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
@@ -120,6 +120,7 @@
 import io.druid.timeline.DataSegment;
 import io.druid.timeline.partition.LinearShardSpec;
 import io.druid.timeline.partition.NumberedShardSpec;
+import io.druid.utils.Runnables;
 import org.apache.commons.io.FileUtils;
 import org.easymock.EasyMock;
 import org.hamcrest.CoreMatchers;
@@ -215,7 +216,7 @@ public InputRow nextRow()
     @Override
     public Runnable commit()
     {
-      return () -> {};
+      return Runnables.getNoopRunnable();
     }
 
     @Override
@@ -1217,7 +1218,6 @@ public void close()
         return Lists.newArrayList();
       }
     };
-
     taskToolboxFactory = new TaskToolboxFactory(
         taskConfig,
         taskActionClientFactory,
@@ -1230,7 +1230,7 @@ public void close()
         EasyMock.createNiceMock(DataSegmentServerAnnouncer.class),
         handoffNotifierFactory,
         () -> conglomerate,
-        MoreExecutors.sameThreadExecutor(), // queryExecutorService
+        Execs.sameThreadExecutor(), // queryExecutorService
         EasyMock.createMock(MonitorScheduler.class),
         new SegmentLoaderFactory(
             new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper())
@@ -1250,16 +1250,17 @@ public void close()
   public long sumMetric(final Task task, final DimFilter filter, final String metric)
   {
     // Do a query.
-    TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
-                                  .dataSource("test_ds")
-                                  .filters(filter)
-                                  .aggregators(
-                                      ImmutableList.<AggregatorFactory>of(
-                                          new LongSumAggregatorFactory(metric, metric)
-                                      )
-                                  ).granularity(Granularities.ALL)
-                                  .intervals("2000/3000")
-                                  .build();
+    TimeseriesQuery query = Druids
+        .newTimeseriesQueryBuilder()
+        .dataSource("test_ds")
+        .filters(filter)
+        .aggregators(
+            ImmutableList.<AggregatorFactory>of(
+                new LongSumAggregatorFactory(metric, metric)
+            )
+        ).granularity(Granularities.ALL)
+        .intervals("2000/3000")
+        .build();
 
     List<Result<TimeseriesResultValue>> results =
         task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java
index 518d1d63b5d..48fbbb66e91 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java
@@ -119,6 +119,7 @@
 import io.druid.server.coordination.DataSegmentServerAnnouncer;
 import io.druid.server.coordination.ServerType;
 import io.druid.timeline.DataSegment;
+import io.druid.utils.Runnables;
 import org.easymock.EasyMock;
 import org.hamcrest.CoreMatchers;
 import org.joda.time.DateTime;
@@ -206,7 +207,7 @@ public InputRow nextRow()
     @Override
     public Runnable commit()
     {
-      return () -> {};
+      return Runnables.getNoopRunnable();
     }
 
     @Override
@@ -1071,7 +1072,7 @@ public void close()
         EasyMock.createNiceMock(DataSegmentServerAnnouncer.class),
         handoffNotifierFactory,
         () -> conglomerate,
-        MoreExecutors.sameThreadExecutor(), // queryExecutorService
+        Execs.sameThreadExecutor(), // queryExecutorService
         EasyMock.createMock(MonitorScheduler.class),
         new SegmentLoaderFactory(
             new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper())
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java
index 5fb348fc363..f1b44732771 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java
@@ -33,7 +33,6 @@
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Ordering;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.client.cache.MapCache;
 import io.druid.data.input.Firehose;
 import io.druid.data.input.FirehoseFactory;
@@ -76,6 +75,7 @@
 import io.druid.java.util.common.Pair;
 import io.druid.java.util.common.RE;
 import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.common.guava.Comparators;
 import io.druid.java.util.emitter.EmittingLogger;
@@ -522,8 +522,15 @@ private TaskToolboxFactory setUpTaskToolboxFactory(
     Preconditions.checkNotNull(emitter);
 
     taskLockbox = new TaskLockbox(taskStorage);
-    tac = new LocalTaskActionClientFactory(taskStorage, new TaskActionToolbox(taskLockbox, mdc, emitter, EasyMock.createMock(
-        SupervisorManager.class)));
+    tac = new LocalTaskActionClientFactory(
+        taskStorage,
+        new TaskActionToolbox(
+            taskLockbox,
+            mdc,
+            emitter,
+            EasyMock.createMock(SupervisorManager.class)
+        )
+    );
     File tmpDir = temporaryFolder.newFolder();
     taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, false, null, null);
 
@@ -592,7 +599,7 @@ public void unannounceSegments(Iterable<DataSegment> segments)
         EasyMock.createNiceMock(DataSegmentServerAnnouncer.class),
         handoffNotifierFactory,
         () -> queryRunnerFactoryConglomerate, // query runner factory conglomerate corporation unionized collective
-        MoreExecutors.sameThreadExecutor(), // query executor service
+        Execs.sameThreadExecutor(), // query executor service
         monitorScheduler, // monitor scheduler
         new SegmentLoaderFactory(
             new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, new DefaultObjectMapper())
@@ -1088,7 +1095,22 @@ public void testResumeTasks() throws Exception
                 mapper
             ),
             new IndexIOConfig(new MockFirehoseFactory(false), false),
-            new IndexTuningConfig(10000, 10, null, null, null, indexSpec, null, false, null, null, null, null, null, null)
+            new IndexTuningConfig(
+                10000,
+                10,
+                null,
+                null,
+                null,
+                indexSpec,
+                null,
+                false,
+                null,
+                null,
+                null,
+                null,
+                null,
+                null
+            )
         ),
         null
     );
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java
index 564ae797312..4f31b2af549 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java
@@ -27,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 io.druid.common.guava.DSuppliers;
 import io.druid.discovery.DiscoveryDruidNode;
 import io.druid.discovery.DruidNodeDiscovery;
@@ -47,6 +46,7 @@
 import io.druid.indexing.worker.TaskAnnouncement;
 import io.druid.indexing.worker.Worker;
 import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.http.client.HttpClient;
 import io.druid.segment.TestHelper;
 import io.druid.server.DruidNode;
@@ -90,7 +90,8 @@ public void testFreshStart() throws Exception
 
     HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner(
         TestHelper.makeJsonMapper(),
-        new HttpRemoteTaskRunnerConfig() {
+        new HttpRemoteTaskRunnerConfig()
+        {
           @Override
           public int getPendingTasksRunnerNumThreads()
           {
@@ -104,7 +105,8 @@ public int getPendingTasksRunnerNumThreads()
         EasyMock.createNiceMock(TaskStorage.class),
         EasyMock.createNiceMock(CuratorFramework.class),
         new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null)
-    ) {
+    )
+    {
       @Override
       protected WorkerHolder createWorkerHolder(
           ObjectMapper smileMapper,
@@ -182,7 +184,8 @@ public void testOneStuckTaskAssignmentDoesntBlockOthers() throws Exception
 
     HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner(
         TestHelper.makeJsonMapper(),
-        new HttpRemoteTaskRunnerConfig() {
+        new HttpRemoteTaskRunnerConfig()
+        {
           @Override
           public int getPendingTasksRunnerNumThreads()
           {
@@ -196,7 +199,8 @@ public int getPendingTasksRunnerNumThreads()
         EasyMock.createNiceMock(TaskStorage.class),
         EasyMock.createNiceMock(CuratorFramework.class),
         new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null)
-    ) {
+    )
+    {
       @Override
       protected WorkerHolder createWorkerHolder(
           ObjectMapper smileMapper,
@@ -281,7 +285,8 @@ public void testTaskRunnerRestart() throws Exception
 
     HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner(
         TestHelper.makeJsonMapper(),
-        new HttpRemoteTaskRunnerConfig() {
+        new HttpRemoteTaskRunnerConfig()
+        {
           @Override
           public int getPendingTasksRunnerNumThreads()
           {
@@ -295,7 +300,8 @@ public int getPendingTasksRunnerNumThreads()
         taskStorageMock,
         EasyMock.createNiceMock(CuratorFramework.class),
         new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null)
-    ) {
+    )
+    {
       @Override
       protected WorkerHolder createWorkerHolder(
           ObjectMapper smileMapper,
@@ -313,7 +319,8 @@ protected WorkerHolder createWorkerHolder(
               config,
               workersSyncExec,
               listener,
-              worker);
+              worker
+          );
         } else {
           throw new ISE("No WorkerHolder for [%s].", worker.getHost());
         }
@@ -412,7 +419,8 @@ public void testWorkerDisapperAndReappearBeforeItsCleanup() throws Exception
 
     HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner(
         TestHelper.makeJsonMapper(),
-        new HttpRemoteTaskRunnerConfig() {
+        new HttpRemoteTaskRunnerConfig()
+        {
           @Override
           public int getPendingTasksRunnerNumThreads()
           {
@@ -426,7 +434,8 @@ public int getPendingTasksRunnerNumThreads()
         EasyMock.createNiceMock(TaskStorage.class),
         EasyMock.createNiceMock(CuratorFramework.class),
         new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null)
-    ) {
+    )
+    {
       @Override
       protected WorkerHolder createWorkerHolder(
           ObjectMapper smileMapper,
@@ -444,7 +453,8 @@ protected WorkerHolder createWorkerHolder(
               config,
               workersSyncExec,
               listener,
-              worker);
+              worker
+          );
         } else {
           throw new ISE("No WorkerHolder for [%s].", worker.getHost());
         }
@@ -575,7 +585,8 @@ public void testWorkerDisapperAndReappearAfterItsCleanup() throws Exception
 
     HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner(
         TestHelper.makeJsonMapper(),
-        new HttpRemoteTaskRunnerConfig() {
+        new HttpRemoteTaskRunnerConfig()
+        {
           @Override
           public Period getTaskCleanupTimeout()
           {
@@ -589,7 +600,8 @@ public Period getTaskCleanupTimeout()
         EasyMock.createNiceMock(TaskStorage.class),
         EasyMock.createNiceMock(CuratorFramework.class),
         new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null)
-    ) {
+    )
+    {
       @Override
       protected WorkerHolder createWorkerHolder(
           ObjectMapper smileMapper,
@@ -607,7 +619,8 @@ protected WorkerHolder createWorkerHolder(
               config,
               workersSyncExec,
               listener,
-              worker);
+              worker
+          );
         } else {
           throw new ISE("No WorkerHolder for [%s].", worker.getHost());
         }
@@ -775,7 +788,8 @@ protected WorkerHolder createWorkerHolder(
               config,
               workersSyncExec,
               listener,
-              worker);
+              worker
+          );
         } else {
           throw new ISE("No WorkerHolder for [%s].", worker.getHost());
         }
@@ -891,9 +905,10 @@ protected WorkerHolder createWorkerHolder(
     Assert.assertEquals(task1.getId(), Iterables.getOnlyElement(taskRunner.getRunningTasks()).getTaskId());
     Assert.assertEquals(task2.getId(), Iterables.getOnlyElement(taskRunner.getPendingTasks()).getTaskId());
 
-    Assert.assertEquals("host3:8080",
-                        Iterables.getOnlyElement(taskRunner.markWorkersLazy(Predicates.alwaysTrue(), Integer.MAX_VALUE))
-                                 .getHost()
+    Assert.assertEquals(
+        "host3:8080",
+        Iterables.getOnlyElement(taskRunner.markWorkersLazy(Predicates.alwaysTrue(), Integer.MAX_VALUE))
+                 .getHost()
     );
   }
 
@@ -946,7 +961,9 @@ public void testTaskAddedOrUpdated1() throws Exception
 
     // Another "rogue-worker" reports running it, and gets asked to shutdown the task
     WorkerHolder rogueWorkerHolder = EasyMock.createMock(WorkerHolder.class);
-    EasyMock.expect(rogueWorkerHolder.getWorker()).andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1")).anyTimes();
+    EasyMock.expect(rogueWorkerHolder.getWorker())
+            .andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1"))
+            .anyTimes();
     rogueWorkerHolder.shutdownTask(task.getId());
     EasyMock.replay(rogueWorkerHolder);
     taskRunner.taskAddedOrUpdated(TaskAnnouncement.create(
@@ -959,7 +976,9 @@ public void testTaskAddedOrUpdated1() throws Exception
 
     // "rogue-worker" reports FAILURE for the task, ignored
     rogueWorkerHolder = EasyMock.createMock(WorkerHolder.class);
-    EasyMock.expect(rogueWorkerHolder.getWorker()).andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1")).anyTimes();
+    EasyMock.expect(rogueWorkerHolder.getWorker())
+            .andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1"))
+            .anyTimes();
     EasyMock.replay(rogueWorkerHolder);
     taskRunner.taskAddedOrUpdated(TaskAnnouncement.create(
         task,
@@ -980,7 +999,9 @@ public void testTaskAddedOrUpdated1() throws Exception
 
     // "rogue-worker" reports running it, and gets asked to shutdown the task
     rogueWorkerHolder = EasyMock.createMock(WorkerHolder.class);
-    EasyMock.expect(rogueWorkerHolder.getWorker()).andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1")).anyTimes();
+    EasyMock.expect(rogueWorkerHolder.getWorker())
+            .andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1"))
+            .anyTimes();
     rogueWorkerHolder.shutdownTask(task.getId());
     EasyMock.replay(rogueWorkerHolder);
     taskRunner.taskAddedOrUpdated(TaskAnnouncement.create(
@@ -993,7 +1014,9 @@ public void testTaskAddedOrUpdated1() throws Exception
 
     // "rogue-worker" reports FAILURE for the tasks, ignored
     rogueWorkerHolder = EasyMock.createMock(WorkerHolder.class);
-    EasyMock.expect(rogueWorkerHolder.getWorker()).andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1")).anyTimes();
+    EasyMock.expect(rogueWorkerHolder.getWorker())
+            .andReturn(new Worker("http", "rogue-worker", "127.0.0.1", 5, "v1"))
+            .anyTimes();
     EasyMock.replay(rogueWorkerHolder);
     taskRunner.taskAddedOrUpdated(TaskAnnouncement.create(
         task,
@@ -1206,7 +1229,7 @@ public void statusChanged(String taskId, TaskStatus status)
               listenerNotificationsAccumulator.add(ImmutableList.of(taskId, status));
             }
           },
-          MoreExecutors.sameThreadExecutor()
+          Execs.sameThreadExecutor()
       );
     }
 
diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java
index 3c5a63755a6..dd1a1b13a04 100644
--- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java
+++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java
@@ -21,6 +21,8 @@
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 import javax.annotation.Nullable;
@@ -49,6 +51,16 @@ public static ExecutorService dummy()
     return DummyExecutorService.INSTANCE;
   }
 
+  /**
+   * Guava changes the name from `MoreExecutors.sameThreadExecutor()` to `MoreExecutors.directExecutor()`
+   * quite abruptly. This serves as a compatability method to work in both environments for the common use cases
+   * in druid.io
+   */
+  public static ListeningExecutorService sameThreadExecutor()
+  {
+    return MoreExecutors.listeningDecorator(new SameThreadExecutorService());
+  }
+
   public static ExecutorService singleThreaded(@NotNull String nameFormat)
   {
     return singleThreaded(nameFormat, null);
diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java
new file mode 100644
index 00000000000..99b8b224ffc
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/SameThreadExecutorService.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.java.util.common.concurrent;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.AbstractExecutorService;
+import java.util.concurrent.Phaser;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A simple class that implements the ExecutorService interface, but runs the code on a call to submit
+ * Use {@link Execs#sameThreadExecutor() Execs.sameThreadExecutor()} to get the instance
+ */
+public class SameThreadExecutorService extends AbstractExecutorService
+{
+  private final AtomicBoolean shutdownLeader = new AtomicBoolean(true);
+  private final Phaser shutdownPhaser = new Phaser(0);
+  private final int initialPhase = shutdownPhaser.register();
+
+  SameThreadExecutorService()
+  {
+
+  }
+
+  @Override
+  public void shutdown()
+  {
+    if (shutdownLeader.getAndSet(false)) {
+      shutdownPhaser.arriveAndDeregister();
+    }
+  }
+
+  @Override
+  public List<Runnable> shutdownNow()
+  {
+    shutdown();
+    return Collections.emptyList();
+  }
+
+  @Override
+  public boolean isShutdown()
+  {
+    return !shutdownLeader.get();
+  }
+
+  @Override
+  public boolean isTerminated()
+  {
+    return isShutdown() && shutdownPhaser.getRegisteredParties() < 1;
+  }
+
+  @Override
+  public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException
+  {
+    try {
+      shutdownPhaser.awaitAdvanceInterruptibly(initialPhase, timeout, unit);
+      return true;
+    }
+    catch (TimeoutException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public void execute(Runnable command)
+  {
+    shutdownPhaser.register();
+    try {
+      if (isShutdown()) {
+        throw new RejectedExecutionException();
+      }
+      command.run();
+    }
+    finally {
+      shutdownPhaser.arriveAndDeregister();
+    }
+  }
+}
diff --git a/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java b/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java
new file mode 100644
index 00000000000..8b2622a3ae7
--- /dev/null
+++ b/java-util/src/test/java/io/druid/java/util/common/concurrent/SameThreadExecutorServiceTest.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.java.util.common.concurrent;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+public class SameThreadExecutorServiceTest
+{
+
+  @Test
+  public void timeoutAndShutdownTest() throws Exception
+  {
+    final SameThreadExecutorService service = new SameThreadExecutorService();
+    Assert.assertFalse(service.awaitTermination(10, TimeUnit.MILLISECONDS));
+  }
+
+  @Test
+  public void runsTasks() throws Exception
+  {
+    final SameThreadExecutorService service = new SameThreadExecutorService();
+    final CountDownLatch finished = new CountDownLatch(1);
+    service.submit(finished::countDown);
+    finished.await();
+  }
+
+  @Test
+  public void multiShutdownIsFine()
+  {
+    final SameThreadExecutorService service = new SameThreadExecutorService();
+    Assert.assertFalse(service.isShutdown());
+    service.shutdown();
+    Assert.assertTrue(service.isShutdown());
+    service.shutdown();
+    Assert.assertTrue(service.isShutdown());
+  }
+
+  @Test(expected = ExecutionException.class)
+  public void exceptionsCaught() throws Exception
+  {
+    final SameThreadExecutorService service = new SameThreadExecutorService();
+    service.submit(() -> {
+      throw new RuntimeException();
+    }).get();
+  }
+}
diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java
index da6494c0eaa..438daa40159 100644
--- a/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java
+++ b/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java
@@ -19,7 +19,7 @@
 
 package io.druid.java.util.common.guava;
 
-import com.google.common.util.concurrent.MoreExecutors;
+import io.druid.java.util.common.concurrent.Execs;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -40,11 +40,11 @@ public void testConsistentEffectApplicationOrder()
         .simple(Arrays.asList(1, 2, 3))
         .withEffect(
             () -> effect1.set(counter.incrementAndGet()),
-            MoreExecutors.sameThreadExecutor()
+            Execs.sameThreadExecutor()
         )
         .withEffect(
             () -> effect2.set(counter.incrementAndGet()),
-            MoreExecutors.sameThreadExecutor()
+            Execs.sameThreadExecutor()
         );
     // Run sequence via accumulate
     sequence.toList();
@@ -70,7 +70,7 @@ public void testEffectExecutedIfWrappedSequenceThrowsExceptionFromClose()
     });
     final AtomicBoolean effectExecuted = new AtomicBoolean();
     Sequence<Integer> seqWithEffect =
-        throwingSeq.withEffect(() -> effectExecuted.set(true), MoreExecutors.sameThreadExecutor());
+        throwingSeq.withEffect(() -> effectExecuted.set(true), Execs.sameThreadExecutor());
     try {
       seqWithEffect.toList();
       Assert.fail("expected RuntimeException");
diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java
index fc05d247a99..bc94c8c5f7c 100644
--- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java
+++ b/processing/src/test/java/io/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 io.druid.java.util.common.DateTimes;
 import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.common.granularity.Granularity;
 import io.druid.java.util.common.guava.MergeSequence;
@@ -81,7 +81,8 @@
 public class QueryRunnerTestHelper
 {
 
-  public static final QueryWatcher NOOP_QUERYWATCHER = (query, future) -> {};
+  public static final QueryWatcher NOOP_QUERYWATCHER = (query, future) -> {
+  };
 
   public static final String segmentId = "testSegment";
   public static final String dataSource = "testing";
@@ -525,7 +526,7 @@ public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunn
   public static IntervalChunkingQueryRunnerDecorator sameThreadIntervalChunkingQueryRunnerDecorator()
   {
     return new IntervalChunkingQueryRunnerDecorator(
-        MoreExecutors.sameThreadExecutor(),
+        Execs.sameThreadExecutor(),
         QueryRunnerTestHelper.NOOP_QUERYWATCHER,
         new ServiceEmitter("dummy", "dummy", new NoopEmitter())
     );
diff --git a/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java b/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java
index 9a96d28db49..180824417c0 100644
--- a/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java
+++ b/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java
@@ -24,7 +24,6 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import com.google.common.io.Closeables;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.data.input.InputRow;
 import io.druid.data.input.impl.DimensionsSpec;
 import io.druid.data.input.impl.MapInputRowParser;
@@ -32,6 +31,7 @@
 import io.druid.data.input.impl.TimestampSpec;
 import io.druid.java.util.common.DateTimes;
 import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.guava.FunctionalIterable;
 import io.druid.java.util.common.guava.Sequence;
 import io.druid.query.aggregation.AggregatorFactory;
@@ -87,11 +87,22 @@
         )
     );
     return ImmutableList.of(
-        parser.parseBatch(ImmutableMap.<String, Object>of("t", "2000-01-01", "c1", "9", "c2", ImmutableList.of("a"))).get(0),
-        parser.parseBatch(ImmutableMap.<String, Object>of("t", "2000-01-02", "c1", "10.1", "c2", ImmutableList.of())).get(0),
-        parser.parseBatch(ImmutableMap.<String, Object>of("t", "2000-01-03", "c1", "2", "c2", ImmutableList.of(""))).get(0),
-        parser.parseBatch(ImmutableMap.<String, Object>of("t", "2001-01-01", "c1", "1", "c2", ImmutableList.of("a", "c"))).get(0),
-        parser.parseBatch(ImmutableMap.<String, Object>of("t", "2001-01-02", "c1", "4", "c2", ImmutableList.of("abc"))).get(0),
+        parser.parseBatch(ImmutableMap.<String, Object>of("t", "2000-01-01", "c1", "9", "c2", ImmutableList.of("a")))
+              .get(0),
+        parser.parseBatch(ImmutableMap.<String, Object>of("t", "2000-01-02", "c1", "10.1", "c2", ImmutableList.of()))
+              .get(0),
+        parser.parseBatch(ImmutableMap.<String, Object>of("t", "2000-01-03", "c1", "2", "c2", ImmutableList.of("")))
+              .get(0),
+        parser.parseBatch(ImmutableMap.<String, Object>of(
+            "t",
+            "2001-01-01",
+            "c1",
+            "1",
+            "c2",
+            ImmutableList.of("a", "c")
+        )).get(0),
+        parser.parseBatch(ImmutableMap.<String, Object>of("t", "2001-01-02", "c1", "4", "c2", ImmutableList.of("abc")))
+              .get(0),
         parser.parseBatch(ImmutableMap.<String, Object>of("t", "2001-01-03", "c1", "5")).get(0)
     );
   }
@@ -105,7 +116,7 @@
     final Sequence<T> results = new FinalizeResultsQueryRunner<>(
         factory.getToolchest().mergeResults(
             factory.mergeRunners(
-                MoreExecutors.sameThreadExecutor(),
+                Execs.sameThreadExecutor(),
                 FunctionalIterable
                     .create(indexes)
                     .transform(
diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java
index 06575a13673..0c4e82aaa41 100644
--- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java
+++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java
@@ -33,20 +33,19 @@
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.io.Closeables;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.collections.StupidPool;
 import io.druid.data.input.InputRow;
 import io.druid.data.input.Row;
 import io.druid.data.input.impl.InputRowParser;
 import io.druid.data.input.impl.StringInputRowParser;
 import io.druid.java.util.common.IAE;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularity;
 import io.druid.java.util.common.guava.CloseQuietly;
 import io.druid.java.util.common.guava.Sequence;
 import io.druid.java.util.common.guava.Sequences;
 import io.druid.java.util.common.guava.Yielder;
 import io.druid.java.util.common.guava.YieldingAccumulator;
-import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
 import io.druid.query.FinalizeResultsQueryRunner;
 import io.druid.query.Query;
 import io.druid.query.QueryPlus;
@@ -79,6 +78,7 @@
 import io.druid.segment.column.ColumnConfig;
 import io.druid.segment.incremental.IncrementalIndex;
 import io.druid.segment.incremental.IncrementalIndexSchema;
+import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.LineIterator;
 import org.junit.rules.TemporaryFolder;
@@ -525,7 +525,7 @@ public Segment apply(File segmentDir)
             toolChest.mergeResults(
                 toolChest.preMergeQueryDecoration(
                     factory.mergeRunners(
-                        MoreExecutors.sameThreadExecutor(),
+                        Execs.sameThreadExecutor(),
                         Lists.transform(
                             segments,
                             new Function<Segment, QueryRunner>()
diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java
index 23f15b7e320..aa1d1733be3 100644
--- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java
@@ -25,12 +25,12 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.collections.DefaultBlockingPool;
 import io.druid.collections.NonBlockingPool;
 import io.druid.collections.ReferenceCountingResourceHolder;
 import io.druid.collections.StupidPool;
 import io.druid.data.input.Row;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.query.DruidProcessingConfig;
 import io.druid.query.QueryContexts;
@@ -60,6 +60,7 @@
 public class GroupByQueryMergeBufferTest
 {
   private static final long TIMEOUT = 5000;
+
   private static class TestBlockingPool extends DefaultBlockingPool<ByteBuffer>
   {
     private int minRemainBufferNum;
@@ -213,7 +214,7 @@ public String getDefaultStrategy()
 
   public GroupByQueryMergeBufferTest(QueryRunner<Row> runner)
   {
-    this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.<QueryRunner<Row>>of(runner));
+    this.runner = factory.mergeRunners(Execs.sameThreadExecutor(), ImmutableList.<QueryRunner<Row>>of(runner));
   }
 
   @Before
@@ -251,7 +252,10 @@ public void testNestedGroupBy()
                             .setDataSource(QueryRunnerTestHelper.dataSource)
                             .setInterval(QueryRunnerTestHelper.firstToThird)
                             .setGranularity(Granularities.ALL)
-                            .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
+                            .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec(
+                                "quality",
+                                "alias"
+                            )))
                             .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
                             .build()
             )
@@ -285,12 +289,16 @@ public void testDoubleNestedGroupBy()
                                                 new DefaultDimensionSpec("quality", "alias"),
                                                 new DefaultDimensionSpec("market", null)
                                             ))
-                                            .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
+                                            .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(
+                                                QueryRunnerTestHelper.rowsCount))
                                             .build()
                             )
                             .setInterval(QueryRunnerTestHelper.firstToThird)
                             .setGranularity(Granularities.ALL)
-                            .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
+                            .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec(
+                                "quality",
+                                "alias"
+                            )))
                             .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
                             .build()
             )
@@ -328,7 +336,8 @@ public void testTripleNestedGroupBy()
                                                                 new DefaultDimensionSpec("market", null),
                                                                 new DefaultDimensionSpec("placement", null)
                                                             ))
-                                                            .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
+                                                            .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(
+                                                                QueryRunnerTestHelper.rowsCount))
                                                             .build()
                                             )
                                             .setInterval(QueryRunnerTestHelper.firstToThird)
@@ -337,12 +346,16 @@ public void testTripleNestedGroupBy()
                                                 new DefaultDimensionSpec("quality", "alias"),
                                                 new DefaultDimensionSpec("market", null)
                                             ))
-                                            .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
+                                            .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(
+                                                QueryRunnerTestHelper.rowsCount))
                                             .build()
                             )
                             .setInterval(QueryRunnerTestHelper.firstToThird)
                             .setGranularity(Granularities.ALL)
-                            .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
+                            .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec(
+                                "quality",
+                                "alias"
+                            )))
                             .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
                             .build()
             )
diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java
index 74d5b2f5d67..3d310e540d8 100644
--- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java
@@ -25,13 +25,13 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.collections.BlockingPool;
 import io.druid.collections.DefaultBlockingPool;
 import io.druid.collections.NonBlockingPool;
 import io.druid.collections.ReferenceCountingResourceHolder;
 import io.druid.collections.StupidPool;
 import io.druid.data.input.Row;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.query.DruidProcessingConfig;
 import io.druid.query.InsufficientResourcesException;
@@ -177,7 +177,7 @@ public String getDefaultStrategy()
 
   public GroupByQueryRunnerFailureTest(QueryRunner<Row> runner)
   {
-    this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(runner));
+    this.runner = factory.mergeRunners(Execs.sameThreadExecutor(), ImmutableList.of(runner));
   }
 
   @Test(timeout = 10000)
@@ -194,7 +194,10 @@ public void testNotEnoughMergeBuffersOnQueryable()
                             .setDataSource(QueryRunnerTestHelper.dataSource)
                             .setInterval(QueryRunnerTestHelper.firstToThird)
                             .setGranularity(Granularities.ALL)
-                            .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
+                            .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec(
+                                "quality",
+                                "alias"
+                            )))
                             .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
                             .build()
             )
@@ -227,12 +230,16 @@ public void testResourceLimitExceededOnBroker()
                                                 new DefaultDimensionSpec("quality", "alias"),
                                                 new DefaultDimensionSpec("market", null)
                                             ))
-                                            .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
+                                            .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(
+                                                QueryRunnerTestHelper.rowsCount))
                                             .build()
                             )
                             .setInterval(QueryRunnerTestHelper.firstToThird)
                             .setGranularity(Granularities.ALL)
-                            .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
+                            .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec(
+                                "quality",
+                                "alias"
+                            )))
                             .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
                             .build()
             )
@@ -257,7 +264,10 @@ public void testInsufficientResourcesOnBroker()
                             .setDataSource(QueryRunnerTestHelper.dataSource)
                             .setInterval(QueryRunnerTestHelper.firstToThird)
                             .setGranularity(Granularities.ALL)
-                            .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
+                            .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec(
+                                "quality",
+                                "alias"
+                            )))
                             .setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
                             .build()
             )
diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java
index 42c0c2f0085..d5b51eefb88 100644
--- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java
@@ -30,7 +30,6 @@
 import com.google.common.collect.Maps;
 import com.google.common.collect.Ordering;
 import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.collections.BlockingPool;
 import io.druid.collections.DefaultBlockingPool;
 import io.druid.collections.NonBlockingPool;
@@ -41,6 +40,7 @@
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.Intervals;
 import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.DurationGranularity;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.common.granularity.PeriodGranularity;
@@ -421,7 +421,7 @@ public GroupByQueryRunnerTest(
   {
     this.config = config;
     this.factory = factory;
-    this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.<QueryRunner<Row>>of(runner));
+    this.runner = factory.mergeRunners(Execs.sameThreadExecutor(), ImmutableList.<QueryRunner<Row>>of(runner));
   }
 
   @Test
@@ -444,25 +444,241 @@ public void testGroupBy()
         .build();
 
     List<Row> 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<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
@@ -530,25 +746,205 @@ public void testGroupByWithStringPostAggregator()
         .build();
 
     List<Row> expectedResults = Arrays.asList(
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "post", "travelx", "rows", 1L, "idx", 119L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "post", "technologyx", "rows", 1L, "idx", 78L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "post", "premiumx", "rows", 3L, "idx", 2900L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "post", "newsx", "rows", 1L, "idx", 121L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "post", "mezzaninex", "rows", 3L, "idx", 2870L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "post", "healthx", "rows", 1L, "idx", 120L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "post", "entertainmentx", "rows", 1L, "idx", 158L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "post", "businessx", "rows", 1L, "idx", 118L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "post", "automotivex", "rows", 1L, "idx", 135L),
-
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "post", "travelx", "rows", 1L, "idx", 126L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "post", "technologyx", "rows", 1L, "idx", 97L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "post", "premiumx", "rows", 3L, "idx", 2505L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "post", "newsx", "rows", 1L, "idx", 114L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "post", "mezzaninex", "rows", 3L, "idx", 2447L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "post", "healthx", "rows", 1L, "idx", 113L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "post", "entertainmentx", "rows", 1L, "idx", 166L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "post", "businessx", "rows", 1L, "idx", 112L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "post", "automotivex", "rows", 1L, "idx", 147L)
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "travel",
+            "post",
+            "travelx",
+            "rows",
+            1L,
+            "idx",
+            119L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "technology",
+            "post",
+            "technologyx",
+            "rows",
+            1L,
+            "idx",
+            78L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "premium",
+            "post",
+            "premiumx",
+            "rows",
+            3L,
+            "idx",
+            2900L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "news",
+            "post",
+            "newsx",
+            "rows",
+            1L,
+            "idx",
+            121L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "mezzanine",
+            "post",
+            "mezzaninex",
+            "rows",
+            3L,
+            "idx",
+            2870L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "health",
+            "post",
+            "healthx",
+            "rows",
+            1L,
+            "idx",
+            120L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "entertainment",
+            "post",
+            "entertainmentx",
+            "rows",
+            1L,
+            "idx",
+            158L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "business",
+            "post",
+            "businessx",
+            "rows",
+            1L,
+            "idx",
+            118L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "automotive",
+            "post",
+            "automotivex",
+            "rows",
+            1L,
+            "idx",
+            135L
+        ),
+
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-02",
+            "alias",
+            "travel",
+            "post",
+            "travelx",
+            "rows",
+            1L,
+            "idx",
+            126L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-02",
+            "alias",
+            "technology",
+            "post",
+            "technologyx",
+            "rows",
+            1L,
+            "idx",
+            97L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-02",
+            "alias",
+            "premium",
+            "post",
+            "premiumx",
+            "rows",
+            3L,
+            "idx",
+            2505L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-02",
+            "alias",
+            "news",
+            "post",
+            "newsx",
+            "rows",
+            1L,
+            "idx",
+            114L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-02",
+            "alias",
+            "mezzanine",
+            "post",
+            "mezzaninex",
+            "rows",
+            3L,
+            "idx",
+            2447L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-02",
+            "alias",
+            "health",
+            "post",
+            "healthx",
+            "rows",
+            1L,
+            "idx",
+            113L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-02",
+            "alias",
+            "entertainment",
+            "post",
+            "entertainmentx",
+            "rows",
+            1L,
+            "idx",
+            166L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-02",
+            "alias",
+            "business",
+            "post",
+            "businessx",
+            "rows",
+            1L,
+            "idx",
+            112L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-02",
+            "alias",
+            "automotive",
+            "post",
+            "automotivex",
+            "rows",
+            1L,
+            "idx",
+            147L
+        )
     );
 
     Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
@@ -865,10 +1261,50 @@ public void testTwoMultiValueDimensions()
         .build();
 
     List<Row> expectedResults = Arrays.asList(
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "a", "alias2", "a", "rows", 2L, "idx", 282L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "a", "alias2", "preferred", "rows", 2L, "idx", 282L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "preferred", "alias2", "a", "rows", 2L, "idx", 282L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "preferred", "alias2", "preferred", "rows", 2L, "idx", 282L)
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "a",
+            "alias2",
+            "a",
+            "rows",
+            2L,
+            "idx",
+            282L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "a",
+            "alias2",
+            "preferred",
+            "rows",
+            2L,
+            "idx",
+            282L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "preferred",
+            "alias2",
+            "a",
+            "rows",
+            2L,
+            "idx",
+            282L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "preferred",
+            "alias2",
+            "preferred",
+            "rows",
+            2L,
+            "idx",
+            282L
+        )
     );
 
     Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
@@ -1348,25 +1784,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)
       );
     }
 
@@ -1444,7 +1896,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),
@@ -1454,7 +1914,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),
@@ -1494,7 +1962,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),
@@ -1504,7 +1980,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),
@@ -1549,7 +2033,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),
@@ -1559,7 +2051,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),
@@ -2127,16 +2627,64 @@ public void testGroupByWithFirstLast()
 
     List<Row> 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)
     );
 
@@ -2928,7 +3476,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),
@@ -3059,8 +3608,16 @@ public void testGroupByOrderLimitNumeric()
                 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();
@@ -3958,10 +4515,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<Row> expectedResults = Arrays.asList(
         GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L),
@@ -5579,8 +6144,10 @@ public void testSubqueryWithOuterFilterAggregator()
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("market", "market"),
-                                                         new DefaultDimensionSpec("quality", "quality")))
+        .setDimensions(Lists.<DimensionSpec>newArrayList(
+            new DefaultDimensionSpec("market", "market"),
+            new DefaultDimensionSpec("quality", "quality")
+        ))
         .setAggregatorSpecs(
             Arrays.asList(
                 QueryRunnerTestHelper.rowsCount,
@@ -5618,8 +6185,10 @@ public void testSubqueryWithOuterTimeFilter()
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("market", "market"),
-                                                         new DefaultDimensionSpec("quality", "quality")))
+        .setDimensions(Lists.<DimensionSpec>newArrayList(
+            new DefaultDimensionSpec("market", "market"),
+            new DefaultDimensionSpec("quality", "quality")
+        ))
         .setAggregatorSpecs(
             Arrays.asList(
                 QueryRunnerTestHelper.rowsCount,
@@ -5629,8 +6198,16 @@ public void testSubqueryWithOuterTimeFilter()
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
-    final DimFilter fridayFilter = new SelectorDimFilter(Column.TIME_COLUMN_NAME, "Friday", new TimeFormatExtractionFn("EEEE", null, null, null, false));
-    final DimFilter firstDaysFilter = new InDimFilter(Column.TIME_COLUMN_NAME, ImmutableList.of("1", "2", "3"), new TimeFormatExtractionFn("d", null, null, null, false));
+    final DimFilter fridayFilter = new SelectorDimFilter(
+        Column.TIME_COLUMN_NAME,
+        "Friday",
+        new TimeFormatExtractionFn("EEEE", null, null, null, false)
+    );
+    final DimFilter firstDaysFilter = new InDimFilter(
+        Column.TIME_COLUMN_NAME,
+        ImmutableList.of("1", "2", "3"),
+        new TimeFormatExtractionFn("d", null, null, null, false)
+    );
     final GroupByQuery query = GroupByQuery
         .builder()
         .setDataSource(subquery)
@@ -5743,12 +6320,13 @@ public void testSubqueryWithOuterCardinalityAggregator()
         .setDimensions(Lists.<DimensionSpec>newArrayList())
         .setAggregatorSpecs(
             ImmutableList.<AggregatorFactory>of(
-                new CardinalityAggregatorFactory("car",
-                                                 ImmutableList.<DimensionSpec>of(new DefaultDimensionSpec(
-                                                     "quality",
-                                                     "quality"
-                                                 )),
-                                                 false
+                new CardinalityAggregatorFactory(
+                    "car",
+                    ImmutableList.<DimensionSpec>of(new DefaultDimensionSpec(
+                        "quality",
+                        "quality"
+                    )),
+                    false
                 )
             )
         )
@@ -5813,8 +6391,10 @@ public void testSubqueryWithOuterDimJavascriptAggregators()
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("market", "market"),
-                                                         new DefaultDimensionSpec("quality", "quality")))
+        .setDimensions(Lists.<DimensionSpec>newArrayList(
+            new DefaultDimensionSpec("market", "market"),
+            new DefaultDimensionSpec("quality", "quality")
+        ))
         .setAggregatorSpecs(
             Arrays.asList(
                 QueryRunnerTestHelper.rowsCount,
@@ -5876,8 +6456,10 @@ public void testSubqueryWithOuterJavascriptAggregators()
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("market", "market"),
-                                                         new DefaultDimensionSpec("quality", "quality")))
+        .setDimensions(Lists.<DimensionSpec>newArrayList(
+            new DefaultDimensionSpec("market", "market"),
+            new DefaultDimensionSpec("quality", "quality")
+        ))
         .setAggregatorSpecs(
             Arrays.asList(
                 QueryRunnerTestHelper.rowsCount,
@@ -6484,7 +7066,7 @@ public ExtractionType getExtractionType()
                         new ExtractionFn[]{
                             new TimeFormatExtractionFn("EEEE", null, null, null, false),
                             nullWednesdays,
-                        }
+                            }
                     )
                 )
             )
@@ -7275,7 +7857,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<DimFilter> superFilterList = new ArrayList<>();
     superFilterList.add(new SelectorDimFilter("quality", "super-mezzanine", extractionFn));
@@ -7359,7 +7945,12 @@ public void testGroupByWithAllFiltersOnNullDimsWithExtractionFns()
         new ContainsSearchQuerySpec("EMPTY", true),
         extractionFn
     ));
-    superFilterList.add(new JavaScriptDimFilter("null_column", jsFn, extractionFn, JavaScriptConfig.getEnabledInstance()));
+    superFilterList.add(new JavaScriptDimFilter(
+        "null_column",
+        jsFn,
+        extractionFn,
+        JavaScriptConfig.getEnabledInstance()
+    ));
     DimFilter superFilter = new AndDimFilter(superFilterList);
 
     GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource)
@@ -7415,12 +8006,60 @@ public void testGroupByCardinalityAggWithExtractionFn()
         .build();
 
     List<Row> 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<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
@@ -7452,12 +8091,60 @@ public void testGroupByCardinalityAggOnFloat()
         .build();
 
     List<Row> 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<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
@@ -7476,7 +8163,11 @@ public void testGroupByLongColumn()
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG)))
+        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec(
+            "qualityLong",
+            "ql_alias",
+            ValueType.LONG
+        )))
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
         .setAggregatorSpecs(
             Arrays.asList(
@@ -7539,7 +8230,11 @@ public void testGroupByLongColumnDescending()
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG)))
+        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec(
+            "qualityLong",
+            "ql_alias",
+            ValueType.LONG
+        )))
         .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null))
         .setAggregatorSpecs(
             Arrays.asList(
@@ -7605,7 +8300,11 @@ public void testGroupByLongColumnWithExFn()
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.<DimensionSpec>newArrayList(new ExtractionDimensionSpec("qualityLong", "ql_alias", jsExtractionFn)))
+        .setDimensions(Lists.<DimensionSpec>newArrayList(new ExtractionDimensionSpec(
+            "qualityLong",
+            "ql_alias",
+            jsExtractionFn
+        )))
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
         .setAggregatorSpecs(
             Arrays.asList(
@@ -7652,7 +8351,11 @@ public void testGroupByLongTimeColumn()
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG)))
+        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec(
+            "__time",
+            "time_alias",
+            ValueType.LONG
+        )))
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
         .setAggregatorSpecs(
             Arrays.asList(
@@ -7697,7 +8400,11 @@ public void testGroupByLongTimeColumnWithExFn()
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.<DimensionSpec>newArrayList(new ExtractionDimensionSpec("__time", "time_alias", jsExtractionFn)))
+        .setDimensions(Lists.<DimensionSpec>newArrayList(new ExtractionDimensionSpec(
+            "__time",
+            "time_alias",
+            jsExtractionFn
+        )))
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
         .setAggregatorSpecs(
             Arrays.asList(
@@ -7744,7 +8451,11 @@ public void testGroupByFloatColumn()
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT)))
+        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec(
+            "index",
+            "index_alias",
+            ValueType.FLOAT
+        )))
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
         .setAggregatorSpecs(
             Arrays.asList(
@@ -7808,7 +8519,11 @@ public void testGroupByFloatColumnDescending()
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("qualityFloat", "qf_alias", ValueType.FLOAT)))
+        .setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec(
+            "qualityFloat",
+            "qf_alias",
+            ValueType.FLOAT
+        )))
         .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null))
         .setAggregatorSpecs(
             Arrays.asList(
@@ -7937,7 +8652,11 @@ public void testGroupByFloatColumnWithExFn()
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.<DimensionSpec>newArrayList(new ExtractionDimensionSpec("index", "index_alias", jsExtractionFn)))
+        .setDimensions(Lists.<DimensionSpec>newArrayList(new ExtractionDimensionSpec(
+            "index",
+            "index_alias",
+            jsExtractionFn
+        )))
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
         .setAggregatorSpecs(
             Arrays.asList(
@@ -9029,11 +9748,61 @@ public void testMergeResultsWithLimitPushDownSortByDimDim()
     Map<String, Object> context = Maps.newHashMap();
 
     List<Row> 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<Row> results = mergedRunner.run(QueryPlus.wrap(allGranQuery), context).toList();
@@ -9123,11 +9892,61 @@ public void testMergeResultsWithLimitPushDownSortByDimAggDim()
     Map<String, Object> context = Maps.newHashMap();
 
     List<Row> 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<Row> results = mergedRunner.run(QueryPlus.wrap(allGranQuery), context).toList();
@@ -9401,19 +10220,42 @@ public void testTypeConversionWithMergingChainedExecutionRunner()
         .build();
 
     List<Row> expectedResults = Arrays.asList(
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "qualityLen", 10L, "rows", 2L, "idx", 156L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "qualityLen", 10L, "rows", 2L, "idx", 194L)
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias",
+            "technology",
+            "qualityLen",
+            10L,
+            "rows",
+            2L,
+            "idx",
+            156L
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-02",
+            "alias",
+            "technology",
+            "qualityLen",
+            10L,
+            "rows",
+            2L,
+            "idx",
+            194L
+        )
     );
 
     ChainedExecutionQueryRunner ceqr = new ChainedExecutionQueryRunner(
-        MoreExecutors.sameThreadExecutor(),
+        Execs.sameThreadExecutor(),
         (query1, future) -> {
           return;
         },
         ImmutableList.<QueryRunner<Row>>of(runner, runner)
     );
 
-    QueryRunner<Row> mergingRunner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.<QueryRunner<Row>>of(ceqr));
+    QueryRunner<Row> mergingRunner = factory.mergeRunners(
+        Execs.sameThreadExecutor(),
+        ImmutableList.<QueryRunner<Row>>of(ceqr)
+    );
 
     Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, mergingRunner, query);
     TestHelper.assertExpectedObjects(expectedResults, results, "");
diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java
index f9ab7f11fbd..12596362bd0 100644
--- a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java
@@ -22,11 +22,11 @@
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.data.input.MapBasedRow;
 import io.druid.data.input.Row;
 import io.druid.java.util.common.DateTimes;
 import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.common.guava.Sequence;
 import io.druid.java.util.common.guava.Sequences;
@@ -80,7 +80,7 @@ public Sequence run(QueryPlus queryPlus, Map responseContext)
                   {
                     TimeseriesQuery tsQuery = (TimeseriesQuery) queryPlus.getQuery();
                     QueryRunner<Row> newRunner = factory.mergeRunners(
-                        MoreExecutors.sameThreadExecutor(), ImmutableList.<QueryRunner<Row>>of(input)
+                        Execs.sameThreadExecutor(), ImmutableList.<QueryRunner<Row>>of(input)
                     );
                     QueryToolChest toolChest = factory.getToolchest();
 
diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java
index 6fe868623e8..90a51876109 100644
--- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java
+++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java
@@ -25,10 +25,10 @@
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.data.input.impl.TimestampSpec;
 import io.druid.jackson.DefaultObjectMapper;
 import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.query.BySegmentResultValue;
 import io.druid.query.BySegmentResultValueClass;
@@ -303,7 +303,7 @@ public void testSegmentMetadataQueryWithRollupMerge()
     QueryRunner myRunner = new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             FACTORY.mergeRunners(
-                MoreExecutors.sameThreadExecutor(),
+                Execs.sameThreadExecutor(),
                 Lists.<QueryRunner<SegmentAnalysis>>newArrayList(
                     toolChest.preMergeQueryDecoration(runner1),
                     toolChest.preMergeQueryDecoration(runner2)
@@ -371,7 +371,7 @@ public void testSegmentMetadataQueryWithHasMultipleValuesMerge()
     QueryRunner myRunner = new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             FACTORY.mergeRunners(
-                MoreExecutors.sameThreadExecutor(),
+                Execs.sameThreadExecutor(),
                 Lists.<QueryRunner<SegmentAnalysis>>newArrayList(
                     toolChest.preMergeQueryDecoration(runner1),
                     toolChest.preMergeQueryDecoration(runner2)
@@ -439,7 +439,7 @@ public void testSegmentMetadataQueryWithComplexColumnMerge()
     QueryRunner myRunner = new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             FACTORY.mergeRunners(
-                MoreExecutors.sameThreadExecutor(),
+                Execs.sameThreadExecutor(),
                 Lists.<QueryRunner<SegmentAnalysis>>newArrayList(
                     toolChest.preMergeQueryDecoration(runner1),
                     toolChest.preMergeQueryDecoration(runner2)
@@ -556,7 +556,7 @@ private void testSegmentMetadataQueryWithDefaultAnalysisMerge(
     QueryRunner myRunner = new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             FACTORY.mergeRunners(
-                MoreExecutors.sameThreadExecutor(),
+                Execs.sameThreadExecutor(),
                 Lists.<QueryRunner<SegmentAnalysis>>newArrayList(
                     toolChest.preMergeQueryDecoration(runner1),
                     toolChest.preMergeQueryDecoration(runner2)
@@ -608,7 +608,7 @@ public void testSegmentMetadataQueryWithNoAnalysisTypesMerge()
     QueryRunner myRunner = new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             FACTORY.mergeRunners(
-                MoreExecutors.sameThreadExecutor(),
+                Execs.sameThreadExecutor(),
                 Lists.<QueryRunner<SegmentAnalysis>>newArrayList(
                     toolChest.preMergeQueryDecoration(runner1),
                     toolChest.preMergeQueryDecoration(runner2)
@@ -670,7 +670,7 @@ public void testSegmentMetadataQueryWithAggregatorsMerge()
     QueryRunner myRunner = new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             FACTORY.mergeRunners(
-                MoreExecutors.sameThreadExecutor(),
+                Execs.sameThreadExecutor(),
                 Lists.<QueryRunner<SegmentAnalysis>>newArrayList(
                     toolChest.preMergeQueryDecoration(runner1),
                     toolChest.preMergeQueryDecoration(runner2)
@@ -728,7 +728,7 @@ public void testSegmentMetadataQueryWithTimestampSpecMerge()
     QueryRunner myRunner = new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             FACTORY.mergeRunners(
-                MoreExecutors.sameThreadExecutor(),
+                Execs.sameThreadExecutor(),
                 Lists.<QueryRunner<SegmentAnalysis>>newArrayList(
                     toolChest.preMergeQueryDecoration(runner1),
                     toolChest.preMergeQueryDecoration(runner2)
@@ -786,7 +786,7 @@ public void testSegmentMetadataQueryWithQueryGranularityMerge()
     QueryRunner myRunner = new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             FACTORY.mergeRunners(
-                MoreExecutors.sameThreadExecutor(),
+                Execs.sameThreadExecutor(),
                 Lists.<QueryRunner<SegmentAnalysis>>newArrayList(
                     toolChest.preMergeQueryDecoration(runner1),
                     toolChest.preMergeQueryDecoration(runner2)
@@ -831,7 +831,7 @@ public void testBySegmentResults()
     QueryRunner myRunner = new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             FACTORY.mergeRunners(
-                MoreExecutors.sameThreadExecutor(),
+                Execs.sameThreadExecutor(),
                 //Note: It is essential to have atleast 2 query runners merged to reproduce the regression bug described in
                 //https://github.com/druid-io/druid/pull/1172
                 //the bug surfaces only when ordering is used which happens only when you have 2 things to compare
diff --git a/processing/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java b/processing/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java
index 0a2d67fe1bc..4d710e8f6f5 100644
--- a/processing/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java
+++ b/processing/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java
@@ -22,8 +22,8 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.io.CharSource;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.java.util.common.DateTimes;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.common.guava.MergeSequence;
 import io.druid.java.util.common.guava.Sequence;
@@ -166,7 +166,10 @@ public static void clear()
   @Parameterized.Parameters(name = "limit={0},batchSize={1}")
   public static Iterable<Object[]> 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;
@@ -195,8 +198,9 @@ public void testMergeRunnersWithLimit()
     ScanQuery query = newBuilder().build();
     List<ScanResultValue> results = factory
         .mergeRunners(
-            MoreExecutors.sameThreadExecutor(),
-            ImmutableList.of(factory.createRunner(segment0), factory.createRunner(segment1)))
+            Execs.sameThreadExecutor(),
+            ImmutableList.of(factory.createRunner(segment0), factory.createRunner(segment1))
+        )
         .run(QueryPlus.wrap(query), new HashMap<>())
         .toList();
     int totalCount = 0;
@@ -214,7 +218,8 @@ public void testMergeRunnersWithLimit()
   public void testMergeResultsWithLimit()
   {
     QueryRunner<ScanResultValue> runner = toolChest.mergeResults(
-        new QueryRunner<ScanResultValue>() {
+        new QueryRunner<ScanResultValue>()
+        {
           @Override
           public Sequence<ScanResultValue> run(
               QueryPlus<ScanResultValue> queryPlus, Map<String, Object> responseContext
diff --git a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java
index 604e3bc60ba..1499e99aced 100644
--- a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java
+++ b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java
@@ -23,10 +23,10 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.jackson.DefaultObjectMapper;
 import io.druid.java.util.common.DateTimes;
 import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.common.guava.Accumulator;
 import io.druid.java.util.common.guava.Sequence;
@@ -118,6 +118,7 @@ public Object accumulate(Object initValue, Accumulator accumulator)
         null, new YieldingAccumulator()
         {
           final List lists = Lists.newArrayList();
+
           @Override
           public Object accumulate(Object accumulated, Object in)
           {
@@ -164,7 +165,7 @@ public void run()
                     throw new SegmentMissingException("FAILSAUCE");
                   }
                 },
-                MoreExecutors.sameThreadExecutor()
+                Execs.sameThreadExecutor()
             );
           }
         },
diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java
index 1e2e441949d..26ae90dacb1 100644
--- a/server/src/main/java/io/druid/client/CachingClusteredClient.java
+++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java
@@ -619,7 +619,7 @@ private void addSequencesFromServer(
                     toolChest.makePreComputeManipulatorFn(downstreamQuery, MetricManipulatorFns.deserializing())::apply
                 );
             if (cachePopulator != null) {
-              res = res.withEffect(cachePopulator::populate, MoreExecutors.sameThreadExecutor());
+              res = res.withEffect(cachePopulator::populate, Execs.sameThreadExecutor());
             }
             return res;
           })
diff --git a/server/src/main/java/io/druid/guice/DruidProcessingModule.java b/server/src/main/java/io/druid/guice/DruidProcessingModule.java
index 5d8f369652d..a71ca8c8d53 100644
--- a/server/src/main/java/io/druid/guice/DruidProcessingModule.java
+++ b/server/src/main/java/io/druid/guice/DruidProcessingModule.java
@@ -19,7 +19,6 @@
 
 package io.druid.guice;
 
-import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.inject.Binder;
 import com.google.inject.Module;
@@ -36,6 +35,7 @@
 import io.druid.guice.annotations.Merging;
 import io.druid.guice.annotations.Processing;
 import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.concurrent.ExecutorServiceConfig;
 import io.druid.java.util.common.lifecycle.Lifecycle;
 import io.druid.java.util.common.logger.Logger;
@@ -80,7 +80,7 @@ public ExecutorService getBackgroundExecutorService(
               .build()
       );
     } else {
-      return MoreExecutors.sameThreadExecutor();
+      return Execs.sameThreadExecutor();
     }
   }
 
diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java
index 989f50012b9..7a4d28c37c5 100644
--- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java
+++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java
@@ -27,7 +27,6 @@
 import com.google.common.base.Throwables;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.MoreExecutors;
 import com.google.inject.Inject;
 import io.druid.data.input.Committer;
 import io.druid.data.input.Firehose;
@@ -175,7 +174,7 @@ public FireDepartmentMetrics getMetrics(String datasource)
 
     return partitionChiefs == null ? new NoopQueryRunner<T>() : factory.getToolchest().mergeResults(
         factory.mergeRunners(
-            MoreExecutors.sameThreadExecutor(),
+            Execs.sameThreadExecutor(),
             // Chaining query runners which wait on submitted chain query runners can make executor pools deadlock
             Iterables.transform(
                 partitionChiefs.values(), new Function<FireChief, QueryRunner<T>>()
@@ -202,7 +201,7 @@ public FireDepartmentMetrics getMetrics(String datasource)
            ? new NoopQueryRunner<T>()
            : factory.getToolchest().mergeResults(
                factory.mergeRunners(
-                   MoreExecutors.sameThreadExecutor(),
+                   Execs.sameThreadExecutor(),
                    Iterables.transform(
                        specs,
                        new Function<SegmentDescriptor, QueryRunner<T>>()
diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java
index 9d758553e71..7ee3c2df6cd 100644
--- a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java
+++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java
@@ -23,16 +23,16 @@
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterables;
-import com.google.common.util.concurrent.MoreExecutors;
-import io.druid.java.util.emitter.EmittingLogger;
-import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.client.CachingQueryRunner;
 import io.druid.client.cache.Cache;
 import io.druid.client.cache.CacheConfig;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.Pair;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.guava.CloseQuietly;
 import io.druid.java.util.common.guava.FunctionalIterable;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.query.BySegmentQueryRunner;
 import io.druid.query.CPUTimeMetricQueryRunner;
 import io.druid.query.MetricsEmittingQueryRunner;
@@ -201,7 +201,7 @@ public SegmentDescriptor apply(final PartitionChunk<Sink> chunk)
                                         sinkSegmentIdentifier,
                                         descriptor.getInterval().getStart(),
                                         factory.mergeRunners(
-                                            MoreExecutors.sameThreadExecutor(),
+                                            Execs.sameThreadExecutor(),
                                             Iterables.transform(
                                                 theSink,
                                                 new Function<FireHydrant, QueryRunner<T>>()
@@ -235,7 +235,7 @@ public SegmentDescriptor apply(final PartitionChunk<Sink> chunk)
                                                             cache,
                                                             toolChest,
                                                             baseRunner,
-                                                            MoreExecutors.sameThreadExecutor(),
+                                                            Execs.sameThreadExecutor(),
                                                             cacheConfig
                                                         );
                                                       } else {
diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java
index 3f7fdde3291..d53524bbc12 100644
--- a/server/src/main/java/io/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java
+++ b/server/src/main/java/io/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java
@@ -27,11 +27,11 @@
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.SettableFuture;
 import io.druid.data.input.Committer;
 import io.druid.data.input.InputRow;
 import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.concurrent.ListenableFutures;
 import io.druid.java.util.common.logger.Logger;
 import io.druid.query.SegmentDescriptor;
@@ -54,11 +54,11 @@
 
 /**
  * This class is specialized for streaming ingestion. In streaming ingestion, the segment lifecycle is like:
- *
+ * <p>
  * <pre>
  * APPENDING -> APPEND_FINISHED -> PUBLISHED
  * </pre>
- *
+ * <p>
  * <ul>
  * <li>APPENDING: Segment is available for appending.</li>
  * <li>APPEND_FINISHED: Segment cannot be updated (data cannot be added anymore) and is waiting for being published.</li>
@@ -330,7 +330,7 @@ public Object persist(final Committer committer) throws InterruptedException
                 segmentIdentifier.getVersion(),
                 segmentIdentifier.getShardSpec().getPartitionNum()
             ),
-            MoreExecutors.sameThreadExecutor(),
+            Execs.sameThreadExecutor(),
             () -> {
               log.info("Segment[%s] successfully handed off, dropping.", segmentIdentifier);
               metrics.incrementHandOffCount();
diff --git a/server/src/main/java/io/druid/server/QueryManager.java b/server/src/main/java/io/druid/server/QueryManager.java
index 791e2cf2ecd..b3a945df650 100644
--- a/server/src/main/java/io/druid/server/QueryManager.java
+++ b/server/src/main/java/io/druid/server/QueryManager.java
@@ -23,7 +23,7 @@
 import com.google.common.collect.Multimaps;
 import com.google.common.collect.SetMultimap;
 import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.MoreExecutors;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.query.Query;
 import io.druid.query.QueryWatcher;
 
@@ -76,7 +76,7 @@ public void run()
             }
           }
         },
-        MoreExecutors.sameThreadExecutor()
+        Execs.sameThreadExecutor()
     );
   }
 
diff --git a/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java b/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java
index 9bb94593f43..8b516215ee3 100644
--- a/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java
+++ b/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java
@@ -20,7 +20,6 @@
 package io.druid.server.coordination.broker;
 
 import com.google.common.base.Predicates;
-import com.google.common.util.concurrent.MoreExecutors;
 import com.google.inject.Inject;
 import io.druid.client.FilteredServerInventoryView;
 import io.druid.client.ServerView;
@@ -28,6 +27,7 @@
 import io.druid.guice.ManageLifecycle;
 import io.druid.guice.annotations.Self;
 import io.druid.java.util.common.Pair;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.lifecycle.LifecycleStart;
 import io.druid.java.util.common.lifecycle.LifecycleStop;
 import io.druid.server.DruidNode;
@@ -53,7 +53,7 @@ public DruidBroker(
     this.serviceAnnouncer = serviceAnnouncer;
 
     serverInventoryView.registerSegmentCallback(
-        MoreExecutors.sameThreadExecutor(),
+        Execs.sameThreadExecutor(),
         new ServerView.BaseSegmentCallback()
         {
           @Override
diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java
index 390bde55996..2f39d381a67 100644
--- a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java
+++ b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java
@@ -22,7 +22,6 @@
 import com.google.common.collect.Lists;
 import com.google.common.collect.Ordering;
 import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.client.cache.Cache;
 import io.druid.client.cache.CacheConfig;
 import io.druid.client.cache.MapCache;
@@ -30,6 +29,7 @@
 import io.druid.client.selector.ServerSelector;
 import io.druid.client.selector.TierSelectorStrategy;
 import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.guava.Sequence;
 import io.druid.query.DataSource;
 import io.druid.query.Druids;
@@ -76,7 +76,7 @@ public void setUp()
     timeline = new VersionedIntervalTimeline<>(Ordering.<String>natural());
     serverView = EasyMock.createNiceMock(TimelineServerView.class);
     cache = MapCache.create(100000);
-    client = makeClient(MoreExecutors.sameThreadExecutor());
+    client = makeClient(Execs.sameThreadExecutor());
   }
 
   @Test
diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java
index 25901cfeffb..2f7ef3fae55 100644
--- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java
+++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java
@@ -60,6 +60,7 @@
 import io.druid.java.util.common.Intervals;
 import io.druid.java.util.common.Pair;
 import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.common.granularity.Granularity;
 import io.druid.java.util.common.granularity.PeriodGranularity;
@@ -258,42 +259,43 @@
   private static final Granularity PT1H_TZ_GRANULARITY = new PeriodGranularity(new Period("PT1H"), null, TIMEZONE);
   private static final String TOP_DIM = "a_dim";
 
-  private static final Supplier<SelectQueryConfig> selectConfigSupplier = Suppliers.ofInstance(new SelectQueryConfig(true));
+  private static final Supplier<SelectQueryConfig> selectConfigSupplier = Suppliers.ofInstance(new SelectQueryConfig(
+      true));
 
   static final QueryToolChestWarehouse WAREHOUSE = new MapQueryToolChestWarehouse(
       ImmutableMap.<Class<? extends Query>, QueryToolChest>builder()
-                  .put(
-                      TimeseriesQuery.class,
-                      new TimeseriesQueryQueryToolChest(
-                          QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
-                      )
-                  )
-                  .put(
-                      TopNQuery.class, new TopNQueryQueryToolChest(
-                          new TopNQueryConfig(),
-                          QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
-                      )
-                  )
-                  .put(
-                      SearchQuery.class, new SearchQueryQueryToolChest(
-                          new SearchQueryConfig(),
-                          QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
-                      )
-                  )
-                  .put(
-                      SelectQuery.class,
-                      new SelectQueryQueryToolChest(
-                          jsonMapper,
-                          QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
-                          selectConfigSupplier
-                      )
-                  )
-                  .put(
-                      GroupByQuery.class,
-                      GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig()).getToolchest()
-                  )
-                  .put(TimeBoundaryQuery.class, new TimeBoundaryQueryQueryToolChest())
-                  .build()
+          .put(
+              TimeseriesQuery.class,
+              new TimeseriesQueryQueryToolChest(
+                  QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+              )
+          )
+          .put(
+              TopNQuery.class, new TopNQueryQueryToolChest(
+                  new TopNQueryConfig(),
+                  QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+              )
+          )
+          .put(
+              SearchQuery.class, new SearchQueryQueryToolChest(
+                  new SearchQueryConfig(),
+                  QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+              )
+          )
+          .put(
+              SelectQuery.class,
+              new SelectQueryQueryToolChest(
+                  jsonMapper,
+                  QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
+                  selectConfigSupplier
+              )
+          )
+          .put(
+              GroupByQuery.class,
+              GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig()).getToolchest()
+          )
+          .put(TimeBoundaryQuery.class, new TimeBoundaryQueryQueryToolChest())
+          .build()
   );
   private final Random random;
   public CachingClusteredClient client;
@@ -331,7 +333,7 @@ public void setUp()
     timeline = new VersionedIntervalTimeline<>(Ordering.<String>natural());
     serverView = EasyMock.createNiceMock(TimelineServerView.class);
     cache = MapCache.create(100000);
-    client = makeClient(MoreExecutors.sameThreadExecutor());
+    client = makeClient(Execs.sameThreadExecutor());
 
     servers = new DruidServer[]{
         new DruidServer("test1", "test1", null, 10, ServerType.HISTORICAL, "bye", 0),
@@ -359,7 +361,7 @@ public void testOutOfOrderBackgroundCachePopulation()
       final ListeningExecutorService delegate = MoreExecutors.listeningDecorator(
           // we need to run everything in the same thread to ensure all callbacks on futures in CachingClusteredClient
           // are complete before moving on to the next query run.
-          MoreExecutors.sameThreadExecutor()
+          Execs.sameThreadExecutor()
       );
 
       @Override
@@ -580,7 +582,7 @@ public void testCachingOverBulkLimitEnforcesLimit()
             .andReturn(ImmutableMap.<Cache.NamedKey, byte[]>of())
             .once();
     EasyMock.replay(cache);
-    client = makeClient(MoreExecutors.sameThreadExecutor(), cache, limit);
+    client = makeClient(Execs.sameThreadExecutor(), cache, limit);
     final DruidServer lastServer = servers[random.nextInt(servers.length)];
     final DataSegment dataSegment = EasyMock.createNiceMock(DataSegment.class);
     EasyMock.expect(dataSegment.getIdentifier()).andReturn(DATA_SOURCE).anyTimes();
@@ -605,7 +607,7 @@ public void testCachingOverBulkLimitEnforcesLimit()
             .andReturn(ImmutableMap.<Cache.NamedKey, byte[]>of())
             .once();
     EasyMock.replay(cache);
-    client = makeClient(MoreExecutors.sameThreadExecutor(), cache, 0);
+    client = makeClient(Execs.sameThreadExecutor(), cache, 0);
     getDefaultQueryRunner().run(QueryPlus.wrap(query), context);
     EasyMock.verify(cache);
     EasyMock.verify(dataSegment);
@@ -1194,14 +1196,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(Arrays.asList(TOP_DIM))
-        .query("how")
-        .context(CONTEXT);
+                                                    .dataSource(DATA_SOURCE)
+                                                    .filters(DIM_FILTER)
+                                                    .granularity(GRANULARITY)
+                                                    .limit(1000)
+                                                    .intervals(SEG_SPEC)
+                                                    .dimensions(Arrays.asList(TOP_DIM))
+                                                    .query("how")
+                                                    .context(CONTEXT);
 
     testQueryCaching(
         getDefaultQueryRunner(),
@@ -1309,18 +1311,18 @@ public void testSelectCaching()
 
         Intervals.of("2011-01-05/2011-01-10"),
         makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-05"),
-            DateTimes.of("2011-01-06"),
-            DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"),
-            DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8),
-            DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9)
+                          DateTimes.of("2011-01-06"),
+                          DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"),
+                          DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8),
+                          DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9)
         ),
 
         Intervals.of("2011-01-05/2011-01-10"),
         makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5),
-            DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6),
-            DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7),
-            DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8),
-            DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9)
+                          DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6),
+                          DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7),
+                          DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8),
+                          DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9)
         )
     );
 
@@ -1335,17 +1337,17 @@ public void testSelectCaching()
     HashMap<String, Object> context = new HashMap<String, Object>();
     TestHelper.assertExpectedResults(
         makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-01"), ImmutableMap.of("a", "b", "rows", 1),
-            DateTimes.of("2011-01-02"), ImmutableMap.of("a", "c", "rows", 5),
-            DateTimes.of("2011-01-05"),
-            DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5),
-            DateTimes.of("2011-01-06"),
-            DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6),
-            DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"),
-            DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7),
-            DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8),
-            DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8),
-            DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9),
-            DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9)
+                          DateTimes.of("2011-01-02"), ImmutableMap.of("a", "c", "rows", 5),
+                          DateTimes.of("2011-01-05"),
+                          DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5),
+                          DateTimes.of("2011-01-06"),
+                          DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6),
+                          DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"),
+                          DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7),
+                          DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8),
+                          DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8),
+                          DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9),
+                          DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9)
         ),
         runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context)
     );
@@ -1358,14 +1360,14 @@ public void testSelectCachingRenamedOutputName()
     final Set<String> metrics = Sets.<String>newHashSet("rows");
 
     Druids.SelectQueryBuilder builder = Druids.newSelectQueryBuilder()
-        .dataSource(DATA_SOURCE)
-        .intervals(SEG_SPEC)
-        .filters(DIM_FILTER)
-        .granularity(GRANULARITY)
-        .dimensions(Arrays.asList("a"))
-        .metrics(Arrays.asList("rows"))
-        .pagingSpec(new PagingSpec(null, 3))
-        .context(CONTEXT);
+                                              .dataSource(DATA_SOURCE)
+                                              .intervals(SEG_SPEC)
+                                              .filters(DIM_FILTER)
+                                              .granularity(GRANULARITY)
+                                              .dimensions(Arrays.asList("a"))
+                                              .metrics(Arrays.asList("rows"))
+                                              .pagingSpec(new PagingSpec(null, 3))
+                                              .context(CONTEXT);
 
     testQueryCaching(
         getDefaultQueryRunner(),
@@ -1453,9 +1455,9 @@ public void testSelectCachingRenamedOutputName()
   public void testGroupByCaching()
   {
     List<AggregatorFactory> aggsWithUniques = ImmutableList.<AggregatorFactory>builder()
-                                                           .addAll(AGGS)
-                                                           .add(new HyperUniquesAggregatorFactory("uniques", "uniques"))
-                                                           .build();
+        .addAll(AGGS)
+        .add(new HyperUniquesAggregatorFactory("uniques", "uniques"))
+        .build();
 
     final HashFunction hashFn = Hashing.murmur3_128();
 
@@ -1656,9 +1658,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(
@@ -1696,14 +1700,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<String, Object> context = new HashMap<>();
@@ -1759,7 +1763,8 @@ public void testSingleDimensionPruning()
   }
 
   private ServerSelector makeMockSingleDimensionSelector(
-      DruidServer server, String dimension, String start, String end, int partitionNum)
+      DruidServer server, String dimension, String start, String end, int partitionNum
+  )
   {
     DataSegment segment = EasyMock.createNiceMock(DataSegment.class);
     EasyMock.expect(segment.getIdentifier()).andReturn(DATA_SOURCE).anyTimes();
@@ -1889,7 +1894,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);
@@ -1947,7 +1957,11 @@ public void run()
     MultipleSpecificSegmentSpec spec = (MultipleSpecificSegmentSpec) query.getQuerySegmentSpec();
     List<Result<TimeseriesResultValue>> ret = Lists.newArrayList();
     for (SegmentDescriptor descriptor : spec.getDescriptors()) {
-      String id = StringUtils.format("%s_%s", queryIntervals.indexOf(descriptor.getInterval()), descriptor.getPartitionNumber());
+      String id = StringUtils.format(
+          "%s_%s",
+          queryIntervals.indexOf(descriptor.getInterval()),
+          descriptor.getPartitionNumber()
+      );
       int index = segmentIds.indexOf(id);
       if (index != -1) {
         ret.add(new Result(
@@ -2459,13 +2473,13 @@ public Result apply(
               (DateTime) objects[i],
               new TimeseriesResultValue(
                   ImmutableMap.<String, Object>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()
               )
           )
       );
@@ -2582,7 +2596,11 @@ public Result apply(
     return retVal;
   }
 
-  private Iterable<Result<SelectResultValue>> makeSelectResults(Set<String> dimensions, Set<String> metrics, Object... objects)
+  private Iterable<Result<SelectResultValue>> makeSelectResults(
+      Set<String> dimensions,
+      Set<String> metrics,
+      Object... objects
+  )
   {
     List<Result<SelectResultValue>> retVal = Lists.newArrayList();
     int index = 0;
@@ -2598,7 +2616,8 @@ public Result apply(
       retVal.add(new Result<>(
           timestamp,
           new SelectResultValue(ImmutableMap.of(timestamp.toString(), 0),
-                                dimensions, metrics, values)
+                                dimensions, metrics, values
+          )
       ));
     }
     return retVal;
@@ -3087,7 +3106,10 @@ public void testIfNoneMatch()
     TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder()
                                     .dataSource(DATA_SOURCE)
                                     .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(interval)))
-                                    .context(ImmutableMap.<String, Object>of("If-None-Match", "aVJV29CJY93rszVW/QBy0arWZo0="))
+                                    .context(ImmutableMap.<String, Object>of(
+                                        "If-None-Match",
+                                        "aVJV29CJY93rszVW/QBy0arWZo0="
+                                    ))
                                     .build();
 
 
@@ -3100,7 +3122,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/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java
index 4ee3013eb7d..ea782ae6c6e 100644
--- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java
+++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java
@@ -24,7 +24,6 @@
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.client.cache.Cache;
 import io.druid.client.cache.CacheConfig;
 import io.druid.client.cache.CacheStats;
@@ -33,6 +32,7 @@
 import io.druid.java.util.common.DateTimes;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.common.guava.Sequence;
 import io.druid.java.util.common.guava.SequenceWrapper;
@@ -105,7 +105,7 @@ public CachingQueryRunnerTest(int numBackgroundThreads)
     if (numBackgroundThreads > 0) {
       backgroundExecutorService = Executors.newFixedThreadPool(numBackgroundThreads);
     } else {
-      backgroundExecutorService = MoreExecutors.sameThreadExecutor();
+      backgroundExecutorService = Execs.sameThreadExecutor();
     }
   }
 
diff --git a/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java b/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java
index 7f7f8883518..717361f68e5 100644
--- a/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java
+++ b/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java
@@ -24,23 +24,23 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.MoreExecutors;
-import io.druid.java.util.http.client.HttpClient;
-import io.druid.java.util.http.client.Request;
-import io.druid.java.util.http.client.response.HttpResponseHandler;
 import io.druid.discovery.DataNodeService;
 import io.druid.discovery.DiscoveryDruidNode;
 import io.druid.discovery.DruidNodeDiscovery;
 import io.druid.discovery.DruidNodeDiscoveryProvider;
 import io.druid.java.util.common.Intervals;
 import io.druid.java.util.common.RE;
+import io.druid.java.util.common.concurrent.Execs;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.HttpResponseHandler;
 import io.druid.segment.TestHelper;
 import io.druid.server.DruidNode;
+import io.druid.server.coordination.ChangeRequestHistory;
+import io.druid.server.coordination.ChangeRequestsSnapshot;
 import io.druid.server.coordination.DruidServerMetadata;
 import io.druid.server.coordination.SegmentChangeRequestDrop;
-import io.druid.server.coordination.ChangeRequestHistory;
 import io.druid.server.coordination.SegmentChangeRequestLoad;
-import io.druid.server.coordination.ChangeRequestsSnapshot;
 import io.druid.server.coordination.ServerType;
 import io.druid.timeline.DataSegment;
 import org.easymock.EasyMock;
@@ -197,7 +197,7 @@ public void testSimple() throws Exception
     );
 
     httpServerInventoryView.registerSegmentCallback(
-        MoreExecutors.sameThreadExecutor(),
+        Execs.sameThreadExecutor(),
         new ServerView.SegmentCallback()
         {
           @Override
@@ -229,7 +229,7 @@ public void testSimple() throws Exception
 
     final CountDownLatch serverRemovedCalled = new CountDownLatch(1);
     httpServerInventoryView.registerServerRemovedCallback(
-        MoreExecutors.sameThreadExecutor(),
+        Execs.sameThreadExecutor(),
         new ServerView.ServerRemovedCallback()
         {
           @Override
@@ -258,8 +258,10 @@ public void testSimple() throws Exception
     segmentDropLatches.get(segment2.getIdentifier()).await();
 
     DruidServer druidServer = httpServerInventoryView.getInventoryValue("host:8080");
-    Assert.assertEquals(ImmutableMap.of(segment3.getIdentifier(), segment3, segment4.getIdentifier(), segment4),
-                        druidServer.getSegments());
+    Assert.assertEquals(
+        ImmutableMap.of(segment3.getIdentifier(), segment3, segment4.getIdentifier(), segment4),
+        druidServer.getSegments()
+    );
 
     druidNodeDiscovery.listener.nodesRemoved(ImmutableList.of(druidNode));
 
@@ -320,7 +322,10 @@ public void registerListener(Listener listener)
 
       if (requestNum.get() == 2) {
         //fail scenario where request is sent to server but we got an unexpected response.
-        HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR);
+        HttpResponse httpResponse = new DefaultHttpResponse(
+            HttpVersion.HTTP_1_1,
+            HttpResponseStatus.INTERNAL_SERVER_ERROR
+        );
         httpResponse.setContent(ChannelBuffers.buffer(0));
         httpResponseHandler.handleResponse(httpResponse);
         return Futures.immediateFailedFuture(new RuntimeException("server error"));
diff --git a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java
index 5119927b852..feb16277d81 100644
--- a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java
+++ b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java
@@ -38,6 +38,7 @@
 import io.druid.java.util.common.DateTimes;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.Pair;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.guava.Comparators;
 import io.druid.segment.TestHelper;
 import io.druid.server.coordination.BatchDataSegmentAnnouncer;
@@ -118,7 +119,7 @@ public void setUp() throws Exception
 
     announcer = new Announcer(
         cf,
-        MoreExecutors.sameThreadExecutor()
+        Execs.sameThreadExecutor()
     );
     announcer.start();
 
@@ -204,7 +205,8 @@ public boolean apply(@Nullable Pair<DruidServerMetadata, DataSegment> input)
             return input.rhs.getInterval().getStart().isBefore(SEGMENT_INTERVAL_START.plusDays(INITIAL_SEGMENTS));
           }
         }
-    ) {
+    )
+    {
       @Override
       protected DruidServer addInnerInventory(
           DruidServer container, String inventoryKey, Set<DataSegment> inventory
@@ -339,7 +341,7 @@ public void testRunWithFilterCallback() throws Exception
     EasyMock.replay(callback);
 
     filteredBatchServerInventoryView.registerSegmentCallback(
-        MoreExecutors.sameThreadExecutor(),
+        Execs.sameThreadExecutor(),
         callback,
         new Predicate<Pair<DruidServerMetadata, DataSegment>>()
         {
@@ -409,7 +411,11 @@ private void waitForUpdateEvents(int count)
     while (inventoryUpdateCounter.get() != count) {
       Thread.sleep(100);
       if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > forWaitingTiming.milliseconds()) {
-        throw new ISE("BatchServerInventoryView is not updating counter expected[%d] value[%d]", count, inventoryUpdateCounter.get());
+        throw new ISE(
+            "BatchServerInventoryView is not updating counter expected[%d] value[%d]",
+            count,
+            inventoryUpdateCounter.get()
+        );
       }
     }
   }
diff --git a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java
index d3ec923a43b..587710886c9 100644
--- a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java
+++ b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java
@@ -23,13 +23,13 @@
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.curator.CuratorTestBase;
 import io.druid.curator.announcement.Announcer;
 import io.druid.discovery.DiscoveryDruidNode;
 import io.druid.discovery.DruidNodeDiscovery;
 import io.druid.discovery.DruidNodeDiscoveryProvider;
 import io.druid.jackson.DefaultObjectMapper;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.server.DruidNode;
 import io.druid.server.initialization.ServerConfig;
 import io.druid.server.initialization.ZkPathsConfig;
@@ -69,7 +69,7 @@ public void testAnnouncementAndDiscovery() throws Exception
 
     Announcer announcer = new Announcer(
         curator,
-        MoreExecutors.sameThreadExecutor()
+        Execs.sameThreadExecutor()
     );
     announcer.start();
 
diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java
index bd4b4210255..97de81de6b4 100644
--- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java
+++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java
@@ -28,7 +28,6 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.data.input.Committer;
 import io.druid.data.input.Firehose;
 import io.druid.data.input.FirehoseFactory;
@@ -41,6 +40,7 @@
 import io.druid.java.util.common.DateTimes;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.common.parsers.ParseException;
 import io.druid.query.BaseQuery;
@@ -1086,7 +1086,7 @@ public Sink getSink(long timestamp)
         return factory.getToolchest()
                       .mergeResults(
                           factory.mergeRunners(
-                              MoreExecutors.sameThreadExecutor(),
+                              Execs.sameThreadExecutor(),
                               Iterables.transform(
                                   baseQuery.getIntervals(),
                                   new Function<Interval, QueryRunner<T>>()
diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java
index c70f65c73f2..55d4bc5ae02 100644
--- a/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java
+++ b/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java
@@ -21,10 +21,10 @@
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.client.ImmutableSegmentLoadInfo;
 import io.druid.client.coordinator.CoordinatorClient;
 import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.query.SegmentDescriptor;
 import io.druid.server.coordination.DruidServerMetadata;
 import io.druid.server.coordination.ServerType;
@@ -90,7 +90,7 @@ public void testHandoffCallbackNotCalled()
     );
     final AtomicBoolean callbackCalled = new AtomicBoolean(false);
     notifier.registerSegmentHandoffCallback(
-        descriptor, MoreExecutors.sameThreadExecutor(), new Runnable()
+        descriptor, Execs.sameThreadExecutor(), new Runnable()
         {
           @Override
           public void run()
@@ -147,7 +147,7 @@ public void testHandoffCallbackCalled()
     );
 
     notifier.registerSegmentHandoffCallback(
-        descriptor, MoreExecutors.sameThreadExecutor(), new Runnable()
+        descriptor, Execs.sameThreadExecutor(), new Runnable()
         {
           @Override
           public void run()
diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java
index 32900d8900f..b65372a60e3 100644
--- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java
+++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java
@@ -25,7 +25,6 @@
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.io.Files;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.client.cache.MapCache;
 import io.druid.data.input.Committer;
 import io.druid.data.input.InputRow;
@@ -37,6 +36,7 @@
 import io.druid.jackson.DefaultObjectMapper;
 import io.druid.java.util.common.DateTimes;
 import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.query.DefaultQueryRunnerFactoryConglomerate;
@@ -118,7 +118,10 @@
   private FireDepartmentMetrics metrics;
   private File tmpDir;
 
-  public RealtimePlumberSchoolTest(RejectionPolicyFactory rejectionPolicy, SegmentWriteOutMediumFactory segmentWriteOutMediumFactory)
+  public RealtimePlumberSchoolTest(
+      RejectionPolicyFactory rejectionPolicy,
+      SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
+  )
   {
     this.rejectionPolicy = rejectionPolicy;
     this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
@@ -220,7 +223,7 @@ public void setUp() throws Exception
         announcer,
         segmentPublisher,
         handoffNotifierFactory,
-        MoreExecutors.sameThreadExecutor(),
+        Execs.sameThreadExecutor(),
         TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory),
         TestHelper.getTestIndexIO(segmentWriteOutMediumFactory),
         MapCache.create(0),
diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java
index 42a2a896a31..d8c21a5c1a1 100644
--- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java
+++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java
@@ -26,7 +26,6 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.client.cache.CacheConfig;
 import io.druid.client.cache.LocalCacheProvider;
 import io.druid.jackson.DefaultObjectMapper;
@@ -34,6 +33,7 @@
 import io.druid.java.util.common.Intervals;
 import io.druid.java.util.common.MapUtils;
 import io.druid.java.util.common.Pair;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.common.granularity.Granularity;
 import io.druid.java.util.common.guava.Sequence;
@@ -152,7 +152,7 @@ public void cleanup(DataSegment segment)
         },
         new NoopServiceEmitter(),
         serverManagerExec,
-        MoreExecutors.sameThreadExecutor(),
+        Execs.sameThreadExecutor(),
         new DefaultObjectMapper(),
         new LocalCacheProvider().get(),
         new CacheConfig(),
diff --git a/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java
index 6d205cfe486..956d557852d 100644
--- a/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java
+++ b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java
@@ -27,16 +27,16 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.curator.PotentiallyGzippedCompressionProvider;
 import io.druid.curator.announcement.Announcer;
 import io.druid.java.util.common.DateTimes;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.segment.TestHelper;
 import io.druid.server.coordination.BatchDataSegmentAnnouncer;
-import io.druid.server.coordination.DataSegmentChangeRequest;
-import io.druid.server.coordination.DruidServerMetadata;
 import io.druid.server.coordination.ChangeRequestHistory;
 import io.druid.server.coordination.ChangeRequestsSnapshot;
+import io.druid.server.coordination.DataSegmentChangeRequest;
+import io.druid.server.coordination.DruidServerMetadata;
 import io.druid.server.coordination.ServerType;
 import io.druid.server.initialization.BatchDataSegmentAnnouncerConfig;
 import io.druid.server.initialization.ZkPathsConfig;
@@ -96,7 +96,7 @@ public void setUp() throws Exception
 
     announcer = new Announcer(
         cf,
-        MoreExecutors.sameThreadExecutor()
+        Execs.sameThreadExecutor()
     );
     announcer.start();
 
diff --git a/services/src/main/java/io/druid/cli/DumpSegment.java b/services/src/main/java/io/druid/cli/DumpSegment.java
index 3d82fbf8d3c..e5956a746b3 100644
--- a/services/src/main/java/io/druid/cli/DumpSegment.java
+++ b/services/src/main/java/io/druid/cli/DumpSegment.java
@@ -28,7 +28,6 @@
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.MoreExecutors;
 import com.google.inject.Binder;
 import com.google.inject.Injector;
 import com.google.inject.Key;
@@ -48,6 +47,7 @@
 import io.druid.java.util.common.IAE;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.common.guava.Accumulator;
 import io.druid.java.util.common.guava.Sequence;
@@ -484,7 +484,7 @@ public int columnCacheSizeBytes()
     final QueryRunnerFactory factory = conglomerate.findFactory(query);
     final QueryRunner<T> runner = factory.createRunner(new QueryableIndexSegment("segment", index));
     final Sequence results = factory.getToolchest().mergeResults(
-        factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.<QueryRunner>of(runner))
+        factory.mergeRunners(Execs.sameThreadExecutor(), ImmutableList.<QueryRunner>of(runner))
     ).run(QueryPlus.wrap(query), Maps.<String, Object>newHashMap());
     return (Sequence<T>) results;
   }
diff --git a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java
index b946afea777..8253d1de91a 100644
--- a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java
+++ b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java
@@ -26,13 +26,13 @@
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.MoreExecutors;
 import com.google.inject.Inject;
 import io.druid.client.ServerView;
 import io.druid.client.TimelineServerView;
 import io.druid.guice.ManageLifecycle;
 import io.druid.java.util.common.DateTimes;
 import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.concurrent.ScheduledExecutors;
 import io.druid.java.util.common.guava.Sequence;
 import io.druid.java.util.common.guava.Yielder;
@@ -142,7 +142,7 @@ public DruidSchema(
     this.escalator = escalator;
 
     serverView.registerTimelineCallback(
-        MoreExecutors.sameThreadExecutor(),
+        Execs.sameThreadExecutor(),
         new TimelineServerView.TimelineCallback()
         {
           @Override
diff --git a/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java
index 5c3f8a97ba1..8b3843a3948 100644
--- a/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java
+++ b/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java
@@ -25,9 +25,9 @@
 import com.google.common.collect.Maps;
 import com.google.common.collect.Ordering;
 import com.google.common.io.Closeables;
-import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.UOE;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.guava.FunctionalIterable;
 import io.druid.java.util.common.guava.Sequence;
 import io.druid.query.FinalizeResultsQueryRunner;
@@ -224,7 +224,7 @@ public void close() throws IOException
     return new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             factory.mergeRunners(
-                MoreExecutors.sameThreadExecutor(),
+                Execs.sameThreadExecutor(),
                 FunctionalIterable
                     .create(specs)
                     .transformCat(


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org