You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by gi...@apache.org on 2018/08/01 03:09:14 UTC

[incubator-druid] branch master updated: Prohibit Lists.newArrayList() with a single argument (#6068)

This is an automated email from the ASF dual-hosted git repository.

gian pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/master by this push:
     new 0754d78  Prohibit Lists.newArrayList() with a single argument (#6068)
0754d78 is described below

commit 0754d78a2edee2f8496a6bde04b841cd3c7dc52a
Author: Roman Leventov <le...@gmail.com>
AuthorDate: Tue Jul 31 22:09:10 2018 -0500

    Prohibit Lists.newArrayList() with a single argument (#6068)
    
    * Prohibit Lists.newArrayList() with a single argument
    
    * Test fixes
    
    * Add Javadoc to Node constructor
---
 .idea/inspectionProfiles/Druid.xml                 |    6 +-
 .../io/druid/data/input/impl/ParseSpecTest.java    |    2 +-
 .../benchmark/GroupByTypeInterfaceBenchmark.java   |   32 +-
 .../io/druid/benchmark/query/GroupByBenchmark.java |   50 +-
 .../io/druid/benchmark/query/SearchBenchmark.java  |    4 +-
 .../io/druid/benchmark/query/SqlBenchmark.java     |   11 +-
 .../collections/OrderedMergeSequenceTest.java      |   22 +-
 .../DistinctCountGroupByQueryTest.java             |   27 +-
 .../MaterializedViewSupervisorTest.java            |   18 +-
 .../materializedview/DataSourceOptimizer.java      |    6 +-
 .../materializedview/DatasourceOptimizerTest.java  |   18 +-
 .../BasicRoleBasedAuthorizerTest.java              |    9 +-
 .../ApproximateHistogramGroupByQueryTest.java      |   58 +-
 .../histogram/sql/QuantileSqlAggregatorTest.java   |   16 +-
 .../io/druid/server/lookup/LoadingLookupTest.java  |    5 +-
 .../lookup/cache/loading/LoadingCacheTest.java     |    3 +-
 .../variance/VarianceGroupByQueryTest.java         |   30 +-
 .../HadoopDruidDetermineConfigurationJob.java      |    3 +-
 .../indexer/HadoopDruidIndexerConfigTest.java      |    5 +-
 .../io/druid/indexer/HadoopIngestionSpecTest.java  |    5 +-
 ...onSpecUpdateDatasourcePathSpecSegmentsTest.java |    6 +-
 .../druid/indexer/IndexGeneratorCombinerTest.java  |    2 +-
 .../indexer/hadoop/DatasourceInputSplitTest.java   |    4 +-
 .../indexer/hadoop/DatasourceRecordReaderTest.java |    2 +-
 .../java/io/druid/indexing/common/task/Tasks.java  |    4 +-
 .../indexing/overlord/http/OverlordResource.java   |    8 +-
 .../autoscaling/EC2AutoScalerSerdeTest.java        |    5 +-
 .../PendingTaskBasedProvisioningStrategyTest.java  |   28 +-
 .../SimpleProvisioningStrategyTest.java            |   24 +-
 .../supervisor/SupervisorResourceTest.java         |    9 +-
 .../java/util/common/guava/MergeSequenceTest.java  |   10 +-
 .../java/io/druid/collections/spatial/Node.java    |   25 +-
 .../java/io/druid/collections/spatial/Point.java   |    7 +-
 .../spatial/split/GutmanSplitStrategy.java         |    7 +-
 .../src/main/java/io/druid/query/Druids.java       |    3 +-
 .../druid/query/IntervalChunkingQueryRunner.java   |    2 +-
 .../aggregation/JavaScriptAggregatorFactory.java   |    5 +-
 .../java/io/druid/query/groupby/GroupByQuery.java  |   14 +
 .../query/groupby/strategy/GroupByStrategyV1.java  |    2 +-
 .../bitmap/ConciseBitmapFactoryTest.java           |    4 +-
 .../bitmap/RoaringBitmapFactoryTest.java           |    4 +-
 .../io/druid/granularity/QueryGranularityTest.java |    5 +-
 .../java/io/druid/query/AsyncQueryRunnerTest.java  |    7 +-
 .../query/ChainedExecutionQueryRunnerTest.java     |    7 +-
 .../test/java/io/druid/query/DataSourceTest.java   |   10 +-
 .../java/io/druid/query/DoubleStorageTest.java     |    3 +-
 .../query/IntervalChunkingQueryRunnerTest.java     |    3 +-
 .../io/druid/query/MultiValuedDimensionTest.java   |   19 +-
 .../java/io/druid/query/TimewarpOperatorTest.java  |    2 +-
 .../query/aggregation/AggregationTestHelper.java   |    5 +-
 .../query/aggregation/AggregatorUtilTest.java      |    5 +-
 .../CardinalityAggregatorBenchmark.java            |    8 +-
 .../cardinality/CardinalityAggregatorTest.java     |   25 +-
 .../hyperloglog/HyperUniquesAggregationTest.java   |    5 +-
 .../FinalizingFieldAccessPostAggregatorTest.java   |    2 +-
 .../io/druid/query/cache/CacheKeyBuilderTest.java  |   15 +-
 .../groupby/DefaultGroupByQueryMetricsTest.java    |   38 +-
 ...GroupByLimitPushDownInsufficientBufferTest.java |   16 +-
 .../GroupByLimitPushDownMultiNodeMergeTest.java    |   34 +-
 .../query/groupby/GroupByMultiSegmentTest.java     |    8 +-
 .../query/groupby/GroupByQueryMergeBufferTest.java |   36 +-
 .../groupby/GroupByQueryRunnerFactoryTest.java     |    6 +-
 .../groupby/GroupByQueryRunnerFailureTest.java     |   25 +-
 .../query/groupby/GroupByQueryRunnerTest.java      | 2433 ++++++--------------
 .../io/druid/query/groupby/GroupByQueryTest.java   |   18 +-
 .../query/groupby/orderby/TopNSequenceTest.java    |    2 +-
 .../io/druid/query/lookup/LookupExtractorTest.java |    3 +-
 .../metadata/SegmentMetadataUnionQueryTest.java    |    6 +-
 .../druid/query/search/SearchQueryRunnerTest.java  |    5 +-
 .../io/druid/query/select/SelectBinaryFnTest.java  |   24 +-
 .../druid/query/select/SelectQueryRunnerTest.java  |   32 +-
 .../timeseries/TimeseriesQueryRunnerTest.java      |   25 +-
 .../io/druid/query/topn/TopNQueryRunnerTest.java   |   25 +-
 .../java/io/druid/query/topn/TopNQueryTest.java    |    3 +-
 .../test/java/io/druid/segment/EmptyIndexTest.java |    4 +-
 .../IncrementalIndexStorageAdapterTest.java        |   36 +-
 .../incremental/OnheapIncrementalIndexTest.java    |    6 +-
 .../java/io/druid/server/ClientInfoResource.java   |    2 +-
 .../io/druid/server/http/InventoryViewUtils.java   |    6 +-
 .../io/druid/server/http/MetadataResource.java     |    4 +-
 .../druid/server/security/AuthorizationUtils.java  |    3 +-
 .../druid/client/CachingClusteredClientTest.java   |   11 +-
 .../io/druid/client/cache/CaffeineCacheTest.java   |    3 +-
 .../client/indexing/ClientAppendQueryTest.java     |    8 +-
 .../client/indexing/ClientMergeQueryTest.java      |    3 +-
 .../initialization/ComposingEmitterModuleTest.java |    4 +-
 .../query/lookup/LookupSnapshotTakerTest.java      |    7 +-
 .../segment/realtime/RealtimeManagerTest.java      |   27 +-
 ...CoordinatorBasedSegmentHandoffNotifierTest.java |   84 +-
 .../DruidCoordinatorRuleRunnerTest.java            |  133 +-
 .../rules/BroadcastDistributionRuleSerdeTest.java  |    3 +-
 .../PreResponseAuthorizationCheckFilterTest.java   |    4 +-
 .../partition/HashBasedNumberedShardSpecTest.java  |    7 +-
 .../cli/MiddleManagerJettyServerInitializer.java   |    6 +-
 .../io/druid/cli/QueryJettyServerInitializer.java  |    6 +-
 .../sql/calcite/schema/InformationSchema.java      |    4 +-
 .../io/druid/sql/calcite/CalciteQueryTest.java     |   16 +-
 97 files changed, 1239 insertions(+), 2513 deletions(-)

diff --git a/.idea/inspectionProfiles/Druid.xml b/.idea/inspectionProfiles/Druid.xml
index 81f67f0..d0029e1 100644
--- a/.idea/inspectionProfiles/Druid.xml
+++ b/.idea/inspectionProfiles/Druid.xml
@@ -112,6 +112,10 @@
         <constraint name="x" within="" contains="" />
         <constraint name="y" nameOfExprType="IndexedInts" exprTypeWithinHierarchy="true" within="" contains="" />
       </searchConfiguration>
+      <searchConfiguration name="Lists.newArrayList() with a single argument. Use Collections.singletonList() instead" created="1532737126203" text="Lists.newArrayList($x$)" recursive="false" caseInsensitive="true" type="JAVA">
+        <constraint name="x" nameOfExprType="java\.lang\.Iterable|java\.util\.Iterator|Object\[\]" expressionTypes="java.lang.Iterable|java.util.Iterator|Object[]" exprTypeWithinHierarchy="true" negateName="true" negateExprType="true" within="" contains="" />
+        <constraint name="__context__" target="true" within="" contains="" />
+      </searchConfiguration>
     </inspection_tool>
     <inspection_tool class="SpellCheckingInspection" enabled="false" level="TYPO" enabled_by_default="false">
       <option name="processCode" value="true" />
@@ -190,4 +194,4 @@
       <option name="ADD_NONJAVA_TO_ENTRIES" value="true" />
     </inspection_tool>
   </profile>
-</component>
+</component>
\ No newline at end of file
diff --git a/api/src/test/java/io/druid/data/input/impl/ParseSpecTest.java b/api/src/test/java/io/druid/data/input/impl/ParseSpecTest.java
index f8e667e..62988de 100644
--- a/api/src/test/java/io/druid/data/input/impl/ParseSpecTest.java
+++ b/api/src/test/java/io/druid/data/input/impl/ParseSpecTest.java
@@ -68,7 +68,7 @@ public class ParseSpecTest
         ),
         new DimensionsSpec(
             DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "B")),
-            Lists.newArrayList("B"),
+            Collections.singletonList("B"),
             Lists.newArrayList()
         ),
         ",",
diff --git a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java
index a3e3865..dd94316 100644
--- a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java
+++ b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java
@@ -24,7 +24,6 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory;
 import com.google.common.base.Supplier;
 import com.google.common.base.Suppliers;
 import com.google.common.base.Throwables;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.io.Files;
 import io.druid.benchmark.datagen.BenchmarkDataGenerator;
@@ -180,10 +179,7 @@ public class GroupByTypeInterfaceBenchmark
       GroupByQuery queryString = GroupByQuery
           .builder()
           .setDataSource("blah")
-          .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(Lists.newArrayList(
-              new DefaultDimensionSpec("dimSequential", null)
-          ))
+          .setQuerySegmentSpec(intervalSpec).setDimensions(new DefaultDimensionSpec("dimSequential", null))
           .setAggregatorSpecs(
               queryAggs
           )
@@ -194,10 +190,8 @@ public class GroupByTypeInterfaceBenchmark
           .builder()
           .setDataSource("blah")
           .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(Lists.newArrayList(
-              new DefaultDimensionSpec("metLongUniform", null),
-              new DefaultDimensionSpec("metFloatNormal", null)
-          ))
+          .setDimensions(new DefaultDimensionSpec("metLongUniform", null),
+                         new DefaultDimensionSpec("metFloatNormal", null))
           .setAggregatorSpecs(
               queryAggs
           )
@@ -207,10 +201,7 @@ public class GroupByTypeInterfaceBenchmark
       GroupByQuery queryLong = GroupByQuery
           .builder()
           .setDataSource("blah")
-          .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(Lists.newArrayList(
-              new DefaultDimensionSpec("metLongUniform", null)
-          ))
+          .setQuerySegmentSpec(intervalSpec).setDimensions(new DefaultDimensionSpec("metLongUniform", null))
           .setAggregatorSpecs(
               queryAggs
           )
@@ -220,10 +211,7 @@ public class GroupByTypeInterfaceBenchmark
       GroupByQuery queryFloat = GroupByQuery
           .builder()
           .setDataSource("blah")
-          .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(Lists.newArrayList(
-              new DefaultDimensionSpec("metFloatNormal", null)
-          ))
+          .setQuerySegmentSpec(intervalSpec).setDimensions(new DefaultDimensionSpec("metFloatNormal", null))
           .setAggregatorSpecs(
               queryAggs
           )
@@ -248,10 +236,7 @@ public class GroupByTypeInterfaceBenchmark
           .builder()
           .setDataSource("blah")
           .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(Lists.newArrayList(
-              new DefaultDimensionSpec("dimSequential", null),
-              new DefaultDimensionSpec("dimZipf", null)
-          ))
+          .setDimensions(new DefaultDimensionSpec("dimSequential", null), new DefaultDimensionSpec("dimZipf", null))
           .setAggregatorSpecs(
               queryAggs
           )
@@ -261,10 +246,7 @@ public class GroupByTypeInterfaceBenchmark
       GroupByQuery queryA = GroupByQuery
           .builder()
           .setDataSource(subqueryA)
-          .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(Lists.newArrayList(
-              new DefaultDimensionSpec("dimSequential", null)
-          ))
+          .setQuerySegmentSpec(intervalSpec).setDimensions(new DefaultDimensionSpec("dimSequential", null))
           .setAggregatorSpecs(
               queryAggs
           )
diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java
index ad98f5b..e27be00 100644
--- a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java
+++ b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java
@@ -189,17 +189,8 @@ public class GroupByBenchmark
           .builder()
           .setDataSource("blah")
           .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(Lists.newArrayList(
-              new DefaultDimensionSpec("dimSequential", null),
-              new DefaultDimensionSpec("dimZipf", null)
-              //new DefaultDimensionSpec("dimUniform", null),
-              //new DefaultDimensionSpec("dimSequentialHalfNull", null)
-              //new DefaultDimensionSpec("dimMultivalEnumerated", null), //multival dims greatly increase the running time, disable for now
-              //new DefaultDimensionSpec("dimMultivalEnumerated2", null)
-          ))
-          .setAggregatorSpecs(
-              queryAggs
-          )
+          .setDimensions(new DefaultDimensionSpec("dimSequential", null), new DefaultDimensionSpec("dimZipf", null))
+          .setAggregatorSpecs(queryAggs)
           .setGranularity(Granularity.fromString(queryGranularity))
           .build();
 
@@ -218,13 +209,8 @@ public class GroupByBenchmark
           .builder()
           .setDataSource("blah")
           .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(Lists.newArrayList(
-              new DefaultDimensionSpec("dimSequential", null),
-              new DefaultDimensionSpec("dimZipf", null)
-          ))
-          .setAggregatorSpecs(
-              queryAggs
-          )
+          .setDimensions(new DefaultDimensionSpec("dimSequential", null), new DefaultDimensionSpec("dimZipf", null))
+          .setAggregatorSpecs(queryAggs)
           .setGranularity(Granularities.DAY)
           .build();
 
@@ -232,12 +218,8 @@ public class GroupByBenchmark
           .builder()
           .setDataSource(subqueryA)
           .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(Lists.newArrayList(
-              new DefaultDimensionSpec("dimSequential", null)
-          ))
-          .setAggregatorSpecs(
-              queryAggs
-          )
+          .setDimensions(new DefaultDimensionSpec("dimSequential", null))
+          .setAggregatorSpecs(queryAggs)
           .setGranularity(Granularities.WEEK)
           .build();
 
@@ -259,7 +241,7 @@ public class GroupByBenchmark
           .builder()
           .setDataSource("blah")
           .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(ImmutableList.of(new DefaultDimensionSpec("dimUniform", null)))
+          .setDimensions(new DefaultDimensionSpec("dimUniform", null))
           .setAggregatorSpecs(queryAggs)
           .setGranularity(Granularity.fromString(queryGranularity))
           .setDimFilter(new BoundDimFilter("dimUniform", "0", "100", true, true, null, null, null))
@@ -283,7 +265,7 @@ public class GroupByBenchmark
           .builder()
           .setDataSource("blah")
           .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(ImmutableList.of(new DefaultDimensionSpec("dimZipf", null)))
+          .setDimensions(new DefaultDimensionSpec("dimZipf", null))
           .setAggregatorSpecs(queryAggs)
           .setGranularity(Granularity.fromString(queryGranularity))
           .build();
@@ -308,9 +290,7 @@ public class GroupByBenchmark
           .builder()
           .setDataSource("blah")
           .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(Lists.newArrayList(
-              new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.STRING)
-          ))
+          .setDimensions(new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.STRING))
           .setAggregatorSpecs(
               queryAggs
           )
@@ -335,9 +315,7 @@ public class GroupByBenchmark
           .builder()
           .setDataSource("blah")
           .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(Lists.newArrayList(
-              new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.LONG)
-          ))
+          .setDimensions(new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.LONG))
           .setAggregatorSpecs(
               queryAggs
           )
@@ -362,12 +340,8 @@ public class GroupByBenchmark
           .builder()
           .setDataSource("blah")
           .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(Lists.newArrayList(
-              new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.FLOAT)
-          ))
-          .setAggregatorSpecs(
-              queryAggs
-          )
+          .setDimensions(new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.FLOAT))
+          .setAggregatorSpecs(queryAggs)
           .setGranularity(Granularity.fromString(queryGranularity))
           .build();
 
diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java
index 7210779..8a9648e 100644
--- a/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java
+++ b/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java
@@ -277,7 +277,7 @@ public class SearchBenchmark
                  .granularity(Granularities.ALL)
                  .intervals(intervalSpec)
                  .query("")
-                 .dimensions(Lists.newArrayList("dimUniform"))
+                 .dimensions(Collections.singletonList("dimUniform"))
                  .filters(new AndDimFilter(dimFilters));
   }
 
@@ -306,7 +306,7 @@ public class SearchBenchmark
                  .granularity(Granularities.ALL)
                  .intervals(intervalSpec)
                  .query("")
-                 .dimensions(Lists.newArrayList("dimUniform"))
+                 .dimensions(Collections.singletonList("dimUniform"))
                  .filters(new AndDimFilter(dimFilters));
   }
 
diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java
index 014d1c8..256c93f 100644
--- a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java
+++ b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java
@@ -62,8 +62,6 @@ import org.openjdk.jmh.annotations.Warmup;
 import org.openjdk.jmh.infra.Blackhole;
 
 import java.io.File;
-import java.util.Arrays;
-import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
@@ -123,13 +121,8 @@ public class SqlBenchmark
         .builder()
         .setDataSource("foo")
         .setInterval(Intervals.ETERNITY)
-        .setDimensions(
-            Arrays.asList(
-                new DefaultDimensionSpec("dimZipf", "d0"),
-                new DefaultDimensionSpec("dimSequential", "d1")
-            )
-        )
-        .setAggregatorSpecs(Collections.singletonList(new CountAggregatorFactory("c")))
+        .setDimensions(new DefaultDimensionSpec("dimZipf", "d0"), new DefaultDimensionSpec("dimSequential", "d1"))
+        .setAggregatorSpecs(new CountAggregatorFactory("c"))
         .setGranularity(Granularities.ALL)
         .build();
 
diff --git a/common/src/test/java/io/druid/collections/OrderedMergeSequenceTest.java b/common/src/test/java/io/druid/collections/OrderedMergeSequenceTest.java
index da97d02..2fb3b5b 100644
--- a/common/src/test/java/io/druid/collections/OrderedMergeSequenceTest.java
+++ b/common/src/test/java/io/druid/collections/OrderedMergeSequenceTest.java
@@ -300,16 +300,14 @@ public class OrderedMergeSequenceTest
   public void testHierarchicalMerge() throws Exception
   {
     final Sequence<Integer> seq1 = makeUnorderedMergedSequence(
-        Ordering.natural(), Lists.newArrayList(
-        TestSequence.create(1)
-    )
+        Ordering.natural(),
+        Collections.singletonList(TestSequence.create(1))
     );
 
 
     final Sequence<Integer> seq2 = makeUnorderedMergedSequence(
-        Ordering.natural(), Lists.newArrayList(
-        TestSequence.create(1)
-    )
+        Ordering.natural(),
+        Collections.singletonList(TestSequence.create(1))
     );
     final OrderedMergeSequence<Integer> finalMerged = new OrderedMergeSequence<Integer>(
         Ordering.natural(),
@@ -325,15 +323,14 @@ public class OrderedMergeSequenceTest
   public void testMergeMerge() throws Exception
   {
     final Sequence<Integer> seq1 = makeUnorderedMergedSequence(
-        Ordering.natural(), Lists.newArrayList(
-            TestSequence.create(1)
-        )
+        Ordering.natural(),
+        Collections.singletonList(TestSequence.create(1))
     );
 
     final OrderedMergeSequence<Integer> finalMerged = new OrderedMergeSequence<Integer>(
         Ordering.natural(),
         Sequences.simple(
-            Lists.newArrayList(seq1)
+            Collections.singletonList(seq1)
         )
     );
 
@@ -344,9 +341,8 @@ public class OrderedMergeSequenceTest
   public void testOne() throws Exception
   {
     final MergeSequence<Integer> seq1 = makeUnorderedMergedSequence(
-        Ordering.natural(), Lists.newArrayList(
-            TestSequence.create(1)
-        )
+        Ordering.natural(),
+        Collections.singletonList(TestSequence.create(1))
     );
 
     SequenceTestHelper.testAll(seq1, Collections.singletonList(1));
diff --git a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
index c18e14d..c02325d 100644
--- a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
+++ b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
@@ -94,31 +94,18 @@ public class DistinctCountGroupByQueryTest
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setGranularity(QueryRunnerTestHelper.allGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    client_type,
-                    client_type
-                )
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec(
+            client_type,
+            client_type
+        ))
         .setInterval(QueryRunnerTestHelper.fullOnInterval)
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        client_type,
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ), 10
-            )
-        )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.rowsCount,
-                new DistinctCountAggregatorFactory("UV", visitor_id, null)
+                Collections.singletonList(new OrderByColumnSpec(client_type, OrderByColumnSpec.Direction.DESCENDING)),
+                10
             )
         )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new DistinctCountAggregatorFactory("UV", visitor_id, null))
         .build();
     final Segment incrementalIndexSegment = new IncrementalIndexSegment(index, null);
 
diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/io/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/io/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
index ef4e881..1b3ecfc 100644
--- a/extensions-contrib/materialized-view-maintenance/src/test/java/io/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
+++ b/extensions-contrib/materialized-view-maintenance/src/test/java/io/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
@@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import io.druid.data.input.impl.DimensionsSpec;
@@ -55,6 +54,7 @@ import org.junit.Rule;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -101,7 +101,7 @@ public class MaterializedViewSupervisorTest
     objectMapper.registerSubtypes(new NamedType(HashBasedNumberedShardSpec.class, "hashed"));
     spec = new MaterializedViewSupervisorSpec(
         "base",
-        new DimensionsSpec(Lists.newArrayList(new StringDimensionSchema("dim")), null, null),
+        new DimensionsSpec(Collections.singletonList(new StringDimensionSchema("dim")), null, null),
         new AggregatorFactory[]{new LongSumAggregatorFactory("m1", "m1")},
         HadoopTuningConfig.makeDefaultTuningConfig(),
         null,
@@ -157,16 +157,16 @@ public class MaterializedViewSupervisorTest
     Map<Interval, List<DataSegment>> expectedSegments = Maps.newHashMap();
     expectedSegments.put(
         Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), 
-        Lists.newArrayList(
+        Collections.singletonList(
             new DataSegment(
-                "base", 
-                Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), 
-                "2015-01-02", 
+                "base",
+                Intervals.of("2015-01-01T00Z/2015-01-02T00Z"),
+                "2015-01-02",
                 ImmutableMap.of(),
-                ImmutableList.of("dim1", "dim2"), 
-                ImmutableList.of("m1"), 
+                ImmutableList.of("dim1", "dim2"),
+                ImmutableList.of("m1"),
                 new HashBasedNumberedShardSpec(0, 1, null, null),
-                9, 
+                9,
                 1024
             )
         )
diff --git a/extensions-contrib/materialized-view-selection/src/main/java/io/druid/query/materializedview/DataSourceOptimizer.java b/extensions-contrib/materialized-view-selection/src/main/java/io/druid/query/materializedview/DataSourceOptimizer.java
index 49436b6..fc20476 100644
--- a/extensions-contrib/materialized-view-selection/src/main/java/io/druid/query/materializedview/DataSourceOptimizer.java
+++ b/extensions-contrib/materialized-view-selection/src/main/java/io/druid/query/materializedview/DataSourceOptimizer.java
@@ -88,7 +88,7 @@ public class DataSourceOptimizer
     Set<DerivativeDataSource> derivatives = DerivativeDataSourceManager.getDerivatives(datasourceName);
     
     if (derivatives.isEmpty()) {
-      return Lists.newArrayList(query);
+      return Collections.singletonList(query);
     }
     lock.readLock().lock();
     try {
@@ -113,7 +113,7 @@ public class DataSourceOptimizer
         missFields.get(datasourceName).putIfAbsent(requiredFields, new AtomicLong(0));
         missFields.get(datasourceName).get(requiredFields).incrementAndGet();
         costTime.get(datasourceName).addAndGet(System.currentTimeMillis() - start);
-        return Lists.newArrayList(query);
+        return Collections.singletonList(query);
       }
       
       List<Query> queries = Lists.newArrayList();
@@ -147,7 +147,7 @@ public class DataSourceOptimizer
 
       if (queries.isEmpty()) {
         costTime.get(datasourceName).addAndGet(System.currentTimeMillis() - start);
-        return Lists.newArrayList(query);
+        return Collections.singletonList(query);
       }
 
       //after materialized view selection, the result of the remaining query interval will be computed based on
diff --git a/extensions-contrib/materialized-view-selection/src/test/java/io/druid/query/materializedview/DatasourceOptimizerTest.java b/extensions-contrib/materialized-view-selection/src/test/java/io/druid/query/materializedview/DatasourceOptimizerTest.java
index f4ef066..a479f9c 100644
--- a/extensions-contrib/materialized-view-selection/src/test/java/io/druid/query/materializedview/DatasourceOptimizerTest.java
+++ b/extensions-contrib/materialized-view-selection/src/test/java/io/druid/query/materializedview/DatasourceOptimizerTest.java
@@ -44,7 +44,6 @@ import io.druid.query.Query;
 import static io.druid.query.QueryRunnerTestHelper.allGran;
 import io.druid.query.QueryToolChestWarehouse;
 import io.druid.query.QueryWatcher;
-import io.druid.query.aggregation.AggregatorFactory;
 import io.druid.query.aggregation.LongSumAggregatorFactory;
 import io.druid.query.spec.MultipleIntervalSegmentSpec;
 import io.druid.query.topn.TopNQuery;
@@ -64,6 +63,7 @@ import org.junit.Rule;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.Executor;
@@ -204,9 +204,7 @@ public class DatasourceOptimizerTest extends CuratorTestBase
         .threshold(4)
         .intervals("2011-04-01/2011-04-06")
         .aggregators(
-            Lists.<AggregatorFactory>newArrayList(
-                new LongSumAggregatorFactory("cost", "cost")
-            )
+            Collections.singletonList(new LongSumAggregatorFactory("cost", "cost"))
         )
         .build();
     
@@ -217,11 +215,9 @@ public class DatasourceOptimizerTest extends CuratorTestBase
             .dimension("dim1")
             .metric("cost")
             .threshold(4)
-            .intervals(new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-01/2011-04-04"))))
+            .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-01/2011-04-04"))))
             .aggregators(
-                Lists.<AggregatorFactory>newArrayList(
-                    new LongSumAggregatorFactory("cost", "cost")
-                )
+                Collections.singletonList(new LongSumAggregatorFactory("cost", "cost"))
             )
             .build(),
         new TopNQueryBuilder()
@@ -230,11 +226,9 @@ public class DatasourceOptimizerTest extends CuratorTestBase
             .dimension("dim1")
             .metric("cost")
             .threshold(4)
-            .intervals(new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-04/2011-04-06"))))
+            .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-04/2011-04-06"))))
             .aggregators(
-                Lists.<AggregatorFactory>newArrayList(
-                    new LongSumAggregatorFactory("cost", "cost")
-                )
+                Collections.singletonList(new LongSumAggregatorFactory("cost", "cost"))
             )
             .build()
     );
diff --git a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/BasicRoleBasedAuthorizerTest.java b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/BasicRoleBasedAuthorizerTest.java
index b258056..3c0ed69 100644
--- a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/BasicRoleBasedAuthorizerTest.java
+++ b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/BasicRoleBasedAuthorizerTest.java
@@ -22,7 +22,6 @@ package io.druid.security.authorization;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.dataformat.smile.SmileFactory;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import io.druid.metadata.MetadataStorageTablesConfig;
 import io.druid.metadata.TestDerbyConnector;
 import io.druid.security.basic.BasicAuthCommonCacheConfig;
@@ -42,6 +41,7 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import java.util.Collections;
 import java.util.List;
 
 public class BasicRoleBasedAuthorizerTest
@@ -107,11 +107,8 @@ public class BasicRoleBasedAuthorizerTest
     updater.createRole(AUTHORIZER_NAME, "druidRole");
     updater.assignRole(AUTHORIZER_NAME, "druid", "druidRole");
 
-    List<ResourceAction> permissions = Lists.newArrayList(
-        new ResourceAction(
-            new Resource("testResource", ResourceType.DATASOURCE),
-            Action.WRITE
-        )
+    List<ResourceAction> permissions = Collections.singletonList(
+        new ResourceAction(new Resource("testResource", ResourceType.DATASOURCE), Action.WRITE)
     );
 
     updater.setPermissions(AUTHORIZER_NAME, "druidRole", permissions);
diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java
index 2a2b902..4acf138 100644
--- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java
+++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java
@@ -150,32 +150,17 @@ public class ApproximateHistogramGroupByQueryTest
 
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setGranularity(QueryRunnerTestHelper.allGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    QueryRunnerTestHelper.marketDimension,
-                    "marketalias"
-                )
-            )
-        )
+        .setGranularity(QueryRunnerTestHelper.allGran).setDimensions(new DefaultDimensionSpec(
+            QueryRunnerTestHelper.marketDimension,
+            "marketalias"
+        ))
         .setInterval(QueryRunnerTestHelper.fullOnInterval)
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "marketalias",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ), 1
+                Collections.singletonList(new OrderByColumnSpec("marketalias", OrderByColumnSpec.Direction.DESCENDING)),
+                1
             )
-        )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.rowsCount,
-                aggFactory
-            )
-        )
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, aggFactory)
         .setPostAggregatorSpecs(
             Collections.singletonList(
                 new QuantilePostAggregator("quantile", "apphisto", 0.5f)
@@ -224,32 +209,17 @@ public class ApproximateHistogramGroupByQueryTest
 
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setGranularity(QueryRunnerTestHelper.allGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    QueryRunnerTestHelper.marketDimension,
-                    "marketalias"
-                )
-            )
-        )
+        .setGranularity(QueryRunnerTestHelper.allGran).setDimensions(new DefaultDimensionSpec(
+            QueryRunnerTestHelper.marketDimension,
+            "marketalias"
+        ))
         .setInterval(QueryRunnerTestHelper.fullOnInterval)
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "marketalias",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ), 1
+                Collections.singletonList(new OrderByColumnSpec("marketalias", OrderByColumnSpec.Direction.DESCENDING)),
+                1
             )
-        )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.rowsCount,
-                aggFactory
-            )
-        )
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, aggFactory)
         .setPostAggregatorSpecs(
             Collections.singletonList(
                 new QuantilePostAggregator("quantile", "quantile", 0.5f)
diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java
index 56f25e8..a61bd49 100644
--- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java
+++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java
@@ -329,7 +329,7 @@ public class QuantileSqlAggregatorTest extends CalciteTestBase
                                           .setDataSource(CalciteTests.DATASOURCE1)
                                           .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
                                           .setGranularity(Granularities.ALL)
-                                          .setDimensions(ImmutableList.of(new DefaultDimensionSpec("dim2", "d0")))
+                                          .setDimensions(new DefaultDimensionSpec("dim2", "d0"))
                                           .setAggregatorSpecs(
                                               ImmutableList.of(
                                                   new DoubleSumAggregatorFactory("a0", "m1")
@@ -341,11 +341,15 @@ public class QuantileSqlAggregatorTest extends CalciteTestBase
                       )
                       .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
                       .setGranularity(Granularities.ALL)
-                      .setAggregatorSpecs(ImmutableList.of(
-                          new DoubleSumAggregatorFactory("_a0:sum", "a0"),
-                          new CountAggregatorFactory("_a0:count"),
-                          new ApproximateHistogramAggregatorFactory("_a1:agg", "a0", null, null, null, null)
-                      ))
+                      .setAggregatorSpecs(new DoubleSumAggregatorFactory("_a0:sum", "a0"),
+                                          new CountAggregatorFactory("_a0:count"),
+                                          new ApproximateHistogramAggregatorFactory("_a1:agg",
+                                                                                    "a0",
+                                                                                    null,
+                                                                                    null,
+                                                                                    null,
+                                                                                    null
+                                          ))
                       .setPostAggregatorSpecs(
                           ImmutableList.of(
                               new ArithmeticPostAggregator(
diff --git a/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/LoadingLookupTest.java b/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/LoadingLookupTest.java
index d5bff2e..c8e8078 100644
--- a/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/LoadingLookupTest.java
+++ b/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/LoadingLookupTest.java
@@ -21,7 +21,6 @@ package io.druid.server.lookup;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
 import io.druid.server.lookup.cache.loading.LoadingCache;
 import org.easymock.EasyMock;
 import org.junit.Assert;
@@ -65,10 +64,10 @@ public class LoadingLookupTest
   public void testUnapplyAll() throws ExecutionException
   {
     EasyMock.expect(reverseLookupCache.get(EasyMock.eq("value"), EasyMock.anyObject(Callable.class)))
-            .andReturn(Lists.newArrayList("key"))
+            .andReturn(Collections.singletonList("key"))
             .once();
     EasyMock.replay(reverseLookupCache);
-    Assert.assertEquals(ImmutableMap.of("value", Lists.newArrayList("key")), loadingLookup.unapplyAll(ImmutableSet.of("value")));
+    Assert.assertEquals(ImmutableMap.of("value", Collections.singletonList("key")), loadingLookup.unapplyAll(ImmutableSet.of("value")));
     EasyMock.verify(reverseLookupCache);
   }
 
diff --git a/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/cache/loading/LoadingCacheTest.java b/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/cache/loading/LoadingCacheTest.java
index 6c1a706..a0dbf64 100644
--- a/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/cache/loading/LoadingCacheTest.java
+++ b/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/cache/loading/LoadingCacheTest.java
@@ -21,7 +21,6 @@ package io.druid.server.lookup.cache.loading;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import io.druid.jackson.DefaultObjectMapper;
 import org.junit.After;
 import org.junit.Assert;
@@ -126,7 +125,7 @@ public class LoadingCacheTest
       }
     });
     Assert.assertEquals("value2", loadingCache.getIfPresent("key2"));
-    loadingCache.invalidateAll(Lists.newArrayList("key2"));
+    loadingCache.invalidateAll(Collections.singletonList("key2"));
     Assert.assertEquals(null, loadingCache.getIfPresent("key2"));
   }
 
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 1b1c3da..5acfe77 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
@@ -20,7 +20,6 @@
 package io.druid.query.aggregation.variance;
 
 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.granularity.PeriodGranularity;
@@ -43,7 +42,6 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -79,8 +77,8 @@ public class VarianceGroupByQueryTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(Collections.singletonList(VarianceTestHelper.indexVarianceAggr))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(VarianceTestHelper.indexVarianceAggr)
         .setPostAggregatorSpecs(Collections.singletonList(VarianceTestHelper.stddevOfIndexPostAggr))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
@@ -121,14 +119,10 @@ public class VarianceGroupByQueryTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                VarianceTestHelper.rowsCount,
-                VarianceTestHelper.indexVarianceAggr,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(VarianceTestHelper.rowsCount,
+                            VarianceTestHelper.indexVarianceAggr,
+                            new LongSumAggregatorFactory("idx", "index"))
         .setPostAggregatorSpecs(
             Collections.singletonList(VarianceTestHelper.stddevOfIndexPostAggr)
         )
@@ -181,14 +175,10 @@ public class VarianceGroupByQueryTest
         .builder()
         .setDataSource(VarianceTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                VarianceTestHelper.rowsCount,
-                VarianceTestHelper.indexLongSum,
-                VarianceTestHelper.indexVarianceAggr
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(VarianceTestHelper.rowsCount,
+                            VarianceTestHelper.indexLongSum,
+                            VarianceTestHelper.indexVarianceAggr)
         .setPostAggregatorSpecs(ImmutableList.of(VarianceTestHelper.stddevOfIndexPostAggr))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setHavingSpec(
diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java
index 67f9e30..1725b71 100644
--- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java
+++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java
@@ -29,6 +29,7 @@ import io.druid.timeline.partition.NoneShardSpec;
 import org.joda.time.DateTime;
 import org.joda.time.Interval;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -81,7 +82,7 @@ public class HadoopDruidDetermineConfigurationJob implements Jobby
           log.info("DateTime[%s], spec[%s]", bucket, specs);
         } else {
           final HadoopyShardSpec spec = new HadoopyShardSpec(NoneShardSpec.instance(), shardCount++);
-          shardSpecs.put(bucket.getMillis(), Lists.newArrayList(spec));
+          shardSpecs.put(bucket.getMillis(), Collections.singletonList(spec));
           log.info("DateTime[%s], spec[%s]", bucket, spec);
         }
       }
diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java
index f1dceb1..2135ae7 100644
--- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java
+++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java
@@ -38,6 +38,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 /**
@@ -149,12 +150,12 @@ public class HadoopDruidIndexerConfigTest
             null,
             null,
             ImmutableMap.of(DateTimes.of("2010-01-01T01:00:00").getMillis(),
-                                                              Lists.newArrayList(new HadoopyShardSpec(
+                                                              Collections.singletonList(new HadoopyShardSpec(
                                                                   NoneShardSpec.instance(),
                                                                   1
                                                               )),
                                                               DateTimes.of("2010-01-01T02:00:00").getMillis(),
-                                                              Lists.newArrayList(new HadoopyShardSpec(
+                                                              Collections.singletonList(new HadoopyShardSpec(
                                                                   NoneShardSpec.instance(),
                                                                   2
                                                               ))
diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecTest.java
index a285db1..5fdf167 100644
--- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecTest.java
+++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecTest.java
@@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.InjectableValues;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Strings;
 import com.google.common.base.Throwables;
-import com.google.common.collect.Lists;
 import io.druid.indexer.partitions.HashedPartitionsSpec;
 import io.druid.indexer.partitions.PartitionsSpec;
 import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
@@ -39,6 +38,8 @@ import org.joda.time.Period;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.util.Collections;
+
 public class HadoopIngestionSpecTest
 {
   private static final ObjectMapper jsonMapper;
@@ -77,7 +78,7 @@ public class HadoopIngestionSpecTest
 
     Assert.assertEquals(
         "getIntervals",
-        Lists.newArrayList(Intervals.of("2012-01-01/P1D")),
+        Collections.singletonList(Intervals.of("2012-01-01/P1D")),
         granularitySpec.getIntervals().get()
     );
 
diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java
index e1a89ec..830a112 100644
--- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java
+++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java
@@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.InjectableValues;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import io.druid.indexer.hadoop.DatasourceIngestionSpec;
 import io.druid.indexer.hadoop.WindowedDataSegment;
 import io.druid.indexer.path.DatasourcePathSpec;
@@ -45,6 +44,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Map;
 
 /**
@@ -307,14 +307,14 @@ public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest
     EasyMock.expect(
         segmentLister.getUsedSegmentsForIntervals(
             testDatasource,
-            Lists.newArrayList(jobInterval != null ? jobInterval.overlap(testDatasourceInterval) : null)
+            Collections.singletonList(jobInterval != null ? jobInterval.overlap(testDatasourceInterval) : null)
         )
     ).andReturn(ImmutableList.of(SEGMENT));
 
     EasyMock.expect(
         segmentLister.getUsedSegmentsForIntervals(
             testDatasource2,
-            Lists.newArrayList(jobInterval != null ? jobInterval.overlap(testDatasourceInterval2) : null)
+            Collections.singletonList(jobInterval != null ? jobInterval.overlap(testDatasourceInterval2) : null)
         )
     ).andReturn(ImmutableList.of(SEGMENT2));
 
diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java
index 555ee76..ba1a0f0 100644
--- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java
+++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java
@@ -127,7 +127,7 @@ public class IndexGeneratorCombinerTest
     BytesWritable key = new BytesWritable(StringUtils.toUtf8("dummy_key"));
     BytesWritable val = new BytesWritable(StringUtils.toUtf8("dummy_row"));
 
-    combiner.reduce(key, Lists.newArrayList(val), context);
+    combiner.reduce(key, Collections.singletonList(val), context);
 
     Assert.assertTrue(captureKey.getValue() == key);
     Assert.assertTrue(captureVal.getValue() == val);
diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceInputSplitTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceInputSplitTest.java
index d579876..9d75dca 100644
--- a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceInputSplitTest.java
+++ b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceInputSplitTest.java
@@ -21,7 +21,6 @@ package io.druid.indexer.hadoop;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import com.google.common.io.ByteArrayDataOutput;
 import com.google.common.io.ByteStreams;
 import io.druid.java.util.common.Intervals;
@@ -32,6 +31,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.DataInput;
+import java.util.Collections;
 
 /**
  */
@@ -42,7 +42,7 @@ public class DatasourceInputSplitTest
   {
     Interval interval = Intervals.of("2000/3000");
     DatasourceInputSplit expected = new DatasourceInputSplit(
-        Lists.newArrayList(
+        Collections.singletonList(
             new WindowedDataSegment(
                 new DataSegment(
                     "test",
diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceRecordReaderTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceRecordReaderTest.java
index 75e9b12..9cfe1da 100644
--- a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceRecordReaderTest.java
+++ b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceRecordReaderTest.java
@@ -57,7 +57,7 @@ public class DatasourceRecordReaderTest
                 this.getClass().getClassLoader().getResource("test-segment/index.zip").getPath()
             )
         );
-    InputSplit split = new DatasourceInputSplit(Lists.newArrayList(WindowedDataSegment.of(segment)), null);
+    InputSplit split = new DatasourceInputSplit(Collections.singletonList(WindowedDataSegment.of(segment)), null);
 
     Configuration config = new Configuration();
     DatasourceInputFormat.addDataSource(
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java
index 30c13ad..e15d6e5 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java
@@ -20,7 +20,6 @@
 package io.druid.indexing.common.task;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 import io.druid.indexing.common.TaskLock;
 import io.druid.indexing.common.TaskLockType;
 import io.druid.indexing.common.actions.LockTryAcquireAction;
@@ -74,7 +73,8 @@ public class Tasks
           toBeAccumulated.add(interval);
         } else {
           compactIntervals.add(JodaUtils.umbrellaInterval(toBeAccumulated));
-          toBeAccumulated = Lists.newArrayList(interval);
+          toBeAccumulated.clear();
+          toBeAccumulated.add(interval);
         }
       }
     }
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java
index 6589e0c..f3b7876 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java
@@ -96,6 +96,7 @@ import javax.ws.rs.core.Response.Status;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -953,11 +954,8 @@ public class OverlordResource
             ).build()
         );
       }
-      return Lists.newArrayList(
-          new ResourceAction(
-              new Resource(taskDatasource, ResourceType.DATASOURCE),
-              Action.READ
-          )
+      return Collections.singletonList(
+          new ResourceAction(new Resource(taskDatasource, ResourceType.DATASOURCE), Action.READ)
       );
     };
     List<TaskStatusPlus> optionalTypeFilteredList = collectionToFilter;
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/EC2AutoScalerSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/EC2AutoScalerSerdeTest.java
index dd6ea2c..76e9132 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/EC2AutoScalerSerdeTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/EC2AutoScalerSerdeTest.java
@@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.BeanProperty;
 import com.fasterxml.jackson.databind.DeserializationContext;
 import com.fasterxml.jackson.databind.InjectableValues;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.Lists;
 import com.google.common.io.BaseEncoding;
 import io.druid.indexing.overlord.autoscaling.ec2.EC2AutoScaler;
 import io.druid.jackson.DefaultObjectMapper;
@@ -31,6 +30,8 @@ import io.druid.java.util.common.StringUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.util.Collections;
+
 public class EC2AutoScalerSerdeTest
 {
   final String json = "{\n"
@@ -99,7 +100,7 @@ public class EC2AutoScalerSerdeTest
     Assert.assertEquals(1, autoScaler.getEnvConfig().getNodeData().getMaxInstances());
     Assert.assertEquals(1, autoScaler.getEnvConfig().getNodeData().getMinInstances());
     Assert.assertEquals(
-        Lists.newArrayList("kingsguard"),
+        Collections.singletonList("kingsguard"),
         autoScaler.getEnvConfig().getNodeData().getSecurityGroupIds()
     );
     Assert.assertEquals("redkeep", autoScaler.getEnvConfig().getNodeData().getSubnetId());
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java
index e2e2748..f4f734c 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java
@@ -122,7 +122,7 @@ public class PendingTaskBasedProvisioningStrategyTest
     );
     EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig());
     EasyMock.expect(autoScaler.provision()).andReturn(
-        new AutoScalingData(Lists.newArrayList("aNode"))
+        new AutoScalingData(Collections.singletonList("aNode"))
     ).times(3);
     EasyMock.replay(runner, autoScaler);
     Provisioner provisioner = strategy.makeProvisioner(runner);
@@ -156,7 +156,7 @@ public class PendingTaskBasedProvisioningStrategyTest
     );
     EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig());
     EasyMock.expect(autoScaler.provision()).andReturn(
-        new AutoScalingData(Lists.newArrayList("aNode"))
+        new AutoScalingData(Collections.singletonList("aNode"))
     ).times(2);
     EasyMock.replay(runner, autoScaler);
     Provisioner provisioner = strategy.makeProvisioner(runner);
@@ -191,7 +191,7 @@ public class PendingTaskBasedProvisioningStrategyTest
     );
     EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig());
     EasyMock.expect(autoScaler.provision()).andReturn(
-        new AutoScalingData(Lists.newArrayList("aNode"))
+        new AutoScalingData(Collections.singletonList("aNode"))
     ).times(2);
     EasyMock.replay(runner, autoScaler);
     Provisioner provisioner = strategy.makeProvisioner(runner);
@@ -213,7 +213,7 @@ public class PendingTaskBasedProvisioningStrategyTest
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
             .andReturn(Lists.<String>newArrayList()).times(2);
     EasyMock.expect(autoScaler.provision()).andReturn(
-        new AutoScalingData(Lists.newArrayList("fake"))
+        new AutoScalingData(Collections.singletonList("fake"))
     );
     RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class);
     EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(
@@ -272,7 +272,7 @@ public class PendingTaskBasedProvisioningStrategyTest
     EasyMock.expect(autoScaler.terminateWithIds(EasyMock.anyObject()))
             .andReturn(null);
     EasyMock.expect(autoScaler.provision()).andReturn(
-        new AutoScalingData(Lists.newArrayList("fake"))
+        new AutoScalingData(Collections.singletonList("fake"))
     );
     EasyMock.replay(autoScaler);
     RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class);
@@ -368,9 +368,9 @@ public class PendingTaskBasedProvisioningStrategyTest
   {
     EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0).times(1);
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
-            .andReturn(Lists.newArrayList("ip")).times(2);
+            .andReturn(Collections.singletonList("ip")).times(2);
     EasyMock.expect(autoScaler.terminate(EasyMock.anyObject())).andReturn(
-        new AutoScalingData(Lists.newArrayList("ip"))
+        new AutoScalingData(Collections.singletonList("ip"))
     );
     EasyMock.replay(autoScaler);
 
@@ -412,7 +412,7 @@ public class PendingTaskBasedProvisioningStrategyTest
     EasyMock.reset(autoScaler);
     EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0);
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
-            .andReturn(Lists.newArrayList("ip"));
+            .andReturn(Collections.singletonList("ip"));
     EasyMock.replay(autoScaler);
 
     RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class);
@@ -444,7 +444,7 @@ public class PendingTaskBasedProvisioningStrategyTest
     EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0);
     EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2);
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
-            .andReturn(Lists.newArrayList("ip"));
+            .andReturn(Collections.singletonList("ip"));
     EasyMock.replay(autoScaler);
 
     boolean provisionedSomething = provisioner.doProvision();
@@ -461,7 +461,7 @@ public class PendingTaskBasedProvisioningStrategyTest
     EasyMock.reset(autoScaler);
     EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0);
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
-            .andReturn(Lists.newArrayList("ip"));
+            .andReturn(Collections.singletonList("ip"));
     EasyMock.replay(autoScaler);
     RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class);
     EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(
@@ -489,7 +489,7 @@ public class PendingTaskBasedProvisioningStrategyTest
     EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0);
     EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2);
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
-            .andReturn(Lists.newArrayList("ip"));
+            .andReturn(Collections.singletonList("ip"));
     EasyMock.replay(autoScaler);
     boolean provisionedSomething = provisioner.doProvision();
     Assert.assertFalse(provisionedSomething);
@@ -500,13 +500,13 @@ public class PendingTaskBasedProvisioningStrategyTest
     EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(3);
     EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(5);
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
-            .andReturn(Lists.newArrayList("ip"));
+            .andReturn(Collections.singletonList("ip"));
     EasyMock.expect(autoScaler.provision()).andReturn(
-        new AutoScalingData(Lists.newArrayList("h3"))
+        new AutoScalingData(Collections.singletonList("h3"))
     );
     // Should provision two new workers
     EasyMock.expect(autoScaler.provision()).andReturn(
-        new AutoScalingData(Lists.newArrayList("h4"))
+        new AutoScalingData(Collections.singletonList("h4"))
     );
     EasyMock.replay(autoScaler);
     provisionedSomething = provisioner.doProvision();
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java
index ffd5aad..d0a120c 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java
@@ -117,7 +117,7 @@ public class SimpleProvisioningStrategyTest
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
             .andReturn(Lists.<String>newArrayList());
     EasyMock.expect(autoScaler.provision()).andReturn(
-        new AutoScalingData(Lists.newArrayList("aNode"))
+        new AutoScalingData(Collections.singletonList("aNode"))
     );
     RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class);
     EasyMock.expect(runner.getPendingTasks()).andReturn(
@@ -155,7 +155,7 @@ public class SimpleProvisioningStrategyTest
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
             .andReturn(Lists.<String>newArrayList()).times(2);
     EasyMock.expect(autoScaler.provision()).andReturn(
-        new AutoScalingData(Lists.newArrayList("fake"))
+        new AutoScalingData(Collections.singletonList("fake"))
     );
     RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class);
     EasyMock.expect(runner.getPendingTasks()).andReturn(
@@ -213,7 +213,7 @@ public class SimpleProvisioningStrategyTest
     EasyMock.expect(autoScaler.terminateWithIds(EasyMock.anyObject()))
             .andReturn(null);
     EasyMock.expect(autoScaler.provision()).andReturn(
-        new AutoScalingData(Lists.newArrayList("fake"))
+        new AutoScalingData(Collections.singletonList("fake"))
     );
     EasyMock.replay(autoScaler);
     RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class);
@@ -304,9 +304,9 @@ public class SimpleProvisioningStrategyTest
     EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0).times(2);
     EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(1).times(2);
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
-            .andReturn(Lists.newArrayList("ip")).times(2);
+            .andReturn(Collections.singletonList("ip")).times(2);
     EasyMock.expect(autoScaler.terminate(EasyMock.anyObject())).andReturn(
-        new AutoScalingData(Lists.newArrayList("ip"))
+        new AutoScalingData(Collections.singletonList("ip"))
     );
     EasyMock.replay(autoScaler);
 
@@ -355,7 +355,7 @@ public class SimpleProvisioningStrategyTest
     EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0);
     EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2);
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
-            .andReturn(Lists.newArrayList("ip"));
+            .andReturn(Collections.singletonList("ip"));
     EasyMock.replay(autoScaler);
 
     RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class);
@@ -386,7 +386,7 @@ public class SimpleProvisioningStrategyTest
     EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0);
     EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2);
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
-            .andReturn(Lists.newArrayList("ip"));
+            .andReturn(Collections.singletonList("ip"));
     EasyMock.replay(autoScaler);
 
     boolean provisionedSomething = provisioner.doProvision();
@@ -404,7 +404,7 @@ public class SimpleProvisioningStrategyTest
     EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0);
     EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2);
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
-            .andReturn(Lists.newArrayList("ip"));
+            .andReturn(Collections.singletonList("ip"));
     EasyMock.replay(autoScaler);
     RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class);
     EasyMock.expect(runner.getPendingTasks()).andReturn(
@@ -430,7 +430,7 @@ public class SimpleProvisioningStrategyTest
     EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0);
     EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2);
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
-            .andReturn(Lists.newArrayList("ip"));
+            .andReturn(Collections.singletonList("ip"));
     EasyMock.replay(autoScaler);
     boolean provisionedSomething = provisioner.doProvision();
     Assert.assertFalse(provisionedSomething);
@@ -441,13 +441,13 @@ public class SimpleProvisioningStrategyTest
     EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(3);
     EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(5);
     EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject()))
-            .andReturn(Lists.newArrayList("ip"));
+            .andReturn(Collections.singletonList("ip"));
     EasyMock.expect(autoScaler.provision()).andReturn(
-        new AutoScalingData(Lists.newArrayList("h3"))
+        new AutoScalingData(Collections.singletonList("h3"))
     );
     // Should provision two new workers
     EasyMock.expect(autoScaler.provision()).andReturn(
-        new AutoScalingData(Lists.newArrayList("h4"))
+        new AutoScalingData(Collections.singletonList("h4"))
     );
     EasyMock.replay(autoScaler);
     provisionedSomething = provisioner.doProvision();
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java
index 614585b..3c37691 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java
@@ -24,7 +24,6 @@ import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import io.druid.indexing.overlord.DataSourceMetadata;
 import io.druid.indexing.overlord.TaskMaster;
@@ -107,7 +106,7 @@ public class SupervisorResourceTest extends EasyMockSupport
       @Override
       public List<String> getDataSources()
       {
-        return Lists.newArrayList("datasource1");
+        return Collections.singletonList("datasource1");
       }
     };
 
@@ -147,7 +146,7 @@ public class SupervisorResourceTest extends EasyMockSupport
       @Override
       public List<String> getDataSources()
       {
-        return Lists.newArrayList("datasource1");
+        return Collections.singletonList("datasource1");
       }
     };
     SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, null) {
@@ -155,7 +154,7 @@ public class SupervisorResourceTest extends EasyMockSupport
       @Override
       public List<String> getDataSources()
       {
-        return Lists.newArrayList("datasource2");
+        return Collections.singletonList("datasource2");
       }
     };
 
@@ -753,7 +752,7 @@ public class SupervisorResourceTest extends EasyMockSupport
     NoopSupervisorSpec deserializedSpec = mapper.readValue(oldSpec, NoopSupervisorSpec.class);
     Assert.assertEquals(expectedSpec, deserializedSpec);
 
-    NoopSupervisorSpec spec1 = new NoopSupervisorSpec("abcd", Lists.newArrayList("defg"));
+    NoopSupervisorSpec spec1 = new NoopSupervisorSpec("abcd", Collections.singletonList("defg"));
     NoopSupervisorSpec spec2 = mapper.readValue(
         mapper.writeValueAsBytes(spec1),
         NoopSupervisorSpec.class
diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java
index 359d05a..55ee207 100644
--- a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java
+++ b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java
@@ -143,16 +143,14 @@ public class MergeSequenceTest
   {
     final Sequence<Integer> seq1 = new MergeSequence<>(
         Ordering.natural(), Sequences.simple(
-        Lists.newArrayList(
-            TestSequence.create(1)
-        )
+        Collections.singletonList(TestSequence.create(1))
     )
     );
 
     final Sequence<Integer> finalMerged = new MergeSequence<>(
         Ordering.natural(),
         Sequences.simple(
-            Lists.newArrayList(seq1)
+            Collections.singletonList(seq1)
         )
     );
 
@@ -164,9 +162,7 @@ public class MergeSequenceTest
   {
     final Sequence<Integer> mergeOne = new MergeSequence<>(
         Ordering.natural(), Sequences.simple(
-        Lists.newArrayList(
-            TestSequence.create(1)
-        )
+        Collections.singletonList(TestSequence.create(1))
     )
     );
 
diff --git a/processing/src/main/java/io/druid/collections/spatial/Node.java b/processing/src/main/java/io/druid/collections/spatial/Node.java
index a3e270b..b9ed875 100644
--- a/processing/src/main/java/io/druid/collections/spatial/Node.java
+++ b/processing/src/main/java/io/druid/collections/spatial/Node.java
@@ -20,11 +20,12 @@
 package io.druid.collections.spatial;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 import io.druid.collections.bitmap.BitmapFactory;
 import io.druid.collections.bitmap.MutableBitmap;
 
+import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
@@ -46,17 +47,30 @@ public class Node
     this(
         minCoordinates,
         maxCoordinates,
-        Lists.newArrayList(),
+        null,
         isLeaf,
         null,
         bitmapFactory.makeEmptyMutableBitmap()
     );
   }
 
+  /**
+   * This constructor accepts a single nullable child Node (null value means no child) instead of a collection of
+   * children Nodes, because Nodes with no more than one child are created in the codebase yet, while passing a
+   * collection of Nodes would necessitate making a defensive copy of this collection in the constructor and extra
+   * overhead.
+   *
+   * (One could note that the principle of making a defensive copy is happily violated just in this
+   * constructor, other parameters: minCoordinates, maxCoordinates and bitmap. These are recognized flaws that are not
+   * tackled yet.)
+   *
+   * If cases when a Node should be created with multiple children arise, this constructor should be changed to accept
+   * a collection of children Nodes.
+   */
   public Node(
       float[] minCoordinates,
       float[] maxCoordinates,
-      List<Node> children,
+      @Nullable Node child,
       boolean isLeaf,
       Node parent,
       MutableBitmap bitmap
@@ -66,8 +80,9 @@ public class Node
 
     this.minCoordinates = minCoordinates;
     this.maxCoordinates = maxCoordinates;
-    this.children = children;
-    for (Node child : children) {
+    this.children = new ArrayList<>(1);
+    if (child != null) {
+      children.add(child);
       child.setParent(this);
     }
     this.isLeaf = isLeaf;
diff --git a/processing/src/main/java/io/druid/collections/spatial/Point.java b/processing/src/main/java/io/druid/collections/spatial/Point.java
index 223197a..9ec6fe4 100644
--- a/processing/src/main/java/io/druid/collections/spatial/Point.java
+++ b/processing/src/main/java/io/druid/collections/spatial/Point.java
@@ -23,7 +23,6 @@ import com.google.common.collect.Lists;
 import io.druid.collections.bitmap.BitmapFactory;
 import io.druid.collections.bitmap.MutableBitmap;
 
-import java.util.Arrays;
 import java.util.List;
 
 /**
@@ -37,8 +36,8 @@ public class Point extends Node
   {
     super(
         coords,
-        Arrays.copyOf(coords, coords.length),
-        Lists.newArrayList(),
+        coords.clone(),
+        null,
         true,
         null,
         makeBitmap(entry, bitmapFactory)
@@ -51,7 +50,7 @@ public class Point extends Node
 
   public Point(float[] coords, MutableBitmap entry)
   {
-    super(coords, Arrays.copyOf(coords, coords.length), Lists.newArrayList(), true, null, entry);
+    super(coords, coords.clone(), null, true, null, entry);
 
     this.coords = coords;
     this.bitmap = entry;
diff --git a/processing/src/main/java/io/druid/collections/spatial/split/GutmanSplitStrategy.java b/processing/src/main/java/io/druid/collections/spatial/split/GutmanSplitStrategy.java
index 552ade4..db8f880 100644
--- a/processing/src/main/java/io/druid/collections/spatial/split/GutmanSplitStrategy.java
+++ b/processing/src/main/java/io/druid/collections/spatial/split/GutmanSplitStrategy.java
@@ -24,7 +24,6 @@ import io.druid.collections.bitmap.BitmapFactory;
 import io.druid.collections.spatial.Node;
 import io.druid.collections.spatial.RTreeUtils;
 
-import java.util.Arrays;
 import java.util.List;
 
 /**
@@ -75,9 +74,9 @@ public abstract class GutmanSplitStrategy implements SplitStrategy
     node.addToBitmapIndex(seeds[0]);
 
     Node group1 = new Node(
-        Arrays.copyOf(seeds[1].getMinCoordinates(), seeds[1].getMinCoordinates().length),
-        Arrays.copyOf(seeds[1].getMaxCoordinates(), seeds[1].getMaxCoordinates().length),
-        Lists.newArrayList(seeds[1]),
+        seeds[1].getMinCoordinates().clone(),
+        seeds[1].getMaxCoordinates().clone(),
+        seeds[1],
         node.isLeaf(),
         node.getParent(),
         bf.makeEmptyMutableBitmap()
diff --git a/processing/src/main/java/io/druid/query/Druids.java b/processing/src/main/java/io/druid/query/Druids.java
index 1bc1c7e..e715423 100644
--- a/processing/src/main/java/io/druid/query/Druids.java
+++ b/processing/src/main/java/io/druid/query/Druids.java
@@ -54,6 +54,7 @@ import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
@@ -412,7 +413,7 @@ public class Druids
 
     public SearchQueryBuilder dimensions(DimensionSpec d)
     {
-      dimensions = Lists.newArrayList(d);
+      dimensions = Collections.singletonList(d);
       return this;
     }
 
diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java
index 5454abf..c174a3d 100644
--- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java
+++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java
@@ -126,7 +126,7 @@ public class IntervalChunkingQueryRunner<T> implements QueryRunner<T>
   private static Iterable<Interval> splitInterval(Interval interval, Period period)
   {
     if (interval.getEndMillis() == interval.getStartMillis()) {
-      return Lists.newArrayList(interval);
+      return Collections.singletonList(interval);
     }
 
     List<Interval> intervals = Lists.newArrayList();
diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java
index dadb7de..a3bc517 100644
--- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java
+++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java
@@ -42,6 +42,7 @@ import java.lang.reflect.Array;
 import java.nio.ByteBuffer;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Objects;
@@ -148,7 +149,7 @@ public class JavaScriptAggregatorFactory extends AggregatorFactory
   @Override
   public AggregatorFactory getCombiningFactory()
   {
-    return new JavaScriptAggregatorFactory(name, Lists.newArrayList(name), fnCombine, fnReset, fnCombine, config);
+    return new JavaScriptAggregatorFactory(name, Collections.singletonList(name), fnCombine, fnReset, fnCombine, config);
   }
 
   @Override
@@ -174,7 +175,7 @@ public class JavaScriptAggregatorFactory extends AggregatorFactory
               @Override
               public AggregatorFactory apply(String input)
               {
-                return new JavaScriptAggregatorFactory(input, Lists.newArrayList(input), fnCombine, fnReset, fnCombine, config);
+                return new JavaScriptAggregatorFactory(input, Collections.singletonList(input), fnCombine, fnReset, fnCombine, config);
               }
             }
         )
diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java
index 66d9ded..89925e7 100644
--- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java
+++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java
@@ -892,6 +892,13 @@ public class GroupByQuery extends BaseQuery<Row>
       return this;
     }
 
+    public Builder setDimensions(DimensionSpec... dimensions)
+    {
+      this.dimensions = new ArrayList<>(Arrays.asList(dimensions));
+      this.postProcessingFn = null;
+      return this;
+    }
+
     public Builder addAggregator(AggregatorFactory aggregator)
     {
       if (aggregatorSpecs == null) {
@@ -910,6 +917,13 @@ public class GroupByQuery extends BaseQuery<Row>
       return this;
     }
 
+    public Builder setAggregatorSpecs(AggregatorFactory... aggregatorSpecs)
+    {
+      this.aggregatorSpecs = new ArrayList<>(Arrays.asList(aggregatorSpecs));
+      this.postProcessingFn = null;
+      return this;
+    }
+
     public Builder setPostAggregatorSpecs(List<PostAggregator> postAggregatorSpecs)
     {
       this.postAggregatorSpecs = Lists.newArrayList(postAggregatorSpecs);
diff --git a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java
index af3b505..8a19d19 100644
--- a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java
+++ b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java
@@ -196,7 +196,7 @@ public class GroupByStrategyV1 implements GroupByStrategy
 
     // We need the inner incremental index to have all the columns required by the outer query
     final GroupByQuery innerQuery = new GroupByQuery.Builder(subquery)
-        .setAggregatorSpecs(Lists.newArrayList(aggs))
+        .setAggregatorSpecs(ImmutableList.copyOf(aggs))
         .setInterval(subquery.getIntervals())
         .setPostAggregatorSpecs(Lists.newArrayList())
         .build();
diff --git a/processing/src/test/java/io/druid/collections/bitmap/ConciseBitmapFactoryTest.java b/processing/src/test/java/io/druid/collections/bitmap/ConciseBitmapFactoryTest.java
index 6148a2c..8c90c5e 100644
--- a/processing/src/test/java/io/druid/collections/bitmap/ConciseBitmapFactoryTest.java
+++ b/processing/src/test/java/io/druid/collections/bitmap/ConciseBitmapFactoryTest.java
@@ -22,13 +22,13 @@ package io.druid.collections.bitmap;
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import io.druid.extendedset.intset.ConciseSet;
 import io.druid.extendedset.intset.ImmutableConciseSet;
 import junit.framework.Assert;
 import org.junit.Test;
 
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Set;
 
 public class ConciseBitmapFactoryTest
@@ -40,7 +40,7 @@ public class ConciseBitmapFactoryTest
 
     ImmutableBitmap bitmap = factory.union(
         Iterables.transform(
-            Lists.newArrayList(new WrappedConciseBitmap()),
+            Collections.singletonList(new WrappedConciseBitmap()),
             new Function<WrappedConciseBitmap, ImmutableBitmap>()
             {
               @Override
diff --git a/processing/src/test/java/io/druid/collections/bitmap/RoaringBitmapFactoryTest.java b/processing/src/test/java/io/druid/collections/bitmap/RoaringBitmapFactoryTest.java
index d7da4d5..600c483 100644
--- a/processing/src/test/java/io/druid/collections/bitmap/RoaringBitmapFactoryTest.java
+++ b/processing/src/test/java/io/druid/collections/bitmap/RoaringBitmapFactoryTest.java
@@ -21,12 +21,12 @@ package io.druid.collections.bitmap;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import org.junit.Assert;
 import org.junit.Test;
 import org.roaringbitmap.IntIterator;
 
 import java.util.Arrays;
+import java.util.Collections;
 
 public class RoaringBitmapFactoryTest
 {
@@ -58,7 +58,7 @@ public class RoaringBitmapFactoryTest
 
     ImmutableBitmap bitmap = factory.union(
         Iterables.transform(
-            Lists.newArrayList(new WrappedRoaringBitmap()),
+            Collections.singletonList(new WrappedRoaringBitmap()),
             new Function<WrappedRoaringBitmap, ImmutableBitmap>()
             {
               @Override
diff --git a/processing/src/test/java/io/druid/granularity/QueryGranularityTest.java b/processing/src/test/java/io/druid/granularity/QueryGranularityTest.java
index 649aa4f..04dad0e 100644
--- a/processing/src/test/java/io/druid/granularity/QueryGranularityTest.java
+++ b/processing/src/test/java/io/druid/granularity/QueryGranularityTest.java
@@ -48,6 +48,7 @@ import org.junit.Test;
 
 import java.net.URL;
 import java.net.URLClassLoader;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.TimeZone;
@@ -630,7 +631,7 @@ public class QueryGranularityTest
     final DateTime baseTime = DateTimes.of("2011-01-01T00:00:00.000Z");
 
     assertSameInterval(
-        Lists.newArrayList(baseTime),
+        Collections.singletonList(baseTime),
         Granularities.ALL.getIterable(new Interval(baseTime, baseTime.plus(Days.days(3))))
     );
   }
@@ -641,7 +642,7 @@ public class QueryGranularityTest
     final DateTime baseTime = DateTimes.of("2011-01-01T09:38:02.992Z");
 
     assertSameInterval(
-        Lists.newArrayList(baseTime),
+        Collections.singletonList(baseTime),
         Granularities.ALL.getIterable(new Interval(baseTime, baseTime.plus(Days.days(3))))
     );
   }
diff --git a/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java b/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java
index 737c7d1..b9f18a6 100644
--- a/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java
+++ b/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java
@@ -20,7 +20,6 @@
 package io.druid.query;
 
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ListenableFuture;
 import io.druid.java.util.common.guava.Sequence;
 import io.druid.java.util.common.guava.Sequences;
@@ -50,7 +49,7 @@ public class AsyncQueryRunnerTest
     query = Druids.newTimeseriesQueryBuilder()
               .dataSource("test")
               .intervals("2014/2015")
-              .aggregators(Lists.newArrayList(new CountAggregatorFactory("count")))
+              .aggregators(Collections.singletonList(new CountAggregatorFactory("count")))
               .build();
   }
   
@@ -65,7 +64,7 @@ public class AsyncQueryRunnerTest
       {
         try {
           latch.await();
-          return Sequences.simple(Lists.newArrayList(1));
+          return Sequences.simple(Collections.singletonList(1));
         }
         catch (InterruptedException ex) {
           throw new RuntimeException(ex);
@@ -81,7 +80,7 @@ public class AsyncQueryRunnerTest
 
     Sequence lazy = asyncRunner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP);
     latch.countDown();
-    Assert.assertEquals(Lists.newArrayList(1), lazy.toList());
+    Assert.assertEquals(Collections.singletonList(1), lazy.toList());
   }
   
   @Test(timeout = TEST_TIMEOUT)
diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java
index 9a1b79a..191ef49 100644
--- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java
+++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java
@@ -35,6 +35,7 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Queue;
@@ -122,7 +123,7 @@ public class ChainedExecutionQueryRunnerTest
     TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
                                   .dataSource("test")
                                   .intervals("2014/2015")
-                                  .aggregators(Lists.newArrayList(new CountAggregatorFactory("count")))
+                                  .aggregators(Collections.singletonList(new CountAggregatorFactory("count")))
                                   .build();
     final Sequence seq = chainedRunner.run(QueryPlus.wrap(query), context);
 
@@ -247,7 +248,7 @@ public class ChainedExecutionQueryRunnerTest
     TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
                                   .dataSource("test")
                                   .intervals("2014/2015")
-                                  .aggregators(Lists.newArrayList(new CountAggregatorFactory("count")))
+                                  .aggregators(Collections.singletonList(new CountAggregatorFactory("count")))
                                   .context(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 100, "queryId", "test"))
                                   .build();
     final Sequence seq = chainedRunner.run(QueryPlus.wrap(query), context);
@@ -345,7 +346,7 @@ public class ChainedExecutionQueryRunnerTest
 
       hasCompleted = true;
       stop.countDown();
-      return Sequences.simple(Lists.newArrayList(123));
+      return Sequences.simple(Collections.singletonList(123));
     }
   }
 }
diff --git a/processing/src/test/java/io/druid/query/DataSourceTest.java b/processing/src/test/java/io/druid/query/DataSourceTest.java
index 29b2dc9..1bf0568 100644
--- a/processing/src/test/java/io/druid/query/DataSourceTest.java
+++ b/processing/src/test/java/io/druid/query/DataSourceTest.java
@@ -29,7 +29,6 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.Arrays;
 
 public class DataSourceTest
 {
@@ -65,13 +64,8 @@ public class DataSourceTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
diff --git a/processing/src/test/java/io/druid/query/DoubleStorageTest.java b/processing/src/test/java/io/druid/query/DoubleStorageTest.java
index 66f94bc..d9a64ea 100644
--- a/processing/src/test/java/io/druid/query/DoubleStorageTest.java
+++ b/processing/src/test/java/io/druid/query/DoubleStorageTest.java
@@ -21,7 +21,6 @@ package io.druid.query;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import io.druid.data.input.impl.DimensionsSpec;
 import io.druid.data.input.impl.InputRowParser;
@@ -300,7 +299,7 @@ public class DoubleStorageTest
         ImmutableList.of(TIME_COLUMN, DIM_NAME, DIM_FLOAT_NAME),
         getStreamOfEvents().collect(Collectors.toList())
     );
-    List<ScanResultValue> expectedResults = Lists.newArrayList(expectedScanResult);
+    List<ScanResultValue> expectedResults = Collections.singletonList(expectedScanResult);
     ScanQueryRunnerTest.verify(expectedResults, results);
   }
 
diff --git a/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java b/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java
index 258de5e..0bc0ea0 100644
--- a/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java
+++ b/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java
@@ -20,7 +20,6 @@
 package io.druid.query;
 
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.java.util.common.guava.Sequences;
 import io.druid.query.Druids.TimeseriesQueryBuilder;
@@ -45,7 +44,7 @@ public class IntervalChunkingQueryRunnerTest
   {
     queryBuilder = Druids.newTimeseriesQueryBuilder()
               .dataSource("test")
-              .aggregators(Lists.newArrayList(new CountAggregatorFactory("count")));
+              .aggregators(Collections.singletonList(new CountAggregatorFactory("count")));
   }
 
   @Before
diff --git a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java
index a8d0a2c..25980a3 100644
--- a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java
+++ b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java
@@ -153,8 +153,8 @@ public class MultiValuedDimensionTest
         .setDataSource("xx")
         .setQuerySegmentSpec(new LegacySegmentSpec("1970/3000"))
         .setGranularity(Granularities.ALL)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("tags", "tags")))
-        .setAggregatorSpecs(Collections.singletonList(new CountAggregatorFactory("count")))
+        .setDimensions(new DefaultDimensionSpec("tags", "tags"))
+        .setAggregatorSpecs(new CountAggregatorFactory("count"))
         .build();
 
     Sequence<Row> result = helper.runQueryOnSegmentsObjs(
@@ -187,8 +187,8 @@ public class MultiValuedDimensionTest
         .setDataSource("xx")
         .setQuerySegmentSpec(new LegacySegmentSpec("1970/3000"))
         .setGranularity(Granularities.ALL)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("tags", "tags")))
-        .setAggregatorSpecs(Collections.singletonList(new CountAggregatorFactory("count")))
+        .setDimensions(new DefaultDimensionSpec("tags", "tags"))
+        .setAggregatorSpecs(new CountAggregatorFactory("count"))
         .setDimFilter(new SelectorDimFilter("tags", "t3", null))
         .build();
 
@@ -219,15 +219,8 @@ public class MultiValuedDimensionTest
         .setDataSource("xx")
         .setQuerySegmentSpec(new LegacySegmentSpec("1970/3000"))
         .setGranularity(Granularities.ALL)
-        .setDimensions(
-            Lists.newArrayList(
-                new RegexFilteredDimensionSpec(
-                    new DefaultDimensionSpec("tags", "tags"),
-                    "t3"
-                )
-            )
-        )
-        .setAggregatorSpecs(Collections.singletonList(new CountAggregatorFactory("count")))
+        .setDimensions(new RegexFilteredDimensionSpec(new DefaultDimensionSpec("tags", "tags"), "t3"))
+        .setAggregatorSpecs(new CountAggregatorFactory("count"))
         .setDimFilter(new SelectorDimFilter("tags", "t3", null))
         .build();
 
diff --git a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java
index 4b08c7d..d15e5f9 100644
--- a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java
+++ b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java
@@ -177,7 +177,7 @@ public class TimewarpOperatorTest
               .build();
 
     Assert.assertEquals(
-        Lists.newArrayList(
+        Collections.singletonList(
             new Result<>(
                 DateTimes.of("2014-08-02"),
                 new TimeBoundaryResultValue(ImmutableMap.<String, Object>of("maxTime", DateTimes.of("2014-08-02")))
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 84c6469..52cfcc7 100644
--- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java
+++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java
@@ -90,6 +90,7 @@ import java.io.InputStream;
 import java.lang.reflect.Array;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -326,7 +327,7 @@ public class AggregationTestHelper
   {
     File segmentDir = tempFolder.newFolder();
     createIndex(inputDataFile, parserJson, aggregators, segmentDir, minTimestamp, gran, maxRowCount);
-    return runQueryOnSegments(Lists.newArrayList(segmentDir), groupByQueryJson);
+    return runQueryOnSegments(Collections.singletonList(segmentDir), groupByQueryJson);
   }
 
   public Sequence<Row> createIndexAndRunQueryOnSegment(
@@ -341,7 +342,7 @@ public class AggregationTestHelper
   {
     File segmentDir = tempFolder.newFolder();
     createIndex(inputDataStream, parserJson, aggregators, segmentDir, minTimestamp, gran, maxRowCount);
-    return runQueryOnSegments(Lists.newArrayList(segmentDir), groupByQueryJson);
+    return runQueryOnSegments(Collections.singletonList(segmentDir), groupByQueryJson);
   }
 
   public void createIndex(
diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregatorUtilTest.java b/processing/src/test/java/io/druid/query/aggregation/AggregatorUtilTest.java
index d230b88..4d1fded 100644
--- a/processing/src/test/java/io/druid/query/aggregation/AggregatorUtilTest.java
+++ b/processing/src/test/java/io/druid/query/aggregation/AggregatorUtilTest.java
@@ -31,6 +31,7 @@ import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import static io.druid.query.QueryRunnerTestHelper.dependentPostAggMetric;
@@ -171,7 +172,7 @@ public class AggregatorUtilTest
     );
 
     Assert.assertEquals(
-        new Pair(Lists.newArrayList(agg1, agg2), Lists.newArrayList(postAgg2)), AggregatorUtil.condensedAggregators(
+        new Pair(Lists.newArrayList(agg1, agg2), Collections.singletonList(postAgg2)), AggregatorUtil.condensedAggregators(
         Lists.newArrayList(agg1, agg2),
         Lists.newArrayList(postAgg1, postAgg2),
         "postAgg"
@@ -206,7 +207,7 @@ public class AggregatorUtilTest
     );
 
     Assert.assertEquals(
-        new Pair(Lists.newArrayList(agg1, agg2), Lists.newArrayList(postAgg2)), AggregatorUtil.condensedAggregators(
+        new Pair(Lists.newArrayList(agg1, agg2), Collections.singletonList(postAgg2)), AggregatorUtil.condensedAggregators(
         Lists.newArrayList(agg1, agg2),
         Lists.newArrayList(postAgg1, postAgg2),
         "postAgg"
diff --git a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorBenchmark.java b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorBenchmark.java
index 47eaac5..d61bb58 100644
--- a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorBenchmark.java
+++ b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorBenchmark.java
@@ -26,7 +26,6 @@ import com.google.common.base.Function;
 import com.google.common.collect.ContiguousSet;
 import com.google.common.collect.DiscreteDomain;
 import com.google.common.collect.FluentIterable;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Range;
 import io.druid.query.ColumnSelectorPlus;
 import io.druid.query.aggregation.cardinality.types.CardinalityAggregatorColumnSelectorStrategy;
@@ -36,6 +35,7 @@ import io.druid.query.dimension.DimensionSpec;
 import io.druid.segment.DimensionSelector;
 
 import java.nio.ByteBuffer;
+import java.util.Collections;
 import java.util.List;
 
 /**
@@ -93,9 +93,7 @@ public class CardinalityAggregatorBenchmark extends SimpleBenchmark
         dim1
     );
 
-    selectorList = Lists.newArrayList(
-        (DimensionSelector) dim1
-    );
+    selectorList = Collections.singletonList((DimensionSelector) dim1);
 
     dimInfos = new ColumnSelectorPlus[] {dimInfo1};
 
@@ -103,7 +101,7 @@ public class CardinalityAggregatorBenchmark extends SimpleBenchmark
 
     CardinalityAggregatorFactory factory = new CardinalityAggregatorFactory(
         "billy",
-        Lists.newArrayList(new DefaultDimensionSpec("dim1", "dim1")),
+        Collections.singletonList(new DefaultDimensionSpec("dim1", "dim1")),
         byRow
     );
 
diff --git a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java
index 3f7fcda..9d54852 100644
--- a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java
+++ b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java
@@ -53,6 +53,7 @@ import org.junit.Test;
 
 import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -480,17 +481,17 @@ public class CardinalityAggregatorTest
   @Test
   public void testCombineRows()
   {
-    List<DimensionSelector> selector1 = Lists.newArrayList((DimensionSelector) dim1);
-    List<DimensionSelector> selector2 = Lists.newArrayList((DimensionSelector) dim2);
-    List<ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>> dimInfo1 = Lists.newArrayList(
-        new ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>(
+    List<DimensionSelector> selector1 = Collections.singletonList(dim1);
+    List<DimensionSelector> selector2 = Collections.singletonList(dim2);
+    List<ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>> dimInfo1 = Collections.singletonList(
+        new ColumnSelectorPlus<>(
             dimSpec1.getDimension(),
             dimSpec1.getOutputName(),
             new StringCardinalityAggregatorColumnSelectorStrategy(), dim1
         )
     );
-    List<ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>> dimInfo2 = Lists.newArrayList(
-        new ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>(
+    List<ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>> dimInfo2 = Collections.singletonList(
+        new ColumnSelectorPlus<>(
             dimSpec1.getDimension(),
             dimSpec1.getOutputName(),
             new StringCardinalityAggregatorColumnSelectorStrategy(), dim2
@@ -525,18 +526,18 @@ public class CardinalityAggregatorTest
   @Test
   public void testCombineValues()
   {
-    List<DimensionSelector> selector1 = Lists.newArrayList((DimensionSelector) dim1);
-    List<DimensionSelector> selector2 = Lists.newArrayList((DimensionSelector) dim2);
+    List<DimensionSelector> selector1 = Collections.singletonList(dim1);
+    List<DimensionSelector> selector2 = Collections.singletonList(dim2);
 
-    List<ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>> dimInfo1 = Lists.newArrayList(
-        new ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>(
+    List<ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>> dimInfo1 = Collections.singletonList(
+        new ColumnSelectorPlus<>(
             dimSpec1.getDimension(),
             dimSpec1.getOutputName(),
             new StringCardinalityAggregatorColumnSelectorStrategy(), dim1
         )
     );
-    List<ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>> dimInfo2 = Lists.newArrayList(
-        new ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>(
+    List<ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>> dimInfo2 = Collections.singletonList(
+        new ColumnSelectorPlus<>(
             dimSpec1.getDimension(),
             dimSpec1.getOutputName(),
             new StringCardinalityAggregatorColumnSelectorStrategy(), dim2
diff --git a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregationTest.java
index 6192c5a..c4df309 100644
--- a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregationTest.java
+++ b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregationTest.java
@@ -36,6 +36,7 @@ import org.junit.runners.Parameterized;
 
 import java.io.File;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 
 @RunWith(Parameterized.class)
@@ -65,7 +66,7 @@ public class HyperUniquesAggregationTest
   public void testIngestAndQuery() throws Exception
   {
     AggregationTestHelper helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
-        Lists.newArrayList(new AggregatorsModule()),
+        Collections.singletonList(new AggregatorsModule()),
         config,
         tempFolder
     );
@@ -126,7 +127,7 @@ public class HyperUniquesAggregationTest
   public void testIngestAndQueryPrecomputedHll() throws Exception
   {
     AggregationTestHelper helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
-            Lists.newArrayList(new AggregatorsModule()),
+            Collections.singletonList(new AggregatorsModule()),
             config,
             tempFolder
     );
diff --git a/processing/src/test/java/io/druid/query/aggregation/post/FinalizingFieldAccessPostAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/post/FinalizingFieldAccessPostAggregatorTest.java
index 45e5e95..7c1e69c 100644
--- a/processing/src/test/java/io/druid/query/aggregation/post/FinalizingFieldAccessPostAggregatorTest.java
+++ b/processing/src/test/java/io/druid/query/aggregation/post/FinalizingFieldAccessPostAggregatorTest.java
@@ -189,7 +189,7 @@ public class FinalizingFieldAccessPostAggregatorTest
   public void testIngestAndQueryWithArithmeticPostAggregator() throws Exception
   {
     AggregationTestHelper helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
-        Lists.newArrayList(new AggregatorsModule()),
+        Collections.singletonList(new AggregatorsModule()),
         GroupByQueryRunnerTest.testConfigs().get(0),
         tempFoler
     );
diff --git a/processing/src/test/java/io/druid/query/cache/CacheKeyBuilderTest.java b/processing/src/test/java/io/druid/query/cache/CacheKeyBuilderTest.java
index b606cde..e2b1075 100644
--- a/processing/src/test/java/io/druid/query/cache/CacheKeyBuilderTest.java
+++ b/processing/src/test/java/io/druid/query/cache/CacheKeyBuilderTest.java
@@ -27,6 +27,7 @@ import org.junit.Test;
 
 import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import static org.junit.Assert.assertArrayEquals;
@@ -260,13 +261,13 @@ public class CacheKeyBuilderTest
 
     keys.add(
         new CacheKeyBuilder((byte) 10)
-            .appendCacheables(Lists.newArrayList(testtest))
+            .appendCacheables(Collections.singletonList(testtest))
             .build()
     );
 
     keys.add(
         new CacheKeyBuilder((byte) 10)
-            .appendCacheables(Lists.newArrayList(testtest))
+            .appendCacheables(Collections.singletonList(testtest))
             .appendCacheables(Lists.newArrayList())
             .build()
     );
@@ -274,7 +275,7 @@ public class CacheKeyBuilderTest
     keys.add(
         new CacheKeyBuilder((byte) 10)
             .appendCacheables(Lists.newArrayList())
-            .appendCacheables(Lists.newArrayList(testtest))
+            .appendCacheables(Collections.singletonList(testtest))
             .build()
     );
 
@@ -298,17 +299,17 @@ public class CacheKeyBuilderTest
         .build();
 
     byte[] key2 = new CacheKeyBuilder((byte) 10)
-        .appendStrings(Lists.newArrayList(""))
+        .appendStrings(Collections.singletonList(""))
         .build();
 
     assertFalse(Arrays.equals(key1, key2));
 
     key1 = new CacheKeyBuilder((byte) 10)
-        .appendStrings(Lists.newArrayList(""))
+        .appendStrings(Collections.singletonList(""))
         .build();
 
     key2 = new CacheKeyBuilder((byte) 10)
-        .appendStrings(Lists.newArrayList((String) null))
+        .appendStrings(Collections.singletonList((String) null))
         .build();
 
     assertArrayEquals(key1, key2);
@@ -322,7 +323,7 @@ public class CacheKeyBuilderTest
         .build();
 
     final byte[] key2 = new CacheKeyBuilder((byte) 10)
-        .appendCacheables(Lists.newArrayList((Cacheable) null))
+        .appendCacheables(Collections.singletonList((Cacheable) null))
         .build();
 
     assertFalse(Arrays.equals(key1, key2));
diff --git a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java
index 9cc2f2c..ca69e82 100644
--- a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java
@@ -20,10 +20,9 @@
 package io.druid.query.groupby;
 
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.java.util.common.Intervals;
 import io.druid.java.util.common.granularity.PeriodGranularity;
+import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.query.CachingEmitter;
 import io.druid.query.DefaultQueryMetricsTest;
 import io.druid.query.DruidMetrics;
@@ -39,7 +38,6 @@ import org.joda.time.Period;
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.Map;
 
@@ -59,31 +57,17 @@ public class DefaultGroupByQueryMetricsTest
     GroupByQuery.Builder builder = GroupByQuery
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(
-            Lists.newArrayList(
-                new ExtractionDimensionSpec(
-                    "quality",
-                    "alias",
-                    new LookupExtractionFn(
-                        new MapLookupExtractor(
-                            ImmutableMap.of(
-                                "mezzanine",
-                                "mezzanine0"
-                            ),
-                            false
-                        ), false, null, true,
-                        false
-                    )
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
+        .setInterval("2011-04-02/2011-04-04").setDimensions(new ExtractionDimensionSpec(
+            "quality",
+            "alias",
+            new LookupExtractionFn(
+                new MapLookupExtractor(ImmutableMap.of("mezzanine", "mezzanine0"), false),
+                false,
+                null,
+                true,
+                false
             )
-        )
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setDimFilter(new SelectorDimFilter("quality", "mezzanine", null))
         .setContext(ImmutableMap.of("bySegment", true));
diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java
index 0079400..abf4825 100644
--- a/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java
@@ -466,12 +466,8 @@ public class GroupByLimitPushDownInsufficientBufferTest
         .builder()
         .setDataSource("blah")
         .setQuerySegmentSpec(intervalSpec)
-        .setDimensions(Lists.newArrayList(
-            new DefaultDimensionSpec("dimA", null)
-        ))
-        .setAggregatorSpecs(
-            Collections.singletonList(new LongSumAggregatorFactory("metA", "metA"))
-        )
+        .setDimensions(new DefaultDimensionSpec("dimA", null))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("metA", "metA"))
         .setLimitSpec(
             new DefaultLimitSpec(
                 Collections.singletonList(new OrderByColumnSpec("dimA", OrderByColumnSpec.Direction.DESCENDING)),
@@ -556,12 +552,8 @@ public class GroupByLimitPushDownInsufficientBufferTest
         .builder()
         .setDataSource("blah")
         .setQuerySegmentSpec(intervalSpec)
-        .setDimensions(Lists.newArrayList(
-            new DefaultDimensionSpec("dimA", null)
-        ))
-        .setAggregatorSpecs(
-            Collections.singletonList(new LongSumAggregatorFactory("metA", "metA"))
-        )
+        .setDimensions(new DefaultDimensionSpec("dimA", null))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("metA", "metA"))
         .setLimitSpec(
             new DefaultLimitSpec(
                 Collections.singletonList(
diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java
index 7dd36b3..0f04bb4 100644
--- a/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java
@@ -506,18 +506,30 @@ public class GroupByLimitPushDownMultiNodeMergeTest
         .setDataSource("blah")
         .setQuerySegmentSpec(intervalSpec)
         .setVirtualColumns(
-            new ExpressionVirtualColumn("d0:v", "timestamp_extract(\"__time\",'YEAR','UTC')", ValueType.LONG, TestExprMacroTable.INSTANCE),
-            new ExpressionVirtualColumn("d1:v", "timestamp_extract(\"__time\",'MONTH','UTC')", ValueType.LONG, TestExprMacroTable.INSTANCE),
-            new ExpressionVirtualColumn("d2:v", "timestamp_extract(\"__time\",'DAY','UTC')", ValueType.LONG, TestExprMacroTable.INSTANCE)
+            new ExpressionVirtualColumn(
+                "d0:v",
+                "timestamp_extract(\"__time\",'YEAR','UTC')",
+                ValueType.LONG,
+                TestExprMacroTable.INSTANCE
+            ),
+            new ExpressionVirtualColumn(
+                "d1:v",
+                "timestamp_extract(\"__time\",'MONTH','UTC')",
+                ValueType.LONG,
+                TestExprMacroTable.INSTANCE
+            ),
+            new ExpressionVirtualColumn(
+                "d2:v",
+                "timestamp_extract(\"__time\",'DAY','UTC')",
+                ValueType.LONG,
+                TestExprMacroTable.INSTANCE
+            )
         )
-        .setDimensions(Lists.newArrayList(
+        .setDimensions(
             new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG),
             new DefaultDimensionSpec("d1:v", "d1", ValueType.LONG),
             new DefaultDimensionSpec("d2:v", "d2", ValueType.LONG)
-        ))
-        .setAggregatorSpecs(
-            Collections.singletonList(new CountAggregatorFactory("a0"))
-        )
+        ).setAggregatorSpecs(new CountAggregatorFactory("a0"))
         .setLimitSpec(
             ls2
         )
@@ -617,7 +629,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
         .builder()
         .setDataSource("blah")
         .setQuerySegmentSpec(intervalSpec)
-        .setDimensions(Lists.newArrayList(
+        .setDimensions(
             new DefaultDimensionSpec("dimA", "dimA"),
             new ExtractionDimensionSpec(
                 Column.TIME_COLUMN_NAME,
@@ -631,10 +643,8 @@ public class GroupByLimitPushDownMultiNodeMergeTest
                     true
                 )
             )
-        ))
-        .setAggregatorSpecs(
-            Collections.singletonList(new LongSumAggregatorFactory("metASum", "metA"))
         )
+        .setAggregatorSpecs(new LongSumAggregatorFactory("metASum", "metA"))
         .setLimitSpec(
             new DefaultLimitSpec(
                 Arrays.asList(
diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java
index 447d6fe..6ec5875 100644
--- a/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java
@@ -321,12 +321,8 @@ public class GroupByMultiSegmentTest
         .builder()
         .setDataSource("blah")
         .setQuerySegmentSpec(intervalSpec)
-        .setDimensions(Lists.newArrayList(
-            new DefaultDimensionSpec("dimA", null)
-        ))
-        .setAggregatorSpecs(
-            Collections.singletonList(new LongSumAggregatorFactory("metA", "metA"))
-        )
+        .setDimensions(new DefaultDimensionSpec("dimA", null))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("metA", "metA"))
         .setLimitSpec(
             new DefaultLimitSpec(
                 Collections.singletonList(new OrderByColumnSpec("dimA", OrderByColumnSpec.Direction.ASCENDING)),
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 87b1906..6ec7e8b 100644
--- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java
@@ -50,6 +50,7 @@ import org.junit.runners.Parameterized.Parameters;
 
 import java.nio.ByteBuffer;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
@@ -228,7 +229,7 @@ public class GroupByQueryMergeBufferTest
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setGranularity(Granularities.ALL)
         .setInterval(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows")))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"))
         .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT))
         .build();
 
@@ -249,14 +250,14 @@ public class GroupByQueryMergeBufferTest
                             .setDataSource(QueryRunnerTestHelper.dataSource)
                             .setInterval(QueryRunnerTestHelper.firstToThird)
                             .setGranularity(Granularities.ALL)
-                            .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-                            .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount))
+                            .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+                            .setAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.rowsCount))
                             .build()
             )
         )
         .setGranularity(Granularities.ALL)
         .setInterval(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows")))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"))
         .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT))
         .build();
 
@@ -279,23 +280,23 @@ public class GroupByQueryMergeBufferTest
                                             .setDataSource(QueryRunnerTestHelper.dataSource)
                                             .setInterval(QueryRunnerTestHelper.firstToThird)
                                             .setGranularity(Granularities.ALL)
-                                            .setDimensions(Lists.newArrayList(
+                                            .setDimensions(
                                                 new DefaultDimensionSpec("quality", "alias"),
                                                 new DefaultDimensionSpec("market", null)
-                                            ))
-                                            .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount))
+                                            )
+                                            .setAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.rowsCount))
                                             .build()
                             )
                             .setInterval(QueryRunnerTestHelper.firstToThird)
                             .setGranularity(Granularities.ALL)
-                            .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-                            .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount))
+                            .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+                            .setAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.rowsCount))
                             .build()
             )
         )
         .setGranularity(Granularities.ALL)
         .setInterval(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows")))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"))
         .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT))
         .build();
 
@@ -326,28 +327,29 @@ public class GroupByQueryMergeBufferTest
                                                                 new DefaultDimensionSpec("market", null),
                                                                 new DefaultDimensionSpec("placement", null)
                                                             ))
-                                                            .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount))
+                                                            .setAggregatorSpecs(Collections.singletonList(
+                                                                QueryRunnerTestHelper.rowsCount))
                                                             .build()
                                             )
                                             .setInterval(QueryRunnerTestHelper.firstToThird)
                                             .setGranularity(Granularities.ALL)
-                                            .setDimensions(Lists.newArrayList(
+                                            .setDimensions(
                                                 new DefaultDimensionSpec("quality", "alias"),
                                                 new DefaultDimensionSpec("market", null)
-                                            ))
-                                            .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount))
+                                            )
+                                            .setAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.rowsCount))
                                             .build()
                             )
                             .setInterval(QueryRunnerTestHelper.firstToThird)
                             .setGranularity(Granularities.ALL)
-                            .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-                            .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount))
+                            .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+                            .setAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.rowsCount))
                             .build()
             )
         )
         .setGranularity(Granularities.ALL)
         .setInterval(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows")))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"))
         .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT))
         .build();
 
diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java
index 32f1927..4394003 100644
--- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java
@@ -21,7 +21,6 @@ package io.druid.query.groupby;
 
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import io.druid.data.input.Row;
 import io.druid.data.input.impl.CSVParseSpec;
@@ -48,7 +47,6 @@ import org.junit.Rule;
 import org.junit.Test;
 
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -67,8 +65,8 @@ public class GroupByQueryRunnerFactoryTest
         .setDataSource("xx")
         .setQuerySegmentSpec(new LegacySegmentSpec("1970/3000"))
         .setGranularity(Granularities.ALL)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("tags", "tags")))
-        .setAggregatorSpecs(Collections.singletonList(new CountAggregatorFactory("count")))
+        .setDimensions(new DefaultDimensionSpec("tags", "tags"))
+        .setAggregatorSpecs(new CountAggregatorFactory("count"))
         .build();
 
     final QueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig());
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 e9791a7..09d4074 100644
--- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java
@@ -56,6 +56,7 @@ import org.junit.runners.Parameterized.Parameters;
 
 import java.nio.ByteBuffer;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.TimeoutException;
 
@@ -193,14 +194,14 @@ public class GroupByQueryRunnerFailureTest
                             .setDataSource(QueryRunnerTestHelper.dataSource)
                             .setInterval(QueryRunnerTestHelper.firstToThird)
                             .setGranularity(Granularities.ALL)
-                            .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-                            .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount))
+                            .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+                            .setAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.rowsCount))
                             .build()
             )
         )
         .setGranularity(Granularities.ALL)
         .setInterval(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows")))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"))
         .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 500))
         .build();
 
@@ -222,23 +223,23 @@ public class GroupByQueryRunnerFailureTest
                                             .setDataSource(QueryRunnerTestHelper.dataSource)
                                             .setInterval(QueryRunnerTestHelper.firstToThird)
                                             .setGranularity(Granularities.ALL)
-                                            .setDimensions(Lists.newArrayList(
+                                            .setDimensions(
                                                 new DefaultDimensionSpec("quality", "alias"),
                                                 new DefaultDimensionSpec("market", null)
-                                            ))
-                                            .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount))
+                                            )
+                                            .setAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.rowsCount))
                                             .build()
                             )
                             .setInterval(QueryRunnerTestHelper.firstToThird)
                             .setGranularity(Granularities.ALL)
-                            .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-                            .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount))
+                            .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+                            .setAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.rowsCount))
                             .build()
             )
         )
         .setGranularity(Granularities.ALL)
         .setInterval(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows")))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"))
         .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 500))
         .build();
 
@@ -256,14 +257,14 @@ public class GroupByQueryRunnerFailureTest
                             .setDataSource(QueryRunnerTestHelper.dataSource)
                             .setInterval(QueryRunnerTestHelper.firstToThird)
                             .setGranularity(Granularities.ALL)
-                            .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-                            .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount))
+                            .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+                            .setAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.rowsCount))
                             .build()
             )
         )
         .setGranularity(Granularities.ALL)
         .setInterval(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows")))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"))
         .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 500))
         .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 dd98105..882cd48 100644
--- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java
@@ -80,7 +80,6 @@ import io.druid.query.aggregation.post.ConstantPostAggregator;
 import io.druid.query.aggregation.post.ExpressionPostAggregator;
 import io.druid.query.aggregation.post.FieldAccessPostAggregator;
 import io.druid.query.dimension.DefaultDimensionSpec;
-import io.druid.query.dimension.DimensionSpec;
 import io.druid.query.dimension.ExtractionDimensionSpec;
 import io.druid.query.dimension.ListFilteredDimensionSpec;
 import io.druid.query.dimension.RegexFilteredDimensionSpec;
@@ -430,15 +429,11 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index"),
-                new FloatSumAggregatorFactory("idxFloat", "indexFloat"),
-                new DoubleSumAggregatorFactory("idxDouble", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new LongSumAggregatorFactory("idx", "index"),
+                            new FloatSumAggregatorFactory("idxFloat", "indexFloat"),
+                            new DoubleSumAggregatorFactory("idxDouble", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -476,12 +471,9 @@ public class GroupByQueryRunnerTest
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("nonexistent0", "alias0"),
-                new ExtractionDimensionSpec("nonexistent1", "alias1", new StringFormatExtractionFn("foo"))
-            )
-        )
-        .setAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.rowsCount))
+            new DefaultDimensionSpec("nonexistent0", "alias0"),
+            new ExtractionDimensionSpec("nonexistent1", "alias1", new StringFormatExtractionFn("foo"))
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -505,13 +497,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setPostAggregatorSpecs(
             ImmutableList.of(
                 new ExpressionPostAggregator("post", "alias + 'x'", null, TestExprMacroTable.INSTANCE)
@@ -569,13 +556,8 @@ public class GroupByQueryRunnerTest
                 TestExprMacroTable.INSTANCE
             )
         )
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("vc", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("vc", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -628,13 +610,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new DurationGranularity(86400L, 0L))
         .build();
 
@@ -674,13 +651,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("alias", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("alias", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
   }
@@ -692,13 +664,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setContext(ImmutableMap.of("sortByDimsFirst", true, "groupByStrategy", "v2"))
         .build();
@@ -743,13 +710,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .setContext(ImmutableMap.of("chunkPeriod", "P1D"))
         .build();
@@ -777,7 +739,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -814,13 +776,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("placementish", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("placementish", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -849,17 +806,9 @@ public class GroupByQueryRunnerTest
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setDimFilter(new SelectorDimFilter("placementish", "a", null))
         .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("placementish", "alias"),
-                new DefaultDimensionSpec("placementish", "alias2")
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+            new DefaultDimensionSpec("placementish", "alias"),
+            new DefaultDimensionSpec("placementish", "alias2")
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -881,16 +830,10 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(
+        .setDimensions(
             new DefaultDimensionSpec("placementish", "alias"),
             new DefaultDimensionSpec("quality", "quality")
-        ))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -1106,16 +1049,10 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(
+        .setDimensions(
             new DefaultDimensionSpec("quality", "quality"),
             new DefaultDimensionSpec("placementish", "alias")
-        ))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -1331,13 +1268,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setContext(ImmutableMap.of("maxResults", 1))
         .build();
@@ -1380,13 +1312,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 60000))
         .build();
@@ -1424,13 +1351,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setContext(ImmutableMap.of("maxOnDiskStorage", 0, "bufferGrouperMaxSize", 1))
         .build();
@@ -1474,13 +1396,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setContext(ImmutableMap.of("maxOnDiskStorage", 0, "maxMergingDictionarySize", 1))
         .build();
@@ -1524,13 +1441,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setContext(ImmutableMap.of("maxOnDiskStorage", 1, "maxMergingDictionarySize", 1))
         .build();
@@ -1579,7 +1491,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setLimitSpec(
             new DefaultLimitSpec(
@@ -1599,8 +1511,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList())
-        .setAggregatorSpecs(ImmutableList.of(new CountAggregatorFactory("count")))
+        .setDimensions(Lists.newArrayList()).setAggregatorSpecs(new CountAggregatorFactory("count"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .setContext(ImmutableMap.of("maxOnDiskStorage", 0, "bufferGrouperMaxSize", 0))
         .build();
@@ -1637,22 +1548,11 @@ public class GroupByQueryRunnerTest
     GroupByQuery query = GroupByQuery
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                new ExtractionDimensionSpec(
-                    "quality",
-                    "alias",
-                    new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, false, false)
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird).setDimensions(new ExtractionDimensionSpec(
+            "quality",
+            "alias",
+            new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, false, false)
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -1715,22 +1615,11 @@ public class GroupByQueryRunnerTest
     GroupByQuery query = GroupByQuery
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                new ExtractionDimensionSpec(
-                    "quality",
-                    "alias",
-                    new LookupExtractionFn(new MapLookupExtractor(map, false), true, null, false, false)
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird).setDimensions(new ExtractionDimensionSpec(
+            "quality",
+            "alias",
+            new LookupExtractionFn(new MapLookupExtractor(map, false), true, null, false, false)
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -1793,22 +1682,11 @@ public class GroupByQueryRunnerTest
     GroupByQuery query = GroupByQuery
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                new ExtractionDimensionSpec(
-                    "quality",
-                    "alias",
-                    new LookupExtractionFn(new MapLookupExtractor(map, false), true, null, true, false)
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird).setDimensions(new ExtractionDimensionSpec(
+            "quality",
+            "alias",
+            new LookupExtractionFn(new MapLookupExtractor(map, false), true, null, true, false)
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -1871,22 +1749,11 @@ public class GroupByQueryRunnerTest
     GroupByQuery query = GroupByQuery
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                new ExtractionDimensionSpec(
-                    "quality",
-                    "alias",
-                    new LookupExtractionFn(new MapLookupExtractor(map, false), false, "MISSING", true, false)
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird).setDimensions(new ExtractionDimensionSpec(
+            "quality",
+            "alias",
+            new LookupExtractionFn(new MapLookupExtractor(map, false), false, "MISSING", true, false)
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -1948,22 +1815,11 @@ public class GroupByQueryRunnerTest
     GroupByQuery query = GroupByQuery
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                new ExtractionDimensionSpec(
-                    "quality",
-                    "alias",
-                    new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, true, false)
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird).setDimensions(new ExtractionDimensionSpec(
+            "quality",
+            "alias",
+            new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, true, false)
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -2016,12 +1872,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                QueryRunnerTestHelper.qualityUniques
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, QueryRunnerTestHelper.qualityUniques)
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -2046,15 +1897,10 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new HyperUniquesAggregatorFactory(
-                    "quality_uniques",
-                    "quality_uniques"
-                )
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new HyperUniquesAggregatorFactory(
+            "quality_uniques",
+            "quality_uniques"
+        ))
         .setPostAggregatorSpecs(
             Collections.singletonList(
                 new HyperUniqueFinalizingPostAggregator("quality_uniques", "quality_uniques")
@@ -2084,12 +1930,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                QueryRunnerTestHelper.qualityCardinality
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, QueryRunnerTestHelper.qualityCardinality)
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -2114,13 +1955,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("market", "market")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new LongFirstAggregatorFactory("first", "index"),
-                new LongLastAggregatorFactory("last", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("market", "market"))
+        .setAggregatorSpecs(new LongFirstAggregatorFactory("first", "index"),
+                            new LongLastAggregatorFactory("last", "index"))
         .setGranularity(QueryRunnerTestHelper.monthGran)
         .build();
 
@@ -2150,16 +1987,12 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.emptyInterval)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("market", "market")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                QueryRunnerTestHelper.indexLongSum,
-                QueryRunnerTestHelper.qualityCardinality,
-                new LongFirstAggregatorFactory("first", "index"),
-                new LongLastAggregatorFactory("last", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("market", "market"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            QueryRunnerTestHelper.indexLongSum,
+                            QueryRunnerTestHelper.qualityCardinality,
+                            new LongFirstAggregatorFactory("first", "index"),
+                            new LongLastAggregatorFactory("last", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -2189,18 +2022,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
-        .setDimensions(
-            Lists.newArrayList(
-                new ExtractionDimensionSpec("quality", "alias", nullExtractionFn)
-            )
-        )
+        .setDimensions(new ExtractionDimensionSpec("quality", "alias", nullExtractionFn))
         .build();
 
     List<Row> expectedResults = Arrays.asList(
@@ -2258,18 +2082,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
-        .setDimensions(
-            Lists.newArrayList(
-                new ExtractionDimensionSpec("quality", "alias", emptyStringExtractionFn)
-            )
-        )
+        .setDimensions(new ExtractionDimensionSpec("quality", "alias", emptyStringExtractionFn))
         .build();
 
     List<Row> expectedResults = Arrays.asList(
@@ -2307,23 +2122,11 @@ public class GroupByQueryRunnerTest
     GroupByQuery query = GroupByQuery.builder()
                                      .setDataSource(QueryRunnerTestHelper.dataSource)
                                      .setInterval("2011-03-31T00:00:00-07:00/2011-04-02T00:00:00-07:00")
-                                     .setDimensions(
-                                         Lists.newArrayList(
-                                             (DimensionSpec) new DefaultDimensionSpec(
-                                                 "quality",
-                                                 "alias"
-                                             )
-                                         )
-                                     )
-                                     .setAggregatorSpecs(
-                                         Arrays.asList(
-                                             QueryRunnerTestHelper.rowsCount,
-                                             new LongSumAggregatorFactory(
-                                                 "idx",
-                                                 "index"
-                                             )
-                                         )
-                                     )
+                                     .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+                                     .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory(
+                                         "idx",
+                                         "index"
+                                     ))
                                      .setGranularity(
                                          new PeriodGranularity(
                                              new Period("P1D"),
@@ -2510,13 +2313,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null));
 
     final GroupByQuery fullQuery = builder.build();
@@ -2532,10 +2330,10 @@ public class GroupByQueryRunnerTest
           {
             // simulate two daily segments
             final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-02/2011-04-03")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
             );
             final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-03/2011-04-04")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
             );
             return new MergeSequence(
                 queryPlus.getQuery().getResultOrdering(),
@@ -2595,13 +2393,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setLimit(limit);
 
@@ -2637,13 +2430,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(Granularities.DAY)
         .setLimit(limit)
         .addOrderByColumn("idx", OrderByColumnSpec.Direction.DESCENDING);
@@ -2694,13 +2482,8 @@ public class GroupByQueryRunnerTest
                 TestExprMacroTable.INSTANCE
             )
         )
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "expr")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "expr"))
         .setGranularity(Granularities.DAY)
         .setLimit(limit)
         .addOrderByColumn("idx", OrderByColumnSpec.Direction.DESCENDING);
@@ -2742,13 +2525,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setLimit(-1);
 
@@ -2821,13 +2599,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setLimitSpec(orderBySpec);
 
@@ -2843,10 +2616,10 @@ public class GroupByQueryRunnerTest
           {
             // simulate two daily segments
             final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-02/2011-04-03")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
             );
             final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-03/2011-04-04")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
             );
             return new MergeSequence(
                 queryPlus.getQuery().getResultOrdering(),
@@ -2869,13 +2642,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .addOrderByColumn("rows")
         .addOrderByColumn("alias", OrderByColumnSpec.Direction.DESCENDING)
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null));
@@ -2966,13 +2734,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .addOrderByColumn("rows", OrderByColumnSpec.Direction.DESCENDING)
         .addOrderByColumn("alias", OrderByColumnSpec.Direction.DESCENDING)
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null));
@@ -3008,13 +2771,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new DoubleSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new DoubleSumAggregatorFactory("idx", "index"))
         .addOrderByColumn("idx", OrderByColumnSpec.Direction.DESCENDING)
         .addOrderByColumn("alias", OrderByColumnSpec.Direction.DESCENDING)
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null));
@@ -3051,13 +2809,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .addOrderByColumn(new OrderByColumnSpec("rows", OrderByColumnSpec.Direction.DESCENDING, StringComparators.NUMERIC))
         .addOrderByColumn(new OrderByColumnSpec("alias", OrderByColumnSpec.Direction.ASCENDING, StringComparators.NUMERIC))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null));
@@ -3091,31 +2844,17 @@ public class GroupByQueryRunnerTest
   {
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setGranularity(QueryRunnerTestHelper.allGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    QueryRunnerTestHelper.marketDimension,
-                    "marketalias"
-                )
-            )
-        )
+        .setGranularity(QueryRunnerTestHelper.allGran).setDimensions(new DefaultDimensionSpec(
+            QueryRunnerTestHelper.marketDimension,
+            "marketalias"
+        ))
         .setInterval(QueryRunnerTestHelper.fullOnInterval)
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "marketalias",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ), 3
-            )
-        )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.rowsCount
+                Collections.singletonList(new OrderByColumnSpec("marketalias", OrderByColumnSpec.Direction.DESCENDING)),
+                3
             )
-        )
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .build();
 
     List<Row> expectedResults = Arrays.asList(
@@ -3152,30 +2891,19 @@ public class GroupByQueryRunnerTest
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setGranularity(QueryRunnerTestHelper.allGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    QueryRunnerTestHelper.marketDimension,
-                    QueryRunnerTestHelper.marketDimension
-                )
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec(
+            QueryRunnerTestHelper.marketDimension,
+            QueryRunnerTestHelper.marketDimension
+        ))
         .setInterval(QueryRunnerTestHelper.fullOnInterval)
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        QueryRunnerTestHelper.marketDimension,
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ), 3
-            )
-        )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.rowsCount
+                Collections.singletonList(
+                    new OrderByColumnSpec(QueryRunnerTestHelper.marketDimension, OrderByColumnSpec.Direction.DESCENDING)
+                ),
+                3
             )
-        )
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .build();
 
     List<Row> expectedResults = Arrays.asList(
@@ -3200,32 +2928,21 @@ public class GroupByQueryRunnerTest
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setGranularity(QueryRunnerTestHelper.allGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    QueryRunnerTestHelper.marketDimension,
-                    QueryRunnerTestHelper.marketDimension
-                )
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec(
+            QueryRunnerTestHelper.marketDimension,
+            QueryRunnerTestHelper.marketDimension
+        ))
         .setInterval(QueryRunnerTestHelper.fullOnInterval)
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        QueryRunnerTestHelper.uniqueMetric,
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ), 3
-            )
-        )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.qualityUniques
+                Collections.singletonList(
+                    new OrderByColumnSpec(QueryRunnerTestHelper.uniqueMetric, OrderByColumnSpec.Direction.DESCENDING)
+                ),
+                3
             )
-        )
+        ).setAggregatorSpecs(QueryRunnerTestHelper.qualityUniques)
         .setPostAggregatorSpecs(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new HyperUniqueFinalizingPostAggregator(
                     QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric,
                     QueryRunnerTestHelper.uniqueMetric
@@ -3274,38 +2991,23 @@ public class GroupByQueryRunnerTest
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setGranularity(QueryRunnerTestHelper.allGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    QueryRunnerTestHelper.marketDimension,
-                    QueryRunnerTestHelper.marketDimension
-                )
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec(
+            QueryRunnerTestHelper.marketDimension,
+            QueryRunnerTestHelper.marketDimension
+        ))
         .setInterval(QueryRunnerTestHelper.fullOnInterval)
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        QueryRunnerTestHelper.uniqueMetric,
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ), 3
-            )
-        )
-        .setHavingSpec(
-            new GreaterThanHavingSpec(
-                QueryRunnerTestHelper.uniqueMetric,
-                8
-            )
-        )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.qualityUniques
+                Collections.singletonList(
+                    new OrderByColumnSpec(QueryRunnerTestHelper.uniqueMetric, OrderByColumnSpec.Direction.DESCENDING)
+                ),
+                3
             )
         )
+        .setHavingSpec(new GreaterThanHavingSpec(QueryRunnerTestHelper.uniqueMetric, 8))
+        .setAggregatorSpecs(QueryRunnerTestHelper.qualityUniques)
         .setPostAggregatorSpecs(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new HyperUniqueFinalizingPostAggregator(
                     QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric,
                     QueryRunnerTestHelper.uniqueMetric
@@ -3336,38 +3038,26 @@ public class GroupByQueryRunnerTest
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setGranularity(QueryRunnerTestHelper.allGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    QueryRunnerTestHelper.marketDimension,
-                    QueryRunnerTestHelper.marketDimension
-                )
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec(
+            QueryRunnerTestHelper.marketDimension,
+            QueryRunnerTestHelper.marketDimension
+        ))
         .setInterval(QueryRunnerTestHelper.fullOnInterval)
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
+                Collections.singletonList(
                     new OrderByColumnSpec(
                         QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric,
                         OrderByColumnSpec.Direction.DESCENDING
                     )
-                ), 3
-            )
-        )
-        .setHavingSpec(
-            new GreaterThanHavingSpec(
-                QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric,
-                8
-            )
-        )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.qualityUniques
+                ),
+                3
             )
         )
+        .setHavingSpec(new GreaterThanHavingSpec(QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric, 8))
+        .setAggregatorSpecs(QueryRunnerTestHelper.qualityUniques)
         .setPostAggregatorSpecs(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new HyperUniqueFinalizingPostAggregator(
                     QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric,
                     QueryRunnerTestHelper.uniqueMetric
@@ -3397,33 +3087,24 @@ public class GroupByQueryRunnerTest
   {
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setGranularity(QueryRunnerTestHelper.allGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    QueryRunnerTestHelper.marketDimension,
-                    QueryRunnerTestHelper.marketDimension
-                )
-            )
-        )
+        .setGranularity(QueryRunnerTestHelper.allGran).setDimensions(new DefaultDimensionSpec(
+            QueryRunnerTestHelper.marketDimension,
+            QueryRunnerTestHelper.marketDimension
+        ))
         .setInterval(QueryRunnerTestHelper.fullOnInterval)
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
+                Collections.singletonList(
                     new OrderByColumnSpec(
                         QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric,
                         OrderByColumnSpec.Direction.DESCENDING
                     )
-                ), 3
-            )
-        )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.qualityUniques
+                ),
+                3
             )
-        )
+        ).setAggregatorSpecs(QueryRunnerTestHelper.qualityUniques)
         .setPostAggregatorSpecs(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new HyperUniqueFinalizingPostAggregator(
                     QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric,
                     QueryRunnerTestHelper.uniqueMetric
@@ -3483,24 +3164,17 @@ public class GroupByQueryRunnerTest
     GroupByQuery query = GroupByQuery
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                new ExtractionDimensionSpec(
-                    "quality",
-                    "alias",
-                    new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, false, false)
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
+        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird).setDimensions(new ExtractionDimensionSpec(
+            "quality",
+            "alias",
+            new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, false, false)
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
+        .setLimitSpec(
+            new DefaultLimitSpec(
+                Collections.singletonList(new OrderByColumnSpec("alias", null, StringComparators.ALPHANUMERIC)),
+                null
             )
         )
-        .setLimitSpec(new DefaultLimitSpec(Lists.newArrayList(
-            new OrderByColumnSpec("alias", null, StringComparators.ALPHANUMERIC)), null))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -3546,24 +3220,17 @@ public class GroupByQueryRunnerTest
     GroupByQuery query = GroupByQuery
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                new ExtractionDimensionSpec(
-                    "quality",
-                    "alias",
-                    new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, false, false)
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
+        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird).setDimensions(new ExtractionDimensionSpec(
+            "quality",
+            "alias",
+            new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, false, false)
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
+        .setLimitSpec(
+            new DefaultLimitSpec(
+                Collections.singletonList(new OrderByColumnSpec("alias", null, StringComparators.ALPHANUMERIC)),
+                11
             )
         )
-        .setLimitSpec(new DefaultLimitSpec(Lists.newArrayList(
-            new OrderByColumnSpec("alias", null, StringComparators.ALPHANUMERIC)), 11))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setContext(ImmutableMap.of("sortByDimsFirst", true))
         .build();
@@ -3599,33 +3266,19 @@ public class GroupByQueryRunnerTest
   {
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setGranularity(QueryRunnerTestHelper.dayGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    QueryRunnerTestHelper.marketDimension,
-                    QueryRunnerTestHelper.marketDimension
-                )
-            )
-        )
+        .setGranularity(QueryRunnerTestHelper.dayGran).setDimensions(new DefaultDimensionSpec(
+            QueryRunnerTestHelper.marketDimension,
+            QueryRunnerTestHelper.marketDimension
+        ))
         .setInterval(QueryRunnerTestHelper.firstToThird)
         // Using a limitSpec here to achieve a per group limit is incorrect.
         // Limit is applied on the overall results.
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "rows",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ), 2
-            )
-        )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.rowsCount
+                Collections.singletonList(new OrderByColumnSpec("rows", OrderByColumnSpec.Direction.DESCENDING)),
+                2
             )
-        )
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .build();
 
     List<Row> expectedResults = Arrays.asList(
@@ -3678,20 +3331,13 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("index", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("index", "index"))
         .setPostAggregatorSpecs(ImmutableList.of(QueryRunnerTestHelper.addRowsIndexConstant))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setHavingSpec(
             new OrHavingSpec(
-                ImmutableList.of(
-                    new GreaterThanHavingSpec(QueryRunnerTestHelper.addRowsIndexConstantMetric, 1000L)
-                )
+                ImmutableList.of(new GreaterThanHavingSpec(QueryRunnerTestHelper.addRowsIndexConstantMetric, 1000L))
             )
         );
 
@@ -3707,10 +3353,10 @@ public class GroupByQueryRunnerTest
           {
             // simulate two daily segments
             final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-02/2011-04-03")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
             );
             final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-03/2011-04-04")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
             );
             return new MergeSequence(
                 queryPlus.getQuery().getResultOrdering(),
@@ -3733,23 +3379,13 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-01-25/2011-01-28")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new DoubleSumAggregatorFactory("index", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new DoubleSumAggregatorFactory("index", "index"))
         .setGranularity(Granularities.ALL)
         .setHavingSpec(new GreaterThanHavingSpec("index", 310L))
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "index",
-                        OrderByColumnSpec.Direction.ASCENDING
-                    )
-                ),
+                Collections.singletonList(new OrderByColumnSpec("index", OrderByColumnSpec.Direction.ASCENDING)),
                 5
             )
         );
@@ -3843,13 +3479,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("index", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("index", "index"))
         .setPostAggregatorSpecs(ImmutableList.of(QueryRunnerTestHelper.addRowsIndexConstant))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setHavingSpec(
@@ -3882,13 +3513,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setHavingSpec(
             new OrHavingSpec(
@@ -3935,13 +3561,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setHavingSpec(havingSpec);
 
@@ -3982,13 +3603,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setHavingSpec(havingSpec);
 
@@ -4013,13 +3629,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setHavingSpec(
             new OrHavingSpec(
@@ -4042,10 +3653,10 @@ public class GroupByQueryRunnerTest
           {
             // simulate two daily segments
             final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-02/2011-04-03")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
             );
             final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-03/2011-04-04")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
             );
             return new MergeSequence(
                 queryPlus.getQuery().getResultOrdering(),
@@ -4104,13 +3715,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setPostAggregatorSpecs(
             Collections.singletonList(
                 new ArithmeticPostAggregator(
@@ -4151,10 +3757,10 @@ public class GroupByQueryRunnerTest
           {
             // simulate two daily segments
             final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-02/2011-04-03")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
             );
             final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-03/2011-04-04")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
             );
             return new MergeSequence(
                 queryPlus.getQuery().getResultOrdering(),
@@ -4203,12 +3809,8 @@ public class GroupByQueryRunnerTest
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
         .setDimFilter(new RegexDimFilter("quality", "auto.*", null))
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "quality")))
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                QueryRunnerTestHelper.rowsCount
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "quality"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null));
 
     final GroupByQuery query = builder.build();
@@ -4230,12 +3832,7 @@ public class GroupByQueryRunnerTest
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
         .addDimension("billy")
-        .addDimension("quality")
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                QueryRunnerTestHelper.rowsCount
-            )
-        )
+        .addDimension("quality").setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null));
 
     final GroupByQuery query = builder.build();
@@ -4289,20 +3886,16 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setDimFilter(new JavaScriptDimFilter(
             "quality",
             "function(dim){ return true; }",
             null,
             JavaScriptConfig.getEnabledInstance()
         ))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index"),
-                new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new LongSumAggregatorFactory("idx", "index"),
+                            new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4310,13 +3903,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new LongSumAggregatorFactory("rows", "rows"),
-                new LongSumAggregatorFactory("idx", "idx")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("alias", "alias"))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"), new LongSumAggregatorFactory("idx", "idx"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4354,20 +3942,16 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setDimFilter(new JavaScriptDimFilter(
             "quality",
             "function(dim){ return true; }",
             null,
             JavaScriptConfig.getEnabledInstance()
         ))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index"),
-                new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new LongSumAggregatorFactory("idx", "index"),
+                            new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4382,13 +3966,8 @@ public class GroupByQueryRunnerTest
                 )
             )
         )
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new LongSumAggregatorFactory("rows", "rows"),
-                new LongSumAggregatorFactory("idx", "idx")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("alias", "alias"))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"), new LongSumAggregatorFactory("idx", "idx"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4426,20 +4005,16 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setDimFilter(new JavaScriptDimFilter(
             "quality",
             "function(dim){ return true; }",
             null,
             JavaScriptConfig.getEnabledInstance()
         ))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index"),
-                new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new LongSumAggregatorFactory("idx", "index"),
+                            new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setContext(ImmutableMap.of("chunkPeriod", "P1D"))
         .build();
@@ -4455,13 +4030,8 @@ public class GroupByQueryRunnerTest
                 )
             )
         )
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new LongSumAggregatorFactory("rows", "rows"),
-                new LongSumAggregatorFactory("idx", "idx")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("alias", "alias"))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"), new LongSumAggregatorFactory("idx", "idx"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4501,20 +4071,16 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setDimFilter(new JavaScriptDimFilter(
             "quality",
             "function(dim){ return true; }",
             null,
             JavaScriptConfig.getEnabledInstance()
         ))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index"),
-                new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new LongSumAggregatorFactory("idx", "index"),
+                            new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4528,20 +4094,8 @@ public class GroupByQueryRunnerTest
                 )
             )
         )
-        .setDimensions(Lists.newArrayList(
-            new ExtractionDimensionSpec(
-                "alias",
-                "alias",
-                new RegexDimExtractionFn("(a).*", true, "a")
-            )
-                       )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new LongSumAggregatorFactory("rows", "rows"),
-                new LongSumAggregatorFactory("idx", "idx")
-            )
-        )
+        .setDimensions(new ExtractionDimensionSpec("alias", "alias", new RegexDimExtractionFn("(a).*", true, "a")))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"), new LongSumAggregatorFactory("idx", "idx"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4562,14 +4116,10 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index"),
-                new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new LongSumAggregatorFactory("idx", "index"),
+                            new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4577,13 +4127,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new DoubleMaxAggregatorFactory("idx", "idx"),
-                new DoubleMaxAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new DoubleMaxAggregatorFactory("idx", "idx"),
+                            new DoubleMaxAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4602,13 +4148,10 @@ public class GroupByQueryRunnerTest
         .setVirtualColumns(
             new ExpressionVirtualColumn("expr", "-index + 100", ValueType.FLOAT, TestExprMacroTable.INSTANCE)
         )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "expr"),
-                new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")
-            )
-        ).build();
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new LongSumAggregatorFactory("idx", "expr"),
+                            new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen"))
+        .build();
     query = (GroupByQuery) query.withDataSource(new QueryDataSource(subquery));
 
     expectedResults = GroupByQueryRunnerTestHelper.createExpectedRows(
@@ -4630,15 +4173,10 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setPostAggregatorSpecs(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new ArithmeticPostAggregator(
                     "post_agg",
                     "+",
@@ -4656,14 +4194,10 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new DoubleMaxAggregatorFactory("idx1", "idx"),
-                new DoubleMaxAggregatorFactory("idx2", "idx"),
-                new DoubleMaxAggregatorFactory("idx3", "post_agg"),
-                new DoubleMaxAggregatorFactory("idx4", "post_agg")
-            )
-        )
+        .setAggregatorSpecs(new DoubleMaxAggregatorFactory("idx1", "idx"),
+                            new DoubleMaxAggregatorFactory("idx2", "idx"),
+                            new DoubleMaxAggregatorFactory("idx3", "post_agg"),
+                            new DoubleMaxAggregatorFactory("idx4", "post_agg"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4688,13 +4222,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "quality")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "quality"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4702,11 +4231,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                new DoubleMaxAggregatorFactory("idx", "idx")
-            )
-        )
+        .setAggregatorSpecs(new DoubleMaxAggregatorFactory("idx", "idx"))
         .setDimFilter(
             new OrDimFilter(
                 Lists.newArrayList(
@@ -4741,13 +4266,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4755,11 +4275,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.secondOnly)
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                new DoubleMaxAggregatorFactory("idx", "idx")
-            )
-        )
+        .setAggregatorSpecs(new DoubleMaxAggregatorFactory("idx", "idx"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4783,22 +4299,11 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("market", "market"),
-                new ExtractionDimensionSpec(
-                    Column.TIME_COLUMN_NAME,
-                    Column.TIME_COLUMN_NAME,
-                    new TimeFormatExtractionFn("EEEE", null, null, null, false)
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                QueryRunnerTestHelper.indexDoubleSum
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("market", "market"), new ExtractionDimensionSpec(
+            Column.TIME_COLUMN_NAME,
+            Column.TIME_COLUMN_NAME,
+            new TimeFormatExtractionFn("EEEE", null, null, null, false)
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, QueryRunnerTestHelper.indexDoubleSum)
         .setPostAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .setDimFilter(
@@ -4825,13 +4330,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "__time")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "__time"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setHavingSpec(
             new OrHavingSpec(
@@ -4857,13 +4357,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.emptyInterval)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4871,11 +4366,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                new DoubleMaxAggregatorFactory("idx", "idx")
-            )
-        )
+        .setAggregatorSpecs(new DoubleMaxAggregatorFactory("idx", "idx"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -4890,19 +4381,13 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setDimFilter(new JavaScriptDimFilter(
             "quality",
             "function(dim){ return true; }",
             null,
             JavaScriptConfig.getEnabledInstance()
-        ))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx_subagg", "index")
-            )
-        )
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx_subagg", "index"))
         .setPostAggregatorSpecs(
             Collections.singletonList(
                 new ArithmeticPostAggregator(
@@ -4921,13 +4406,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new LongSumAggregatorFactory("rows", "rows"),
-                new LongSumAggregatorFactory("idx", "idx_subpostagg")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("alias", "alias"))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"),
+                            new LongSumAggregatorFactory("idx", "idx_subpostagg"))
         .setPostAggregatorSpecs(
             Collections.singletonList(
                 new ArithmeticPostAggregator(
@@ -5156,19 +4637,13 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setDimFilter(new JavaScriptDimFilter(
             "quality",
             "function(dim){ return true; }",
             null,
             JavaScriptConfig.getEnabledInstance()
-        ))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx_subagg", "index")
-            )
-        )
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx_subagg", "index"))
         .setPostAggregatorSpecs(
             Collections.singletonList(
                 new ArithmeticPostAggregator(
@@ -5200,13 +4675,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new LongSumAggregatorFactory("rows", "rows"),
-                new LongSumAggregatorFactory("idx", "idx_subpostagg")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("alias", "alias"))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"),
+                            new LongSumAggregatorFactory("idx", "idx_subpostagg"))
         .setPostAggregatorSpecs(
             Collections.singletonList(
                 new ArithmeticPostAggregator(
@@ -5413,27 +4884,23 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setDimFilter(new JavaScriptDimFilter(
             "market",
             "function(dim){ return true; }",
             null,
             JavaScriptConfig.getEnabledInstance()
         ))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new DoubleSumAggregatorFactory("idx_subagg", "index"),
-                new JavaScriptAggregatorFactory(
-                    "js_agg",
-                    Arrays.asList("index", "market"),
-                    "function(current, index, dim){return current + index + dim.length;}",
-                    "function(){return 0;}",
-                    "function(a,b){return a + b;}",
-                    JavaScriptConfig.getEnabledInstance()
-                )
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new DoubleSumAggregatorFactory("idx_subagg", "index"),
+                            new JavaScriptAggregatorFactory(
+                                "js_agg",
+                                Arrays.asList("index", "market"),
+                                "function(current, index, dim){return current + index + dim.length;}",
+                                "function(){return 0;}",
+                                "function(a,b){return a + b;}",
+                                JavaScriptConfig.getEnabledInstance()
+                            ))
         .setPostAggregatorSpecs(
             Collections.singletonList(
                 new ArithmeticPostAggregator(
@@ -5465,14 +4932,10 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new LongSumAggregatorFactory("rows", "rows"),
-                new LongSumAggregatorFactory("idx", "idx_subpostagg"),
-                new DoubleSumAggregatorFactory("js_outer_agg", "js_agg")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("alias", "alias"))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"),
+                            new LongSumAggregatorFactory("idx", "idx_subpostagg"),
+                            new DoubleSumAggregatorFactory("js_outer_agg", "js_agg"))
         .setPostAggregatorSpecs(
             Collections.singletonList(
                 new ArithmeticPostAggregator(
@@ -5578,14 +5041,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("market", "market"),
-                                                         new DefaultDimensionSpec("quality", "quality")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("index", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("market", "market"), new DefaultDimensionSpec("quality", "quality"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("index", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -5595,11 +5052,7 @@ public class GroupByQueryRunnerTest
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
         .setDimensions(Lists.newArrayList())
-        .setAggregatorSpecs(
-            ImmutableList.of(
-                new FilteredAggregatorFactory(QueryRunnerTestHelper.rowsCount, filter)
-            )
-        )
+        .setAggregatorSpecs(new FilteredAggregatorFactory(QueryRunnerTestHelper.rowsCount, filter))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -5617,14 +5070,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("market", "market"),
-                                                         new DefaultDimensionSpec("quality", "quality")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("index", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("market", "market"), new DefaultDimensionSpec("quality", "quality"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("index", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -5636,11 +5083,7 @@ public class GroupByQueryRunnerTest
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
         .setDimensions(Lists.newArrayList())
         .setDimFilter(firstDaysFilter)
-        .setAggregatorSpecs(
-            ImmutableList.of(
-                new FilteredAggregatorFactory(QueryRunnerTestHelper.rowsCount, fridayFilter)
-            )
-        )
+        .setAggregatorSpecs(new FilteredAggregatorFactory(QueryRunnerTestHelper.rowsCount, fridayFilter))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -5666,7 +5109,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -5674,8 +5117,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList())
-        .setAggregatorSpecs(ImmutableList.of(new CountAggregatorFactory("count")))
+        .setDimensions(Lists.newArrayList()).setAggregatorSpecs(new CountAggregatorFactory("count"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 10000))
         .build();
@@ -5694,7 +5136,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -5703,8 +5145,7 @@ public class GroupByQueryRunnerTest
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setVirtualColumns(new ExpressionVirtualColumn("expr", "1", ValueType.FLOAT, TestExprMacroTable.INSTANCE))
-        .setDimensions(Lists.newArrayList())
-        .setAggregatorSpecs(ImmutableList.of(new LongSumAggregatorFactory("count", "expr")))
+        .setDimensions(Lists.newArrayList()).setAggregatorSpecs(new LongSumAggregatorFactory("count", "expr"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -5722,16 +5163,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(Lists.newArrayList(
-            new DefaultDimensionSpec("market", "market"),
-            new DefaultDimensionSpec("quality", "quality")
-        ))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("index", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("market", "market"), new DefaultDimensionSpec("quality", "quality"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("index", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -5739,18 +5172,14 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(Lists.newArrayList())
-        .setAggregatorSpecs(
-            ImmutableList.of(
-                new CardinalityAggregatorFactory("car",
-                                                 ImmutableList.of(new DefaultDimensionSpec(
-                                                     "quality",
-                                                     "quality"
-                                                 )),
-                                                 false
-                )
-            )
-        )
+        .setDimensions(Lists.newArrayList()).setAggregatorSpecs(new CardinalityAggregatorFactory(
+            "car",
+            ImmutableList.of(new DefaultDimensionSpec(
+                "quality",
+                "quality"
+            )),
+            false
+        ))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -5768,7 +5197,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setLimitSpec(
             new DefaultLimitSpec(
@@ -5782,8 +5211,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList())
-        .setAggregatorSpecs(ImmutableList.of(new CountAggregatorFactory("count")))
+        .setDimensions(Lists.newArrayList()).setAggregatorSpecs(new CountAggregatorFactory("count"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -5812,14 +5240,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("market", "market"),
-                                                         new DefaultDimensionSpec("quality", "quality")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("index", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("market", "market"), new DefaultDimensionSpec("quality", "quality"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("index", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -5827,19 +5249,15 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "quality")))
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                new JavaScriptAggregatorFactory(
-                    "js_agg",
-                    Arrays.asList("index", "market"),
-                    "function(current, index, dim){return current + index + dim.length;}",
-                    "function(){return 0;}",
-                    "function(a,b){return a + b;}",
-                    JavaScriptConfig.getEnabledInstance()
-                )
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "quality"))
+        .setAggregatorSpecs(new JavaScriptAggregatorFactory(
+            "js_agg",
+            Arrays.asList("index", "market"),
+            "function(current, index, dim){return current + index + dim.length;}",
+            "function(){return 0;}",
+            "function(a,b){return a + b;}",
+            JavaScriptConfig.getEnabledInstance()
+        ))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -5875,14 +5293,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("market", "market"),
-                                                         new DefaultDimensionSpec("quality", "quality")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("index", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("market", "market"), new DefaultDimensionSpec("quality", "quality"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("index", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -5890,19 +5302,15 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "quality")))
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                new JavaScriptAggregatorFactory(
-                    "js_agg",
-                    Arrays.asList("index", "rows"),
-                    "function(current, index, rows){return current + index + rows;}",
-                    "function(){return 0;}",
-                    "function(a,b){return a + b;}",
-                    JavaScriptConfig.getEnabledInstance()
-                )
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "quality"))
+        .setAggregatorSpecs(new JavaScriptAggregatorFactory(
+            "js_agg",
+            Arrays.asList("index", "rows"),
+            "function(current, index, rows){return current + index + rows;}",
+            "function(){return 0;}",
+            "function(a,b){return a + b;}",
+            JavaScriptConfig.getEnabledInstance()
+        ))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -5938,14 +5346,10 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index"),
-                new HyperUniquesAggregatorFactory("quality_uniques", "quality_uniques")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new LongSumAggregatorFactory("idx", "index"),
+                            new HyperUniquesAggregatorFactory("quality_uniques", "quality_uniques"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -5953,14 +5357,10 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new LongSumAggregatorFactory("rows", "rows"),
-                new LongSumAggregatorFactory("idx", "idx"),
-                new HyperUniquesAggregatorFactory("uniq", "quality_uniques")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("alias", "alias"))
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"),
+                            new LongSumAggregatorFactory("idx", "idx"),
+                            new HyperUniquesAggregatorFactory("uniq", "quality_uniques"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -6079,13 +5479,9 @@ public class GroupByQueryRunnerTest
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setDimensions(Lists.newArrayList())
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index"),
-                new HyperUniquesAggregatorFactory("quality_uniques_inner", "quality_uniques")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new LongSumAggregatorFactory("idx", "index"),
+                            new HyperUniquesAggregatorFactory("quality_uniques_inner", "quality_uniques"))
         .setPostAggregatorSpecs(
             Collections.singletonList(
                 new FieldAccessPostAggregator("quality_uniques_inner_post", "quality_uniques_inner")
@@ -6099,13 +5495,9 @@ public class GroupByQueryRunnerTest
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setDimensions(Lists.newArrayList())
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new LongSumAggregatorFactory("rows", "rows"),
-                new LongSumAggregatorFactory("idx", "idx"),
-                new HyperUniquesAggregatorFactory("quality_uniques_outer", "quality_uniques_inner_post")
-            )
-        )
+        .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"),
+                            new LongSumAggregatorFactory("idx", "idx"),
+                            new HyperUniquesAggregatorFactory("quality_uniques_outer", "quality_uniques_inner_post"))
         .setPostAggregatorSpecs(
             Collections.singletonList(
                 new HyperUniqueFinalizingPostAggregator("quality_uniques_outer_post", "quality_uniques_outer")
@@ -6140,14 +5532,10 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(ImmutableList.of(new DefaultDimensionSpec("market", "market")))
-        .setAggregatorSpecs(
-            ImmutableList.of(
-                QueryRunnerTestHelper.rowsCount,
-                new LongFirstAggregatorFactory("innerfirst", "index"),
-                new LongLastAggregatorFactory("innerlast", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("market", "market"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new LongFirstAggregatorFactory("innerfirst", "index"),
+                            new LongLastAggregatorFactory("innerlast", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setContext(ImmutableMap.of("finalize", true))
         .build();
@@ -6157,12 +5545,8 @@ public class GroupByQueryRunnerTest
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
         .setDimensions(Lists.newArrayList())
-        .setAggregatorSpecs(
-            ImmutableList.of(
-                new LongFirstAggregatorFactory("first", "innerfirst"),
-                new LongLastAggregatorFactory("last", "innerlast")
-            )
-        )
+        .setAggregatorSpecs(new LongFirstAggregatorFactory("first", "innerfirst"),
+                            new LongLastAggregatorFactory("last", "innerlast"))
         .setGranularity(QueryRunnerTestHelper.monthGran)
         .build();
 
@@ -6184,13 +5568,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                QueryRunnerTestHelper.jsCountIfTimeGreaterThan,
-                QueryRunnerTestHelper.__timeLongSum
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            QueryRunnerTestHelper.jsCountIfTimeGreaterThan,
+                            QueryRunnerTestHelper.__timeLongSum)
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -6217,22 +5597,11 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("market", "market"),
-                new ExtractionDimensionSpec(
-                    Column.TIME_COLUMN_NAME,
-                    "dayOfWeek",
-                    new TimeFormatExtractionFn("EEEE", null, null, null, false)
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                QueryRunnerTestHelper.indexDoubleSum
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("market", "market"), new ExtractionDimensionSpec(
+            Column.TIME_COLUMN_NAME,
+            "dayOfWeek",
+            new TimeFormatExtractionFn("EEEE", null, null, null, false)
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, QueryRunnerTestHelper.indexDoubleSum)
         .setPostAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .setDimFilter(
@@ -6473,27 +5842,16 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
-        .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("market", "market"),
-                new ExtractionDimensionSpec(
-                    Column.TIME_COLUMN_NAME,
-                    "dayOfWeek",
-                    new CascadeExtractionFn(
-                        new ExtractionFn[]{
-                            new TimeFormatExtractionFn("EEEE", null, null, null, false),
-                            nullWednesdays,
-                        }
-                    )
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                QueryRunnerTestHelper.indexDoubleSum
+        .setDimensions(new DefaultDimensionSpec("market", "market"), new ExtractionDimensionSpec(
+            Column.TIME_COLUMN_NAME,
+            "dayOfWeek",
+            new CascadeExtractionFn(
+                new ExtractionFn[]{
+                    new TimeFormatExtractionFn("EEEE", null, null, null, false),
+                    nullWednesdays,
+                    }
             )
-        )
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, QueryRunnerTestHelper.indexDoubleSum)
         .setPostAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .setDimFilter(
@@ -6723,13 +6081,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setDimFilter(new SelectorDimFilter("quality", "mezzanine", null))
         .setContext(ImmutableMap.of("bySegment", true));
@@ -6784,31 +6137,17 @@ public class GroupByQueryRunnerTest
     GroupByQuery.Builder builder = GroupByQuery
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(
-            Lists.newArrayList(
-                new ExtractionDimensionSpec(
-                    "quality",
-                    "alias",
-                    new LookupExtractionFn(
-                        new MapLookupExtractor(
-                            ImmutableMap.of(
-                                "mezzanine",
-                                "mezzanine0"
-                            ),
-                            false
-                        ), false, null, false,
-                        false
-                    )
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
+        .setInterval("2011-04-02/2011-04-04").setDimensions(new ExtractionDimensionSpec(
+            "quality",
+            "alias",
+            new LookupExtractionFn(
+                new MapLookupExtractor(ImmutableMap.of("mezzanine", "mezzanine0"), false),
+                false,
+                null,
+                false,
+                false
             )
-        )
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setDimFilter(new SelectorDimFilter("quality", "mezzanine", null))
         .setContext(ImmutableMap.of("bySegment", true));
@@ -6858,31 +6197,17 @@ public class GroupByQueryRunnerTest
     GroupByQuery.Builder builder = GroupByQuery
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(
-            Lists.newArrayList(
-                new ExtractionDimensionSpec(
-                    "quality",
-                    "alias",
-                    new LookupExtractionFn(
-                        new MapLookupExtractor(
-                            ImmutableMap.of(
-                                "mezzanine",
-                                "mezzanine0"
-                            ),
-                            false
-                        ), false, null, true,
-                        false
-                    )
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
+        .setInterval("2011-04-02/2011-04-04").setDimensions(new ExtractionDimensionSpec(
+            "quality",
+            "alias",
+            new LookupExtractionFn(
+                new MapLookupExtractor(ImmutableMap.of("mezzanine", "mezzanine0"), false),
+                false,
+                null,
+                true,
+                false
             )
-        )
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setDimFilter(new SelectorDimFilter("quality", "mezzanine", null))
         .setContext(ImmutableMap.of("bySegment", true));
@@ -6928,22 +6253,12 @@ public class GroupByQueryRunnerTest
         new SelectorDimFilter("quality", "travel", null)
     );
 
-    GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource)
+    GroupByQuery query = GroupByQuery.builder()
+                                     .setDataSource(QueryRunnerTestHelper.dataSource)
                                      .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-                                     .setDimensions(
-                                         Lists.newArrayList(
-                                             new DefaultDimensionSpec(
-                                                 "quality",
-                                                 "alias"
-                                             )
-                                         )
-                                     )
-                                     .setAggregatorSpecs(
-                                         Arrays.asList(
-                                             QueryRunnerTestHelper.rowsCount,
-                                             new LongSumAggregatorFactory("idx", "index")
-                                         )
-                                     )
+                                     .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+                                     .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                                                         new LongSumAggregatorFactory("idx", "index"))
                                      .setGranularity(QueryRunnerTestHelper.dayGran)
                                      .setDimFilter(new OrDimFilter(dimFilters))
                                      .build();
@@ -6990,22 +6305,12 @@ public class GroupByQueryRunnerTest
 
     MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false);
     LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, false);
-    GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource)
+    GroupByQuery query = GroupByQuery.builder()
+                                     .setDataSource(QueryRunnerTestHelper.dataSource)
                                      .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-                                     .setDimensions(
-                                         Lists.newArrayList(
-                                             new DefaultDimensionSpec(
-                                                 "quality",
-                                                 "alias"
-                                             )
-                                         )
-                                     )
-                                     .setAggregatorSpecs(
-                                         Arrays.asList(
-                                             QueryRunnerTestHelper.rowsCount,
-                                             new LongSumAggregatorFactory("idx", "index")
-                                         )
-                                     )
+                                     .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+                                     .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                                                         new LongSumAggregatorFactory("idx", "index"))
                                      .setGranularity(QueryRunnerTestHelper.dayGran)
                                      .setDimFilter(new ExtractionDimFilter("quality", "", lookupExtractionFn, null))
                                      .build();
@@ -7027,31 +6332,17 @@ public class GroupByQueryRunnerTest
     MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false);
     LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, false);
 
-    GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource)
+    GroupByQuery query = GroupByQuery.builder()
+                                     .setDataSource(QueryRunnerTestHelper.dataSource)
                                      .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-                                     .setDimensions(
-                                         Lists.newArrayList(
-                                             new DefaultDimensionSpec(
-                                                 "quality",
-                                                 "alias"
-                                             )
-                                         )
-                                     )
-                                     .setAggregatorSpecs(
-                                         Arrays.asList(
-                                             QueryRunnerTestHelper.rowsCount,
-                                             new LongSumAggregatorFactory("idx", "index")
-                                         )
-                                     )
+                                     .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+                                     .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                                                         new LongSumAggregatorFactory("idx", "index"))
                                      .setGranularity(QueryRunnerTestHelper.dayGran)
                                      .setDimFilter(
-                                         new ExtractionDimFilter(
-                                             "quality",
-                                             "NOT_THERE",
-                                             lookupExtractionFn,
-                                             null
-                                         )
-                                     ).build();
+                                         new ExtractionDimFilter("quality", "NOT_THERE", lookupExtractionFn, null)
+                                     )
+                                     .build();
     List<Row> expectedResults = Collections.emptyList();
 
     Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
@@ -7068,31 +6359,17 @@ public class GroupByQueryRunnerTest
     MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false);
     LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, false);
 
-    GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource)
+    GroupByQuery query = GroupByQuery.builder()
+                                     .setDataSource(QueryRunnerTestHelper.dataSource)
                                      .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-                                     .setDimensions(
-                                         Lists.newArrayList(
-                                             new DefaultDimensionSpec(
-                                                 "null_column",
-                                                 "alias"
-                                             )
-                                         )
-                                     )
-                                     .setAggregatorSpecs(
-                                         Arrays.asList(
-                                             QueryRunnerTestHelper.rowsCount,
-                                             new LongSumAggregatorFactory("idx", "index")
-                                         )
-                                     )
+                                     .setDimensions(new DefaultDimensionSpec("null_column", "alias"))
+                                     .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                                                         new LongSumAggregatorFactory("idx", "index"))
                                      .setGranularity(QueryRunnerTestHelper.dayGran)
                                      .setDimFilter(
-                                         new ExtractionDimFilter(
-                                             "null_column",
-                                             "NULLorEMPTY",
-                                             lookupExtractionFn,
-                                             null
-                                         )
-                                     ).build();
+                                         new ExtractionDimFilter("null_column", "NULLorEMPTY", lookupExtractionFn, null)
+                                     )
+                                     .build();
     List<Row> expectedResults = Arrays
         .asList(
             GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", null, "rows", 13L, "idx", 6619L),
@@ -7120,27 +6397,17 @@ public class GroupByQueryRunnerTest
     MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false);
     LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, "missing", true, false);
     DimFilter filter = new ExtractionDimFilter("quality", "mezzanineANDnews", lookupExtractionFn, null);
-    GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource)
+    GroupByQuery query = GroupByQuery.builder()
+                                     .setDataSource(QueryRunnerTestHelper.dataSource)
                                      .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-                                     .setDimensions(
-                                         Lists.newArrayList(
-                                             new DefaultDimensionSpec(
-                                                 "quality",
-                                                 "alias"
-                                             )
-                                         )
-                                     )
-                                     .setAggregatorSpecs(
-                                         Arrays.asList(
-                                             new FilteredAggregatorFactory(QueryRunnerTestHelper.rowsCount, filter),
-                                             (AggregatorFactory) new FilteredAggregatorFactory(
-                                                 new LongSumAggregatorFactory(
-                                                     "idx",
-                                                     "index"
-                                                 ), filter
-                                             )
-                                         )
-                                     )
+                                     .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+                                     .setAggregatorSpecs(new FilteredAggregatorFactory(
+                                         QueryRunnerTestHelper.rowsCount,
+                                         filter
+                                     ), new FilteredAggregatorFactory(
+                                         new LongSumAggregatorFactory("idx", "index"),
+                                         filter
+                                     ))
                                      .setGranularity(QueryRunnerTestHelper.dayGran)
                                      .build();
     List<Row> expectedResults = Arrays.asList(
@@ -7179,24 +6446,21 @@ public class GroupByQueryRunnerTest
 
     MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false);
     LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, true);
-    GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource)
+    GroupByQuery query = GroupByQuery.builder()
+                                     .setDataSource(QueryRunnerTestHelper.dataSource)
                                      .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-                                     .setDimensions(Lists.newArrayList(new DefaultDimensionSpec(
-                                         "quality",
-                                         "alias"
-                                     )))
-                                     .setAggregatorSpecs(
-                                         Arrays.asList(
-                                             QueryRunnerTestHelper.rowsCount,
-                                             new LongSumAggregatorFactory("idx", "index")
-                                         ))
+                                     .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+                                     .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                                                         new LongSumAggregatorFactory("idx", "index"))
                                      .setGranularity(QueryRunnerTestHelper.dayGran)
-                                     .setDimFilter(new ExtractionDimFilter(
-                                         "quality",
-                                         "newsANDmezzanine",
-                                         lookupExtractionFn,
-                                         null
-                                     ))
+                                     .setDimFilter(
+                                         new ExtractionDimFilter(
+                                             "quality",
+                                             "newsANDmezzanine",
+                                             lookupExtractionFn,
+                                             null
+                                         )
+                                     )
                                      .build();
     List<Row> expectedResults = Arrays.asList(
         GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L),
@@ -7219,24 +6483,22 @@ public class GroupByQueryRunnerTest
     MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false);
     LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, true);
 
-    GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource)
+    GroupByQuery query = GroupByQuery.builder()
+                                     .setDataSource(QueryRunnerTestHelper.dataSource)
                                      .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-                                     .setDimensions(Lists.newArrayList(new DefaultDimensionSpec(
-                                         "null_column",
-                                         "alias"
-                                     )))
-                                     .setAggregatorSpecs(
-                                         Arrays.asList(
-                                             QueryRunnerTestHelper.rowsCount,
-                                             new LongSumAggregatorFactory("idx", "index")
-                                         ))
+                                     .setDimensions(new DefaultDimensionSpec("null_column", "alias"))
+                                     .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                                                         new LongSumAggregatorFactory("idx", "index"))
                                      .setGranularity(QueryRunnerTestHelper.dayGran)
-                                     .setDimFilter(new ExtractionDimFilter(
-                                         "null_column",
-                                         "EMPTY",
-                                         lookupExtractionFn,
-                                         null
-                                     )).build();
+                                     .setDimFilter(
+                                         new ExtractionDimFilter(
+                                             "null_column",
+                                             "EMPTY",
+                                             lookupExtractionFn,
+                                             null
+                                         )
+                                     )
+                                     .build();
     List<Row> expectedResults = Arrays
         .asList(
             GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", null, "rows", 13L, "idx", 6619L),
@@ -7306,13 +6568,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
         .setDimFilter(superFilter)
         .setContext(ImmutableMap.of("bySegment", true));
@@ -7353,33 +6610,27 @@ public class GroupByQueryRunnerTest
                                            StringComparators.ALPHANUMERIC
     ));
     superFilterList.add(new RegexDimFilter("null_column", "EMPTY", extractionFn));
-    superFilterList.add(new SearchQueryDimFilter(
-        "null_column",
-        new ContainsSearchQuerySpec("EMPTY", true),
-        extractionFn
-    ));
+    superFilterList.add(
+        new SearchQueryDimFilter("null_column", new ContainsSearchQuerySpec("EMPTY", true), extractionFn)
+    );
     superFilterList.add(new JavaScriptDimFilter("null_column", jsFn, extractionFn, JavaScriptConfig.getEnabledInstance()));
     DimFilter superFilter = new AndDimFilter(superFilterList);
 
-    GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource)
+    GroupByQuery query = GroupByQuery.builder()
+                                     .setDataSource(QueryRunnerTestHelper.dataSource)
                                      .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-                                     .setDimensions(Lists.newArrayList(new DefaultDimensionSpec(
-                                         "null_column",
-                                         "alias"
-                                     )))
+                                     .setDimensions(new DefaultDimensionSpec("null_column", "alias"))
                                      .setAggregatorSpecs(
-                                         Arrays.asList(
-                                             QueryRunnerTestHelper.rowsCount,
-                                             new LongSumAggregatorFactory("idx", "index")
-                                         ))
+                                         QueryRunnerTestHelper.rowsCount,
+                                         new LongSumAggregatorFactory("idx", "index")
+                                     )
                                      .setGranularity(QueryRunnerTestHelper.dayGran)
                                      .setDimFilter(superFilter).build();
 
-    List<Row> expectedResults = Arrays
-        .asList(
-            GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", null, "rows", 13L, "idx", 6619L),
-            GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", null, "rows", 13L, "idx", 5827L)
-        );
+    List<Row> expectedResults = Arrays.asList(
+        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", null, "rows", 13L, "idx", 6619L),
+        GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", null, "rows", 13L, "idx", 5827L)
+    );
 
     Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
     TestHelper.assertExpectedObjects(expectedResults, results, "");
@@ -7395,21 +6646,16 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("market", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new CardinalityAggregatorFactory(
-                    "numVals",
-                    ImmutableList.of(new ExtractionDimensionSpec(
-                        QueryRunnerTestHelper.qualityDimension,
-                        QueryRunnerTestHelper.qualityDimension,
-                        helloFn
-                    )),
-                    false
-                )
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("market", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new CardinalityAggregatorFactory(
+            "numVals",
+            ImmutableList.of(new ExtractionDimensionSpec(
+                QueryRunnerTestHelper.qualityDimension,
+                QueryRunnerTestHelper.qualityDimension,
+                helloFn
+            )),
+            false
+        ))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -7433,20 +6679,15 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("market", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new CardinalityAggregatorFactory(
-                    "numVals",
-                    ImmutableList.of(new DefaultDimensionSpec(
-                        QueryRunnerTestHelper.indexMetric,
-                        QueryRunnerTestHelper.indexMetric
-                    )),
-                    false
-                )
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("market", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new CardinalityAggregatorFactory(
+            "numVals",
+            ImmutableList.of(new DefaultDimensionSpec(
+                QueryRunnerTestHelper.indexMetric,
+                QueryRunnerTestHelper.indexMetric
+            )),
+            false
+        ))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -7475,14 +6716,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG)))
+        .setDimensions(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG))
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .addOrderByColumn(new OrderByColumnSpec(
             "ql_alias",
             OrderByColumnSpec.Direction.ASCENDING,
@@ -7538,14 +6774,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG)))
+        .setDimensions(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG))
         .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .addOrderByColumn(new OrderByColumnSpec(
             "ql_alias",
             OrderByColumnSpec.Direction.DESCENDING,
@@ -7604,14 +6835,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec("qualityLong", "ql_alias", jsExtractionFn)))
+        .setDimensions(new ExtractionDimensionSpec("qualityLong", "ql_alias", jsExtractionFn))
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -7651,14 +6877,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG)))
+        .setDimensions(new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG))
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -7696,14 +6917,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec("__time", "time_alias", jsExtractionFn)))
+        .setDimensions(new ExtractionDimensionSpec("__time", "time_alias", jsExtractionFn))
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -7743,14 +6959,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT)))
+        .setDimensions(new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT))
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .addOrderByColumn(new OrderByColumnSpec(
             "index_alias",
             OrderByColumnSpec.Direction.ASCENDING,
@@ -7807,14 +7018,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("qualityFloat", "qf_alias", ValueType.FLOAT)))
+        .setDimensions(new DefaultDimensionSpec("qualityFloat", "qf_alias", ValueType.FLOAT))
         .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .addOrderByColumn(new OrderByColumnSpec(
             "qf_alias",
             OrderByColumnSpec.Direction.DESCENDING,
@@ -7870,14 +7076,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("qualityDouble", "alias", ValueType.DOUBLE)))
+        .setDimensions(new DefaultDimensionSpec("qualityDouble", "alias", ValueType.DOUBLE))
         .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .addOrderByColumn(new OrderByColumnSpec(
             "alias",
             OrderByColumnSpec.Direction.DESCENDING,
@@ -7936,14 +7137,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec("index", "index_alias", jsExtractionFn)))
+        .setDimensions(new ExtractionDimensionSpec("index", "index_alias", jsExtractionFn))
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -7987,18 +7183,11 @@ public class GroupByQueryRunnerTest
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("market", "alias"),
-                new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG),
-                new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG),
-                new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT)
-            )
-        )
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                QueryRunnerTestHelper.rowsCount
-            )
-        )
+            new DefaultDimensionSpec("market", "alias"),
+            new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG),
+            new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG),
+            new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT)
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .setHavingSpec(
             new DimFilterHavingSpec(
                 new AndDimFilter(
@@ -8051,18 +7240,11 @@ public class GroupByQueryRunnerTest
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("qualityLong", "ql_alias"),
-                new DefaultDimensionSpec("qualityFloat", "qf_alias")
-            )
+            new DefaultDimensionSpec("qualityLong", "ql_alias"),
+            new DefaultDimensionSpec("qualityFloat", "qf_alias")
         )
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -8107,19 +7289,12 @@ public class GroupByQueryRunnerTest
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("qualityLong", "ql_alias"),
-                new DefaultDimensionSpec("qualityFloat", "qf_alias"),
-                new DefaultDimensionSpec(Column.TIME_COLUMN_NAME, "time_alias")
-            )
+            new DefaultDimensionSpec("qualityLong", "ql_alias"),
+            new DefaultDimensionSpec("qualityFloat", "qf_alias"),
+            new DefaultDimensionSpec(Column.TIME_COLUMN_NAME, "time_alias")
         )
         .setDimFilter(new SelectorDimFilter("quality", "entertainment", null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -8128,18 +7303,11 @@ public class GroupByQueryRunnerTest
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("time_alias", "time_alias2", ValueType.LONG),
-                new DefaultDimensionSpec("ql_alias", "ql_alias_long", ValueType.LONG),
-                new DefaultDimensionSpec("qf_alias", "qf_alias_float", ValueType.FLOAT),
-                new DefaultDimensionSpec("ql_alias", "ql_alias_float", ValueType.FLOAT)
-            )
-        )
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                new CountAggregatorFactory("count")
-            )
-        )
+            new DefaultDimensionSpec("time_alias", "time_alias2", ValueType.LONG),
+            new DefaultDimensionSpec("ql_alias", "ql_alias_long", ValueType.LONG),
+            new DefaultDimensionSpec("qf_alias", "qf_alias_float", ValueType.FLOAT),
+            new DefaultDimensionSpec("ql_alias", "ql_alias_float", ValueType.FLOAT)
+        ).setAggregatorSpecs(new CountAggregatorFactory("count"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -8190,18 +7358,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                regexSpec,
-                listFilteredSpec
-            )
-        )
+        .setDimensions(regexSpec, listFilteredSpec)
         .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null))
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                new CountAggregatorFactory("count")
-            )
-        )
+        .setAggregatorSpecs(new CountAggregatorFactory("count"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -8248,18 +7407,9 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                regexSpec,
-                listFilteredSpec
-            )
-        )
+        .setDimensions(regexSpec, listFilteredSpec)
         .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null))
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                new CountAggregatorFactory("count")
-            )
-        )
+        .setAggregatorSpecs(new CountAggregatorFactory("count"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -8295,25 +7445,17 @@ public class GroupByQueryRunnerTest
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("quality", "alias"),
-                new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG),
-                new DefaultDimensionSpec("qualityFloat", "qf_alias", ValueType.FLOAT)
-            )
+            new DefaultDimensionSpec("quality", "alias"),
+            new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG),
+            new DefaultDimensionSpec("qualityFloat", "qf_alias", ValueType.FLOAT)
         )
         .setDimFilter(
             new InDimFilter(
                 "quality",
-                Lists.newArrayList("entertainment"),
+                Collections.singletonList("entertainment"),
                 null
             )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -8322,10 +7464,8 @@ public class GroupByQueryRunnerTest
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("ql_alias", "quallong", ValueType.LONG),
-                new DefaultDimensionSpec("qf_alias", "qualfloat", ValueType.FLOAT)
-            )
+            new DefaultDimensionSpec("ql_alias", "quallong", ValueType.LONG),
+            new DefaultDimensionSpec("qf_alias", "qualfloat", ValueType.FLOAT)
         )
         .setDimFilter(
             new AndDimFilter(
@@ -8344,12 +7484,8 @@ public class GroupByQueryRunnerTest
                 )
             )
         )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new LongSumAggregatorFactory("ql_alias_sum", "ql_alias"),
-                new DoubleSumAggregatorFactory("qf_alias_sum", "qf_alias")
-            )
-        )
+        .setAggregatorSpecs(new LongSumAggregatorFactory("ql_alias_sum", "ql_alias"),
+                            new DoubleSumAggregatorFactory("qf_alias_sum", "qf_alias"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -8380,17 +7516,10 @@ public class GroupByQueryRunnerTest
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("market", "alias"),
-                new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG),
-                new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT)
-            )
-        )
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                QueryRunnerTestHelper.rowsCount
-            )
-        )
+            new DefaultDimensionSpec("market", "alias"),
+            new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG),
+            new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT)
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -8399,17 +7528,11 @@ public class GroupByQueryRunnerTest
         .setDataSource(subQuery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
         .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("alias", "market"),
-                new DefaultDimensionSpec("time_alias", "time_alias2", ValueType.LONG)
-            )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                new LongMaxAggregatorFactory("time_alias_max", "time_alias"),
-                new DoubleMaxAggregatorFactory("index_alias_max", "index_alias")
-            )
+            new DefaultDimensionSpec("alias", "market"),
+            new DefaultDimensionSpec("time_alias", "time_alias2", ValueType.LONG)
         )
+        .setAggregatorSpecs(new LongMaxAggregatorFactory("time_alias_max", "time_alias"),
+                            new DoubleMaxAggregatorFactory("index_alias_max", "index_alias"))
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -8476,19 +7599,14 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec(
+        .setDimensions(new ExtractionDimensionSpec(
             QueryRunnerTestHelper.qualityDimension,
             "alias",
             ValueType.LONG,
             strlenFn
-        )))
+        ))
         .setDimFilter(new SelectorDimFilter(QueryRunnerTestHelper.qualityDimension, "entertainment", null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -8524,33 +7642,29 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setDimFilter(new SelectorDimFilter("quality", "technology", null))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("qlLong", "qualityLong"),
-                new DoubleSumAggregatorFactory("qlFloat", "qualityLong"),
-                new JavaScriptAggregatorFactory(
-                    "qlJs",
-                    ImmutableList.of("qualityLong"),
-                    "function(a,b) { return a + b; }",
-                    "function() { return 0; }",
-                    "function(a,b) { return a + b }",
-                    JavaScriptConfig.getEnabledInstance()
-                ),
-                new DoubleSumAggregatorFactory("qfFloat", "qualityFloat"),
-                new LongSumAggregatorFactory("qfLong", "qualityFloat"),
-                new JavaScriptAggregatorFactory(
-                    "qfJs",
-                    ImmutableList.of("qualityFloat"),
-                    "function(a,b) { return a + b; }",
-                    "function() { return 0; }",
-                    "function(a,b) { return a + b }",
-                    JavaScriptConfig.getEnabledInstance()
-                )
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount,
+                            new LongSumAggregatorFactory("qlLong", "qualityLong"),
+                            new DoubleSumAggregatorFactory("qlFloat", "qualityLong"),
+                            new JavaScriptAggregatorFactory(
+                                "qlJs",
+                                ImmutableList.of("qualityLong"),
+                                "function(a,b) { return a + b; }",
+                                "function() { return 0; }",
+                                "function(a,b) { return a + b }",
+                                JavaScriptConfig.getEnabledInstance()
+                            ),
+                            new DoubleSumAggregatorFactory("qfFloat", "qualityFloat"),
+                            new LongSumAggregatorFactory("qfLong", "qualityFloat"),
+                            new JavaScriptAggregatorFactory(
+                                "qfJs",
+                                ImmutableList.of("qualityFloat"),
+                                "function(a,b) { return a + b; }",
+                                "function() { return 0; }",
+                                "function(a,b) { return a + b }",
+                                JavaScriptConfig.getEnabledInstance()
+                            ))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -8598,23 +7712,14 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("quality", "alias"),
-                new ExtractionDimensionSpec(
-                    "qualityFloat",
-                    "qf_inner",
-                    ValueType.FLOAT,
-                    jsExtractionFn
-                )
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"), new ExtractionDimensionSpec(
+            "qualityFloat",
+            "qf_inner",
+            ValueType.FLOAT,
+            jsExtractionFn
+        ))
         .setDimFilter(new SelectorDimFilter("quality", "technology", null))
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                QueryRunnerTestHelper.rowsCount
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -8622,22 +7727,12 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("alias", "alias"),
-                new ExtractionDimensionSpec(
-                    "qf_inner",
-                    "qf_outer",
-                    ValueType.FLOAT,
-                    jsExtractionFn
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                QueryRunnerTestHelper.rowsCount
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("alias", "alias"), new ExtractionDimensionSpec(
+            "qf_inner",
+            "qf_outer",
+            ValueType.FLOAT,
+            jsExtractionFn
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -8666,23 +7761,14 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("quality", "alias"),
-                new ExtractionDimensionSpec(
-                    Column.TIME_COLUMN_NAME,
-                    "time_day",
-                    ValueType.LONG,
-                    new TimeFormatExtractionFn(null, null, null, Granularities.DAY, true)
-                )
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"), new ExtractionDimensionSpec(
+            Column.TIME_COLUMN_NAME,
+            "time_day",
+            ValueType.LONG,
+            new TimeFormatExtractionFn(null, null, null, Granularities.DAY, true)
+        ))
         .setDimFilter(new SelectorDimFilter("quality", "technology", null))
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                QueryRunnerTestHelper.rowsCount
-            )
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -8690,22 +7776,12 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(
-            Lists.newArrayList(
-                new DefaultDimensionSpec("alias", "alias"),
-                new ExtractionDimensionSpec(
-                    "time_day",
-                    "time_week",
-                    ValueType.LONG,
-                    new TimeFormatExtractionFn(null, null, null, Granularities.WEEK, true)
-                )
-            )
-        )
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                QueryRunnerTestHelper.rowsCount
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("alias", "alias"), new ExtractionDimensionSpec(
+            "time_day",
+            "time_week",
+            ValueType.LONG,
+            new TimeFormatExtractionFn(null, null, null, Granularities.WEEK, true)
+        )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .setGranularity(QueryRunnerTestHelper.allGran)
         .build();
 
@@ -8730,38 +7806,21 @@ public class GroupByQueryRunnerTest
     }
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setGranularity(QueryRunnerTestHelper.allGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    QueryRunnerTestHelper.marketDimension,
-                    "marketalias"
-                )
-            )
-        )
+        .setGranularity(QueryRunnerTestHelper.allGran).setDimensions(new DefaultDimensionSpec(
+            QueryRunnerTestHelper.marketDimension,
+            "marketalias"
+        ))
         .setInterval(QueryRunnerTestHelper.fullOnInterval)
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "marketalias",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ),
+                Collections.singletonList(new OrderByColumnSpec(
+                    "marketalias",
+                    OrderByColumnSpec.Direction.DESCENDING
+                )),
                 2
             )
-        )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.rowsCount
-            )
-        )
-        .setContext(
-            ImmutableMap.of(
-                GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN,
-                true
-            )
-        )
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
+        .setContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, true))
         .build();
 
     List<Row> expectedResults = Arrays.asList(
@@ -8795,30 +7854,15 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "alias",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ),
+                Collections.singletonList(new OrderByColumnSpec("alias", OrderByColumnSpec.Direction.DESCENDING)),
                 5
             )
         )
-        .setContext(
-            ImmutableMap.of(
-                GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN,
-                true
-            )
-        )
+        .setContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, true))
         .setGranularity(Granularities.ALL);
 
     final GroupByQuery allGranQuery = builder.build();
@@ -8831,10 +7875,10 @@ public class GroupByQueryRunnerTest
           {
             // simulate two daily segments
             final QueryPlus<Row> queryPlus1 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-02/2011-04-03")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
             );
             final QueryPlus<Row> queryPlus2 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-03/2011-04-04")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
             );
 
             return factory.getToolchest().mergeResults(
@@ -8877,30 +7921,15 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "idx",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ),
+                Collections.singletonList(new OrderByColumnSpec("idx", OrderByColumnSpec.Direction.DESCENDING)),
                 5
             )
         )
-        .setContext(
-            ImmutableMap.of(
-                GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN,
-                true
-            )
-        )
+        .setContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, true))
         .setGranularity(Granularities.ALL);
 
     final GroupByQuery allGranQuery = builder.build();
@@ -8915,10 +7944,10 @@ public class GroupByQueryRunnerTest
           {
             // simulate two daily segments
             final QueryPlus<Row> queryPlus1 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-02/2011-04-03")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
             );
             final QueryPlus<Row> queryPlus2 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-03/2011-04-04")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
             );
 
             return factory.getToolchest().mergeResults(
@@ -8959,37 +7988,19 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(
-            new DefaultDimensionSpec("quality", "alias"),
-            new DefaultDimensionSpec("market", "market")
-                       )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"), new DefaultDimensionSpec("market", "market"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setLimitSpec(
             new DefaultLimitSpec(
                 Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "alias",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    ),
-                    new OrderByColumnSpec(
-                        "market",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
+                    new OrderByColumnSpec("alias", OrderByColumnSpec.Direction.DESCENDING),
+                    new OrderByColumnSpec("market", OrderByColumnSpec.Direction.DESCENDING)
                 ),
                 5
             )
         )
         .setContext(
-            ImmutableMap.of(
-                GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN,
-                true
-            )
+            ImmutableMap.of(GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, true)
         )
         .setGranularity(Granularities.ALL);
 
@@ -9005,10 +8016,10 @@ public class GroupByQueryRunnerTest
           {
             // simulate two daily segments
             final QueryPlus<Row> queryPlus1 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-02/2011-04-03")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
             );
             final QueryPlus<Row> queryPlus2 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-03/2011-04-04")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
             );
 
             return factory.getToolchest().mergeResults(
@@ -9049,17 +8060,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setInterval("2011-04-02/2011-04-04")
-        .setDimensions(Lists.newArrayList(
-            new DefaultDimensionSpec("quality", "alias"),
-            new DefaultDimensionSpec("market", "market")
-                       )
-        )
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"), new DefaultDimensionSpec("market", "market"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setLimitSpec(
             new DefaultLimitSpec(
                 Lists.newArrayList(
@@ -9099,10 +8101,10 @@ public class GroupByQueryRunnerTest
           {
             // simulate two daily segments
             final QueryPlus<Row> queryPlus1 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-02/2011-04-03")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
             );
             final QueryPlus<Row> queryPlus2 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-04-03/2011-04-04")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
             );
 
             return factory.getToolchest().mergeResults(
@@ -9141,36 +8143,22 @@ public class GroupByQueryRunnerTest
 
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
-        .setGranularity(QueryRunnerTestHelper.allGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    QueryRunnerTestHelper.marketDimension,
-                    "marketalias"
-                )
-            )
-        )
+        .setGranularity(QueryRunnerTestHelper.allGran).setDimensions(new DefaultDimensionSpec(
+            QueryRunnerTestHelper.marketDimension,
+            "marketalias"
+        ))
         .setInterval(QueryRunnerTestHelper.fullOnInterval)
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "constant",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ),
+                Collections.singletonList(new OrderByColumnSpec(
+                    "constant",
+                    OrderByColumnSpec.Direction.DESCENDING
+                )),
                 2
             )
-        )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.rowsCount
-            )
-        )
+        ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
         .setPostAggregatorSpecs(
-            Lists.newArrayList(
-                new ConstantPostAggregator("constant", 1)
-            )
+            Collections.singletonList(new ConstantPostAggregator("constant", 1))
         )
         .setContext(
             ImmutableMap.of(
@@ -9190,21 +8178,14 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.emptyInterval)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
+                Collections.singletonList(new OrderByColumnSpec(
                         "alias",
                         OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ),
+                    )),
                 5
             )
         )
@@ -9215,17 +8196,8 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(subquery)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setAggregatorSpecs(
-            Collections.singletonList(
-                new DoubleMaxAggregatorFactory("idx", "idx")
-            )
-        )
-        .setLimitSpec(
-            new DefaultLimitSpec(
-                null,
-                5
-            )
-        )
+        .setAggregatorSpecs(new DoubleMaxAggregatorFactory("idx", "idx"))
+        .setLimitSpec(new DefaultLimitSpec(null, 5))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
@@ -9241,7 +8213,7 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
         .setDimFilter(new JavaScriptDimFilter(
             "quality",
             "function(dim){ return true; }",
@@ -9250,21 +8222,14 @@ public class GroupByQueryRunnerTest
         ))
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "alias",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ),
+                Collections.singletonList(new OrderByColumnSpec("alias", OrderByColumnSpec.Direction.DESCENDING)),
                 12
             )
         )
         .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index"),
-                new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")
-            )
+            QueryRunnerTestHelper.rowsCount,
+            new LongSumAggregatorFactory("idx", "index"),
+            new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen")
         )
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
@@ -9279,24 +8244,16 @@ public class GroupByQueryRunnerTest
                     Intervals.of("2011-04-02T00:00:00.000Z/2011-04-03T00:00:00.000Z")
                 )
             )
-        )
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias")))
+        ).setDimensions(new DefaultDimensionSpec("alias", "alias"))
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "alias",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ),
+                Collections.singletonList(new OrderByColumnSpec("alias", OrderByColumnSpec.Direction.DESCENDING)),
                 15
             )
         )
         .setAggregatorSpecs(
-            Arrays.asList(
-                new LongSumAggregatorFactory("rows", "rows"),
-                new LongSumAggregatorFactory("idx", "idx")
-            )
+            new LongSumAggregatorFactory("rows", "rows"),
+            new LongSumAggregatorFactory("idx", "idx")
         )
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
@@ -9332,40 +8289,17 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setGranularity(QueryRunnerTestHelper.allGran)
-        .setDimensions(
-            Collections.singletonList(
-                new DefaultDimensionSpec(
-                    QueryRunnerTestHelper.marketDimension,
-                    "marketalias"
-                )
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec(QueryRunnerTestHelper.marketDimension, "marketalias"))
         .setInterval(QueryRunnerTestHelper.fullOnInterval)
         .setLimitSpec(
             new DefaultLimitSpec(
-                Lists.newArrayList(
-                    new OrderByColumnSpec(
-                        "marketalias",
-                        OrderByColumnSpec.Direction.DESCENDING
-                    )
-                ),
+                Collections.singletonList(new OrderByColumnSpec("marketalias", OrderByColumnSpec.Direction.DESCENDING)),
                 2
             )
         )
-        .setAggregatorSpecs(
-            Lists.newArrayList(
-                QueryRunnerTestHelper.rowsCount
-            )
-        )
-        .setContext(
-            ImmutableMap.of(
-                GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN,
-                true
-            )
-        )
-        .setHavingSpec(
-            new GreaterThanHavingSpec("rows", 10)
-        )
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount)
+        .setContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, true))
+        .setHavingSpec(new GreaterThanHavingSpec("rows", 10))
         .build();
   }
 
@@ -9381,21 +8315,12 @@ public class GroupByQueryRunnerTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(
+        .setDimensions(
             new DefaultDimensionSpec("quality", "alias"),
             new ExtractionDimensionSpec("quality", "qualityLen", ValueType.LONG, StrlenExtractionFn.instance())
-        ))
-        .setDimFilter(new SelectorDimFilter(
-            "quality",
-            "technology",
-            null
-        ))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
         )
+        .setDimFilter(new SelectorDimFilter("quality", "technology", null))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java
index 0be6412..eee80eb 100644
--- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java
@@ -22,7 +22,6 @@ package io.druid.query.groupby;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Ordering;
 import io.druid.data.input.MapBasedRow;
 import io.druid.data.input.Row;
@@ -46,7 +45,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 public class GroupByQueryTest
@@ -60,13 +59,8 @@ public class GroupByQueryTest
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
         .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .setPostAggregatorSpecs(ImmutableList.of(new FieldAccessPostAggregator("x", "idx")))
         .setLimitSpec(
@@ -110,11 +104,11 @@ public class GroupByQueryTest
   @Test
   public void testSegmentLookUpForNestedQueries()
   {
-    QuerySegmentSpec innerQuerySegmentSpec = new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of(
+    QuerySegmentSpec innerQuerySegmentSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of(
         "2011-11-07/2011-11-08")));
-    QuerySegmentSpec outerQuerySegmentSpec = new MultipleIntervalSegmentSpec(Lists.newArrayList((Intervals.of(
+    QuerySegmentSpec outerQuerySegmentSpec = new MultipleIntervalSegmentSpec(Collections.singletonList((Intervals.of(
         "2011-11-04/2011-11-08"))));
-    List<AggregatorFactory> aggs = Lists.newArrayList(QueryRunnerTestHelper.rowsCount);
+    List<AggregatorFactory> aggs = Collections.singletonList(QueryRunnerTestHelper.rowsCount);
     final GroupByQuery innerQuery = GroupByQuery.builder()
                                                 .setDataSource("blah")
                                                 .setInterval(innerQuerySegmentSpec)
diff --git a/processing/src/test/java/io/druid/query/groupby/orderby/TopNSequenceTest.java b/processing/src/test/java/io/druid/query/groupby/orderby/TopNSequenceTest.java
index ddd1f62..728040c 100644
--- a/processing/src/test/java/io/druid/query/groupby/orderby/TopNSequenceTest.java
+++ b/processing/src/test/java/io/druid/query/groupby/orderby/TopNSequenceTest.java
@@ -43,7 +43,7 @@ public class TopNSequenceTest
   private static final Ordering<String> DESC = Ordering.natural().reverse();
 
   private static final List<String> EMPTY = Collections.EMPTY_LIST;
-  private static final List<String> SINGLE = Lists.newArrayList("a");
+  private static final List<String> SINGLE = Collections.singletonList("a");
   private static final List<String> RAW_ASC = Lists.newArrayList(Splitter.fixedLength(1).split("abcdefghijk"));
   private static final List<String> RAW_DESC = Lists.newArrayList(Splitter.fixedLength(1).split("kjihgfedcba"));
 
diff --git a/processing/src/test/java/io/druid/query/lookup/LookupExtractorTest.java b/processing/src/test/java/io/druid/query/lookup/LookupExtractorTest.java
index a8f98de..b37a910 100644
--- a/processing/src/test/java/io/druid/query/lookup/LookupExtractorTest.java
+++ b/processing/src/test/java/io/druid/query/lookup/LookupExtractorTest.java
@@ -21,7 +21,6 @@ package io.druid.query.lookup;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import io.druid.jackson.DefaultObjectMapper;
 import io.druid.query.extraction.MapLookupExtractor;
 import org.junit.Assert;
@@ -88,7 +87,7 @@ public class LookupExtractorTest
   {
     Map<String, String> expected = new HashMap<>();
     expected.put("not there", null);
-    Assert.assertEquals(expected, lookupExtractor.applyAll(Lists.newArrayList("not there")));
+    Assert.assertEquals(expected, lookupExtractor.applyAll(Collections.singletonList("not there")));
   }
 
   @Test
diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java
index 31587a2..ba78ea5 100644
--- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java
+++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java
@@ -21,7 +21,6 @@ package io.druid.query.metadata;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import io.druid.java.util.common.Intervals;
 import io.druid.query.Druids;
@@ -42,6 +41,7 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import java.util.Collections;
 import java.util.List;
 
 @RunWith(Parameterized.class)
@@ -96,7 +96,7 @@ public class SegmentMetadataUnionQueryTest
   {
     SegmentAnalysis expected = new SegmentAnalysis(
         QueryRunnerTestHelper.segmentId,
-        Lists.newArrayList(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
+        Collections.singletonList(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
         ImmutableMap.of(
             "placement",
             new ColumnAnalysis(
@@ -119,7 +119,7 @@ public class SegmentMetadataUnionQueryTest
     SegmentMetadataQuery query = new Druids.SegmentMetadataQueryBuilder()
         .dataSource(QueryRunnerTestHelper.unionDataSource)
         .intervals(QueryRunnerTestHelper.fullOnInterval)
-        .toInclude(new ListColumnIncluderator(Lists.newArrayList("placement")))
+        .toInclude(new ListColumnIncluderator(Collections.singletonList("placement")))
         .analysisTypes(
             SegmentMetadataQuery.AnalysisType.CARDINALITY,
             SegmentMetadataQuery.AnalysisType.SIZE,
diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java
index 6217ac6..1903957 100644
--- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java
+++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java
@@ -64,6 +64,7 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -166,10 +167,10 @@ public class SearchQueryRunnerTest
           )
           {
             final QueryPlus<Result<SearchResultValue>> queryPlus1 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-01-12/2011-02-28")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-01-12/2011-02-28")))
             );
             final QueryPlus<Result<SearchResultValue>> queryPlus2 = queryPlus.withQuerySegmentSpec(
-                new MultipleIntervalSegmentSpec(Lists.newArrayList(Intervals.of("2011-03-01/2011-04-15")))
+                new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-03-01/2011-04-15")))
             );
             return Sequences.concat(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext));
           }
diff --git a/processing/src/test/java/io/druid/query/select/SelectBinaryFnTest.java b/processing/src/test/java/io/druid/query/select/SelectBinaryFnTest.java
index f75c6e8..6c080ad 100644
--- a/processing/src/test/java/io/druid/query/select/SelectBinaryFnTest.java
+++ b/processing/src/test/java/io/druid/query/select/SelectBinaryFnTest.java
@@ -20,7 +20,6 @@
 package io.druid.query.select;
 
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import io.druid.java.util.common.DateTimes;
@@ -31,6 +30,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -221,15 +221,13 @@ public class SelectBinaryFnTest
             ImmutableMap.of(),
             Sets.newHashSet("first", "second", "fourth"),
             Sets.newHashSet("eight", "nineth"),
-            Lists.newArrayList(
+            Collections.singletonList(
                 new EventHolder(
                     segmentId1,
                     0,
-                    ImmutableMap.of(
-                        EventHolder.timestampKey,
-                        DateTimes.of("2013-01-01T00"), "dim", "first"
-                    )
-                ))
+                    ImmutableMap.of(EventHolder.timestampKey, DateTimes.of("2013-01-01T00"), "dim", "first")
+                )
+            )
         )
     );
 
@@ -239,17 +237,13 @@ public class SelectBinaryFnTest
             ImmutableMap.of(),
             Sets.newHashSet("third", "second", "fifth"),
             Sets.newHashSet("seventh"),
-            Lists.newArrayList(
+            Collections.singletonList(
                 new EventHolder(
                     segmentId2,
                     0,
-                    ImmutableMap.of(
-                        EventHolder.timestampKey,
-                        DateTimes.of("2013-01-01T00"),
-                        "dim",
-                        "second"
-                    )
-                ))
+                    ImmutableMap.of(EventHolder.timestampKey, DateTimes.of("2013-01-01T00"), "dim", "second")
+                )
+            )
         )
     );
 
diff --git a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java
index 87f2a5b..7898ee4 100644
--- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java
+++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java
@@ -393,8 +393,8 @@ public class SelectQueryRunnerTest
             },
             V_0112_0114
         ),
-        Lists.newArrayList("market"),
-        Lists.newArrayList("index"),
+        Collections.singletonList("market"),
+        Collections.singletonList("index"),
         offset.startOffset(),
         offset.threshold()
     );
@@ -423,8 +423,8 @@ public class SelectQueryRunnerTest
             },
             V_0112_0114
         ),
-        Lists.newArrayList("quality"),
-        Lists.newArrayList("index"),
+        Collections.singletonList("quality"),
+        Collections.singletonList("index"),
         offset.startOffset(),
         offset.threshold()
     );
@@ -441,7 +441,7 @@ public class SelectQueryRunnerTest
           .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null))
           .granularity(QueryRunnerTestHelper.dayGran)
           .dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.qualityDimension))
-          .metrics(Lists.newArrayList(QueryRunnerTestHelper.indexMetric))
+          .metrics(Collections.singletonList(QueryRunnerTestHelper.indexMetric))
           .pagingSpec(new PagingSpec(toPagingIdentifier(param[0], descending), param[1]))
           .build();
 
@@ -485,8 +485,8 @@ public class SelectQueryRunnerTest
       PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId);
       List<Result<SelectResultValue>> expectedResults = toExpected(
           events,
-          Lists.newArrayList("quality"),
-          Lists.newArrayList("index"),
+          Collections.singletonList("quality"),
+          Collections.singletonList("index"),
           offset.startOffset(),
           offset.threshold()
       );
@@ -509,7 +509,7 @@ public class SelectQueryRunnerTest
         )
         .granularity(QueryRunnerTestHelper.allGran)
         .dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.qualityDimension))
-        .metrics(Lists.newArrayList(QueryRunnerTestHelper.indexMetric))
+        .metrics(Collections.singletonList(QueryRunnerTestHelper.indexMetric))
         .pagingSpec(new PagingSpec(null, 10, true))
         .virtualColumns(
             new ExpressionVirtualColumn("expr", "index / 10.0", ValueType.FLOAT, TestExprMacroTable.INSTANCE)
@@ -538,8 +538,8 @@ public class SelectQueryRunnerTest
     PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId);
     List<Result<SelectResultValue>> expectedResults = toExpected(
         events,
-        Lists.newArrayList("quality"),
-        Lists.newArrayList("index"),
+        Collections.singletonList("quality"),
+        Collections.singletonList("index"),
         offset.startOffset(),
         offset.threshold()
     );
@@ -559,7 +559,7 @@ public class SelectQueryRunnerTest
         .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "replaced", lookupExtractionFn))
         .granularity(QueryRunnerTestHelper.dayGran)
         .dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.qualityDimension))
-        .metrics(Lists.newArrayList(QueryRunnerTestHelper.indexMetric))
+        .metrics(Collections.singletonList(QueryRunnerTestHelper.indexMetric))
         .build();
 
     Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), Maps.newHashMap()).toList();
@@ -591,8 +591,8 @@ public class SelectQueryRunnerTest
     PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId);
     List<Result<SelectResultValue>> expectedResults = toExpected(
         events,
-        Lists.newArrayList(QueryRunnerTestHelper.qualityDimension),
-        Lists.newArrayList(QueryRunnerTestHelper.indexMetric),
+        Collections.singletonList(QueryRunnerTestHelper.qualityDimension),
+        Collections.singletonList(QueryRunnerTestHelper.indexMetric),
         offset.startOffset(),
         offset.threshold()
     );
@@ -658,7 +658,7 @@ public class SelectQueryRunnerTest
     SelectQuery query = newTestQuery()
         .intervals(I_0112_0114)
         .dimensionSpecs(DefaultDimensionSpec.toSpec("foo"))
-        .metrics(Lists.newArrayList("foo2"))
+        .metrics(Collections.singletonList("foo2"))
         .build();
 
     Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), Maps.newHashMap()).toList();
@@ -675,8 +675,8 @@ public class SelectQueryRunnerTest
     PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId);
     List<Result<SelectResultValue>> expectedResults = toExpected(
         events,
-        Lists.newArrayList("foo"),
-        Lists.newArrayList("foo2"),
+        Collections.singletonList("foo"),
+        Collections.singletonList("foo2"),
         offset.startOffset(),
         offset.threshold()
     );
diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java
index 81cd78f..9f748cb 100644
--- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java
+++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java
@@ -49,7 +49,6 @@ import io.druid.query.expression.TestExprMacroTable;
 import io.druid.query.extraction.MapLookupExtractor;
 import io.druid.query.filter.AndDimFilter;
 import io.druid.query.filter.BoundDimFilter;
-import io.druid.query.filter.DimFilter;
 import io.druid.query.filter.InDimFilter;
 import io.druid.query.filter.NotDimFilter;
 import io.druid.query.filter.OrDimFilter;
@@ -2023,12 +2022,10 @@ public class TimeseriesQueryRunnerTest
             Lists.newArrayList(
                 Iterables.concat(
                     aggregatorFactoryList,
-                    Lists.newArrayList(
-                        new FilteredAggregatorFactory(
+                    Collections.singletonList(new FilteredAggregatorFactory(
                             new CountAggregatorFactory("filteredAgg"),
                             new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)
-                        )
-                    )
+                        ))
                 )
             )
         )
@@ -2067,7 +2064,7 @@ public class TimeseriesQueryRunnerTest
             Lists.newArrayList(
                 Iterables.concat(
                     aggregatorFactoryList,
-                    Lists.newArrayList(
+                    Collections.singletonList(
                         new FilteredAggregatorFactory(
                             new CountAggregatorFactory("filteredAgg"),
                             new SelectorDimFilter("abraKaDabra", "Lol", null)
@@ -2112,7 +2109,7 @@ public class TimeseriesQueryRunnerTest
             Lists.newArrayList(
                 Iterables.concat(
                     aggregatorFactoryList,
-                    Lists.newArrayList(
+                    Collections.singletonList(
                         new FilteredAggregatorFactory(
                             new CountAggregatorFactory("filteredAgg"),
                             new SelectorDimFilter("abraKaDabra", null, null)
@@ -2157,7 +2154,7 @@ public class TimeseriesQueryRunnerTest
             Lists.newArrayList(
                 Iterables.concat(
                     aggregatorFactoryList,
-                    Lists.newArrayList(
+                    Collections.singletonList(
                         new FilteredAggregatorFactory(
                             new CountAggregatorFactory("filteredAgg"),
                             new NotDimFilter(
@@ -2203,7 +2200,7 @@ public class TimeseriesQueryRunnerTest
             Lists.newArrayList(
                 Iterables.concat(
                     aggregatorFactoryList,
-                    Lists.newArrayList(
+                    Collections.singletonList(
                         new FilteredAggregatorFactory(
                             new CountAggregatorFactory("filteredAgg"),
                             new NotDimFilter(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, null, null))
@@ -2242,11 +2239,9 @@ public class TimeseriesQueryRunnerTest
                                   .dataSource(QueryRunnerTestHelper.dataSource)
                                   .intervals(QueryRunnerTestHelper.firstToThird)
                                   .aggregators(
-                                      Arrays.asList(
-                                          QueryRunnerTestHelper.rowsCount,
-                                          QueryRunnerTestHelper.jsCountIfTimeGreaterThan,
-                                          QueryRunnerTestHelper.__timeLongSum
-                                      )
+                                      QueryRunnerTestHelper.rowsCount,
+                                      QueryRunnerTestHelper.jsCountIfTimeGreaterThan,
+                                      QueryRunnerTestHelper.__timeLongSum
                                   )
                                   .granularity(QueryRunnerTestHelper.allGran)
                                   .descending(descending)
@@ -2302,7 +2297,7 @@ public class TimeseriesQueryRunnerTest
                                                   null,
                                                   StringComparators.LEXICOGRAPHIC
                                               ),
-                                              (DimFilter) new BoundDimFilter(
+                                              new BoundDimFilter(
                                                   QueryRunnerTestHelper.marketDimension,
                                                   "SPOT",
                                                   "spot",
diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java
index ffbcb33..547edc4 100644
--- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java
+++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java
@@ -1720,11 +1720,8 @@ public class TopNQueryRunnerTest
         .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
         .build();
     assertExpectedResults(
-        Lists.newArrayList(
-            new Result<TopNResultValue>(
-                DateTimes.of("2011-04-01T00:00:00.000Z"),
-                new TopNResultValue(Lists.<Map<String, Object>>newArrayList())
-            )
+        Collections.singletonList(
+            new Result<>(DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue(Collections.emptyList()))
         ),
         query
     );
@@ -1749,12 +1746,10 @@ public class TopNQueryRunnerTest
         .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
         .build();
     assertExpectedResults(
-        Lists.newArrayList(
-            new Result<TopNResultValue>(
-                DateTimes.of("2011-04-01T00:00:00.000Z"),
-                new TopNResultValue(Lists.<Map<String, Object>>newArrayList())
-            )
-        ), query
+        Collections.singletonList(
+            new Result<>(DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue(Collections.emptyList()))
+        ),
+        query
     );
   }
 
@@ -1844,7 +1839,7 @@ public class TopNQueryRunnerTest
         .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
         .build();
 
-    final ArrayList<Result<TopNResultValue>> expectedResults = Lists.newArrayList(
+    final List<Result<TopNResultValue>> expectedResults = Collections.singletonList(
         new Result<>(
             DateTimes.of("2011-04-01T00:00:00.000Z"),
             new TopNResultValue(
@@ -1885,7 +1880,7 @@ public class TopNQueryRunnerTest
         .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
         .build();
 
-    final ArrayList<Result<TopNResultValue>> expectedResults = Lists.newArrayList(
+    final List<Result<TopNResultValue>> expectedResults = Collections.singletonList(
         new Result<>(
             DateTimes.of("2011-04-01T00:00:00.000Z"),
             new TopNResultValue(
@@ -1933,8 +1928,8 @@ public class TopNQueryRunnerTest
         .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
         .build();
 
-    final ArrayList<Result<TopNResultValue>> expectedResults = Lists.newArrayList(
-        new Result<>(
+    final List<Result<TopNResultValue>> expectedResults = Collections.singletonList(
+        new Result<TopNResultValue>(
             DateTimes.of("2011-04-01T00:00:00.000Z"),
             new TopNResultValue(
                 Arrays.<Map<String, Object>>asList(
diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java
index d22de5c..5720101 100644
--- a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java
+++ b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java
@@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import io.druid.query.Query;
-import io.druid.query.aggregation.AggregatorFactory;
 import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
 import io.druid.query.aggregation.DoubleMinAggregatorFactory;
 import io.druid.query.dimension.ExtractionDimensionSpec;
@@ -125,7 +124,7 @@ public class TopNQueryTest
         .metric(new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC))
         .threshold(2)
         .intervals(fullOnInterval.getIntervals())
-        .aggregators(Lists.<AggregatorFactory>newArrayList(rowsCount))
+        .aggregators(Collections.singletonList(rowsCount))
         .build();
     String jsonQuery = "{\n"
                        + "  \"queryType\": \"topN\",\n"
diff --git a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java
index 0f096c4..2592f14 100644
--- a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java
+++ b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java
@@ -21,7 +21,6 @@ package io.druid.segment;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import io.druid.collections.bitmap.ConciseBitmapFactory;
 import io.druid.java.util.common.Intervals;
 import io.druid.query.aggregation.AggregatorFactory;
@@ -39,6 +38,7 @@ import org.junit.runners.Parameterized;
 
 import java.io.File;
 import java.util.Collection;
+import java.util.Collections;
 
 @RunWith(Parameterized.class)
 public class EmptyIndexTest
@@ -83,7 +83,7 @@ public class EmptyIndexTest
           new ConciseBitmapFactory()
       );
       TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory).merge(
-          Lists.newArrayList(emptyIndexAdapter),
+          Collections.singletonList(emptyIndexAdapter),
           true,
           new AggregatorFactory[0],
           tmpDir,
diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java
index d8e4d2d..69e493d 100644
--- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java
+++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java
@@ -115,14 +115,14 @@ public class IncrementalIndexStorageAdapterTest
     index.add(
         new MapBasedInputRow(
             System.currentTimeMillis() - 1,
-            Lists.newArrayList("billy"),
+            Collections.singletonList("billy"),
             ImmutableMap.of("billy", "hi")
         )
     );
     index.add(
         new MapBasedInputRow(
             System.currentTimeMillis() - 1,
-            Lists.newArrayList("sally"),
+            Collections.singletonList("sally"),
             ImmutableMap.of("sally", "bo")
         )
     );
@@ -159,7 +159,7 @@ public class IncrementalIndexStorageAdapterTest
     index.add(
         new MapBasedInputRow(
             DateTimes.of("2014-09-01T00:00:00"),
-            Lists.newArrayList("billy"),
+            Collections.singletonList("billy"),
             ImmutableMap.of("billy", "hi")
         )
     );
@@ -249,14 +249,14 @@ public class IncrementalIndexStorageAdapterTest
     index.add(
         new MapBasedInputRow(
             t.minus(1).getMillis(),
-            Lists.newArrayList("billy"),
+            Collections.singletonList("billy"),
             ImmutableMap.of("billy", "hi")
         )
     );
     index.add(
         new MapBasedInputRow(
             t.minus(1).getMillis(),
-            Lists.newArrayList("sally"),
+            Collections.singletonList("sally"),
             ImmutableMap.of("sally", "bo")
         )
     );
@@ -284,7 +284,7 @@ public class IncrementalIndexStorageAdapterTest
       index.add(
           new MapBasedInputRow(
               t.minus(1).getMillis(),
-              Lists.newArrayList("sally"),
+              Collections.singletonList("sally"),
               ImmutableMap.of("sally", "ah")
           )
       );
@@ -307,7 +307,7 @@ public class IncrementalIndexStorageAdapterTest
     index.add(
         new MapBasedInputRow(
             t.minus(1).getMillis(),
-            Lists.newArrayList("sally"),
+            Collections.singletonList("sally"),
             ImmutableMap.of("sally", "bo")
         )
     );
@@ -331,7 +331,7 @@ public class IncrementalIndexStorageAdapterTest
             new TopNQueryBuilder()
                 .dataSource("test")
                 .granularity(Granularities.ALL)
-                .intervals(Lists.newArrayList(new Interval(DateTimes.EPOCH, DateTimes.nowUtc())))
+                .intervals(Collections.singletonList(new Interval(DateTimes.EPOCH, DateTimes.nowUtc())))
                 .dimension("sally")
                 .metric("cnt")
                 .threshold(10)
@@ -353,14 +353,14 @@ public class IncrementalIndexStorageAdapterTest
     index.add(
         new MapBasedInputRow(
             System.currentTimeMillis() - 1,
-            Lists.newArrayList("billy"),
+            Collections.singletonList("billy"),
             ImmutableMap.of("billy", "hi")
         )
     );
     index.add(
         new MapBasedInputRow(
             System.currentTimeMillis() - 1,
-            Lists.newArrayList("sally"),
+            Collections.singletonList("sally"),
             ImmutableMap.of("sally", "bo")
         )
     );
@@ -398,7 +398,7 @@ public class IncrementalIndexStorageAdapterTest
       index.add(
           new MapBasedInputRow(
               timestamp,
-              Lists.newArrayList("billy"),
+              Collections.singletonList("billy"),
               ImmutableMap.of("billy", "v1" + i)
           )
       );
@@ -426,7 +426,7 @@ public class IncrementalIndexStorageAdapterTest
           //index gets more rows at this point, while other thread is iterating over the cursor
           try {
             for (int i = 0; i < 1; i++) {
-              index.add(new MapBasedInputRow(timestamp, Lists.newArrayList("billy"), ImmutableMap.of("billy", "v2" + i)));
+              index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v2" + i)));
             }
           }
           catch (Exception ex) {
@@ -460,7 +460,7 @@ public class IncrementalIndexStorageAdapterTest
       index.add(
           new MapBasedInputRow(
               timestamp,
-              Lists.newArrayList("billy"),
+              Collections.singletonList("billy"),
               ImmutableMap.of("billy", "v0" + i)
           )
       );
@@ -487,7 +487,7 @@ public class IncrementalIndexStorageAdapterTest
 
           //index gets more rows at this point, while other thread is iterating over the cursor
           try {
-            index.add(new MapBasedInputRow(timestamp, Lists.newArrayList("billy"), ImmutableMap.of("billy", "v1")));
+            index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v1")));
           }
           catch (Exception ex) {
             throw new RuntimeException(ex);
@@ -498,8 +498,8 @@ public class IncrementalIndexStorageAdapterTest
               .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy"));
           //index gets more rows at this point, while other thread is iterating over the cursor
           try {
-            index.add(new MapBasedInputRow(timestamp, Lists.newArrayList("billy"), ImmutableMap.of("billy", "v2")));
-            index.add(new MapBasedInputRow(timestamp, Lists.newArrayList("billy2"), ImmutableMap.of("billy2", "v3")));
+            index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v2")));
+            index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy2"), ImmutableMap.of("billy2", "v3")));
           }
           catch (Exception ex) {
             throw new RuntimeException(ex);
@@ -514,8 +514,8 @@ public class IncrementalIndexStorageAdapterTest
               .makeDimensionSelector(new DefaultDimensionSpec("billy2", "billy2"));
           //index gets more rows at this point, while other thread is iterating over the cursor
           try {
-            index.add(new MapBasedInputRow(timestamp, Lists.newArrayList("billy"), ImmutableMap.of("billy", "v3")));
-            index.add(new MapBasedInputRow(timestamp, Lists.newArrayList("billy3"), ImmutableMap.of("billy3", "")));
+            index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v3")));
+            index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy3"), ImmutableMap.of("billy3", "")));
           }
           catch (Exception ex) {
             throw new RuntimeException(ex);
diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexTest.java
index 0c41ca8..8678a9a 100644
--- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexTest.java
+++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexTest.java
@@ -20,7 +20,6 @@
 package io.druid.segment.incremental;
 
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import io.druid.data.input.MapBasedInputRow;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.query.aggregation.Aggregator;
@@ -30,6 +29,7 @@ import org.easymock.EasyMock;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.util.Collections;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -64,7 +64,7 @@ public class OnheapIncrementalIndexTest
             for (int j = 0; j < MAX_ROWS / addThreadCount; ++j) {
               index.add(new MapBasedInputRow(
                   0,
-                  Lists.newArrayList("billy"),
+                  Collections.singletonList("billy"),
                   ImmutableMap.of("billy", random.nextLong(), "max", 1)
               ));
             }
@@ -122,7 +122,7 @@ public class OnheapIncrementalIndexTest
 
     index.add(new MapBasedInputRow(
             0,
-            Lists.newArrayList("billy"),
+            Collections.singletonList("billy"),
             ImmutableMap.of("billy", 1, "max", 1)
     ));
 
diff --git a/server/src/main/java/io/druid/server/ClientInfoResource.java b/server/src/main/java/io/druid/server/ClientInfoResource.java
index e40b3a8..16e1eeb 100644
--- a/server/src/main/java/io/druid/server/ClientInfoResource.java
+++ b/server/src/main/java/io/druid/server/ClientInfoResource.java
@@ -121,7 +121,7 @@ public class ClientInfoResource
   public Iterable<String> getDataSources(@Context final HttpServletRequest request)
   {
     Function<String, Iterable<ResourceAction>> raGenerator = datasourceName -> {
-      return Lists.newArrayList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName));
+      return Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName));
     };
 
     return AuthorizationUtils.filterAuthorizedResources(
diff --git a/server/src/main/java/io/druid/server/http/InventoryViewUtils.java b/server/src/main/java/io/druid/server/http/InventoryViewUtils.java
index f1ac7c4..da89fff 100644
--- a/server/src/main/java/io/druid/server/http/InventoryViewUtils.java
+++ b/server/src/main/java/io/druid/server/http/InventoryViewUtils.java
@@ -19,7 +19,6 @@
 
 package io.druid.server.http;
 
-import com.google.common.collect.Lists;
 import io.druid.client.DruidDataSource;
 import io.druid.client.ImmutableDruidDataSource;
 import io.druid.client.InventoryView;
@@ -66,9 +65,8 @@ public interface InventoryViewUtils
     Iterable<ImmutableDruidDataSource> filteredResources = AuthorizationUtils.filterAuthorizedResources(
         request,
         getDataSources(inventoryView),
-        datasource -> Lists.newArrayList(
-            AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasource.getName())
-        ),
+        datasource ->
+            Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasource.getName())),
         authorizerMapper
     );
     SortedSet<ImmutableDruidDataSource> set = new TreeSet<>(comparingByName());
diff --git a/server/src/main/java/io/druid/server/http/MetadataResource.java b/server/src/main/java/io/druid/server/http/MetadataResource.java
index f0ed5c1..fe88f42 100644
--- a/server/src/main/java/io/druid/server/http/MetadataResource.java
+++ b/server/src/main/java/io/druid/server/http/MetadataResource.java
@@ -22,7 +22,6 @@ package io.druid.server.http;
 import com.google.common.base.Function;
 import com.google.common.collect.Collections2;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.inject.Inject;
 import com.sun.jersey.spi.container.ResourceFilters;
@@ -48,6 +47,7 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
@@ -96,7 +96,7 @@ public class MetadataResource
 
     final Set<String> dataSourceNamesPostAuth = Sets.newTreeSet();
     Function<String, Iterable<ResourceAction>> raGenerator = datasourceName -> {
-      return Lists.newArrayList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName));
+      return Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName));
     };
 
     Iterables.addAll(
diff --git a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java
index abce4c1..2c13848 100644
--- a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java
+++ b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java
@@ -27,6 +27,7 @@ import com.google.common.collect.Sets;
 import io.druid.java.util.common.ISE;
 
 import javax.servlet.http.HttpServletRequest;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -59,7 +60,7 @@ public class AuthorizationUtils
   {
     return authorizeAllResourceActions(
         request,
-        Lists.newArrayList(resourceAction),
+        Collections.singletonList(resourceAction),
         authorizerMapper
     );
   }
diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java
index e127283..31f68ff 100644
--- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java
+++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java
@@ -1426,7 +1426,7 @@ public class CachingClusteredClientTest
 
     SelectQuery query = builder
         .intervals("2011-01-01/2011-01-10")
-        .dimensionSpecs(Lists.newArrayList(new DefaultDimensionSpec("a", "a2")))
+        .dimensionSpecs(Collections.singletonList(new DefaultDimensionSpec("a", "a2")))
         .build();
     TestHelper.assertExpectedResults(
         makeSelectResults(
@@ -1462,8 +1462,7 @@ public class CachingClusteredClientTest
         .setDataSource(DATA_SOURCE)
         .setQuerySegmentSpec(SEG_SPEC)
         .setDimFilter(DIM_FILTER)
-        .setGranularity(GRANULARITY)
-        .setDimensions(Collections.singletonList(new DefaultDimensionSpec("a", "a")))
+        .setGranularity(GRANULARITY).setDimensions(new DefaultDimensionSpec("a", "a"))
         .setAggregatorSpecs(aggsWithUniques)
         .setPostAggregatorSpecs(POST_AGGS)
         .setContext(CONTEXT);
@@ -2974,8 +2973,7 @@ public class CachingClusteredClientTest
         .setDataSource(DATA_SOURCE)
         .setQuerySegmentSpec(SEG_SPEC)
         .setDimFilter(DIM_FILTER)
-        .setGranularity(GRANULARITY)
-        .setDimensions(Collections.singletonList(new DefaultDimensionSpec("a", "output")))
+        .setGranularity(GRANULARITY).setDimensions(new DefaultDimensionSpec("a", "output"))
         .setAggregatorSpecs(AGGS)
         .setContext(CONTEXT);
 
@@ -3036,8 +3034,7 @@ public class CachingClusteredClientTest
     );
 
     GroupByQuery query = builder
-        .setInterval("2011-01-05/2011-01-10")
-        .setDimensions(Collections.singletonList(new DefaultDimensionSpec("a", "output2")))
+        .setInterval("2011-01-05/2011-01-10").setDimensions(new DefaultDimensionSpec("a", "output2"))
         .setAggregatorSpecs(RENAMED_AGGS)
         .build();
     TestHelper.assertExpectedObjects(
diff --git a/server/src/test/java/io/druid/client/cache/CaffeineCacheTest.java b/server/src/test/java/io/druid/client/cache/CaffeineCacheTest.java
index 19f70c6..d9f9245 100644
--- a/server/src/test/java/io/druid/client/cache/CaffeineCacheTest.java
+++ b/server/src/test/java/io/druid/client/cache/CaffeineCacheTest.java
@@ -38,6 +38,7 @@ import org.junit.Test;
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
+import java.util.Collections;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
@@ -166,7 +167,7 @@ public class CaffeineCacheTest
     Assert.assertEquals(10, Ints.fromByteArray(result.get(key2)));
 
     Cache.NamedKey missingKey = new Cache.NamedKey("missing", HI);
-    result = cache.getBulk(Lists.newArrayList(missingKey));
+    result = cache.getBulk(Collections.singletonList(missingKey));
     Assert.assertEquals(result.size(), 0);
 
     result = cache.getBulk(Lists.newArrayList());
diff --git a/server/src/test/java/io/druid/client/indexing/ClientAppendQueryTest.java b/server/src/test/java/io/druid/client/indexing/ClientAppendQueryTest.java
index c0cfca4..0d0ed59 100644
--- a/server/src/test/java/io/druid/client/indexing/ClientAppendQueryTest.java
+++ b/server/src/test/java/io/druid/client/indexing/ClientAppendQueryTest.java
@@ -19,7 +19,6 @@
 
 package io.druid.client.indexing;
 
-import com.google.common.collect.Lists;
 import io.druid.java.util.common.DateTimes;
 import io.druid.timeline.DataSegment;
 import org.joda.time.DateTime;
@@ -28,6 +27,7 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.Collections;
 import java.util.List;
 
 public class ClientAppendQueryTest
@@ -35,9 +35,9 @@ public class ClientAppendQueryTest
   private ClientAppendQuery clientAppendQuery;
   private static final String DATA_SOURCE = "data_source";
   private final DateTime start = DateTimes.nowUtc();
-  private List<DataSegment> segments = Lists.newArrayList(
-      new DataSegment(DATA_SOURCE, new Interval(start, start.plus(1)), start.toString(), null,
-                      null, null, null, 0, 0));
+  private List<DataSegment> segments = Collections.singletonList(
+      new DataSegment(DATA_SOURCE, new Interval(start, start.plus(1)), start.toString(), null, null, null, null, 0, 0)
+  );
 
   @Before
   public void setUp()
diff --git a/server/src/test/java/io/druid/client/indexing/ClientMergeQueryTest.java b/server/src/test/java/io/druid/client/indexing/ClientMergeQueryTest.java
index 6b99e52..a1404a5 100644
--- a/server/src/test/java/io/druid/client/indexing/ClientMergeQueryTest.java
+++ b/server/src/test/java/io/druid/client/indexing/ClientMergeQueryTest.java
@@ -28,6 +28,7 @@ import org.joda.time.Interval;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.util.Collections;
 import java.util.List;
 
 public class ClientMergeQueryTest
@@ -37,7 +38,7 @@ public class ClientMergeQueryTest
   private static final Interval INTERVAL = new Interval(START, START.plus(1));
   private static final DataSegment DATA_SEGMENT =
       new DataSegment(DATA_SOURCE, INTERVAL, START.toString(), null, null, null, null, 0, 0);
-  private static final List<DataSegment> SEGMENT_LIST = Lists.newArrayList(DATA_SEGMENT);
+  private static final List<DataSegment> SEGMENT_LIST = Collections.singletonList(DATA_SEGMENT);
   private static final List<AggregatorFactory> AGGREGATOR_LIST = Lists.newArrayList();
   private static final ClientMergeQuery CLIENT_MERGE_QUERY =
       new ClientMergeQuery(DATA_SOURCE, SEGMENT_LIST, AGGREGATOR_LIST);
diff --git a/server/src/test/java/io/druid/initialization/ComposingEmitterModuleTest.java b/server/src/test/java/io/druid/initialization/ComposingEmitterModuleTest.java
index 0835ac1..4d0db84 100644
--- a/server/src/test/java/io/druid/initialization/ComposingEmitterModuleTest.java
+++ b/server/src/test/java/io/druid/initialization/ComposingEmitterModuleTest.java
@@ -19,7 +19,6 @@
 
 package io.druid.initialization;
 
-import com.google.common.collect.Lists;
 import com.google.inject.Binder;
 import com.google.inject.Guice;
 import com.google.inject.Injector;
@@ -37,6 +36,7 @@ import org.junit.Test;
 
 import javax.validation.Validation;
 import javax.validation.Validator;
+import java.util.Collections;
 import java.util.Properties;
 
 /**
@@ -58,7 +58,7 @@ public class ComposingEmitterModuleTest
   public void testGetEmitter()
   {
     ComposingEmitterConfig config = EasyMock.createMock(ComposingEmitterConfig.class);
-    EasyMock.expect(config.getEmitters()).andReturn(Lists.newArrayList(testEmitterType)).anyTimes();
+    EasyMock.expect(config.getEmitters()).andReturn(Collections.singletonList(testEmitterType)).anyTimes();
 
     Injector injector = EasyMock.createMock(Injector.class);
     EasyMock.expect(injector.getInstance(Key.get(Emitter.class, Names.named(testEmitterType)))).andReturn(emitter);
diff --git a/server/src/test/java/io/druid/query/lookup/LookupSnapshotTakerTest.java b/server/src/test/java/io/druid/query/lookup/LookupSnapshotTakerTest.java
index 7ebd737..4a4af93 100644
--- a/server/src/test/java/io/druid/query/lookup/LookupSnapshotTakerTest.java
+++ b/server/src/test/java/io/druid/query/lookup/LookupSnapshotTakerTest.java
@@ -21,7 +21,6 @@ package io.druid.query.lookup;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import com.google.common.io.Files;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.StringUtils;
@@ -83,9 +82,9 @@ public class LookupSnapshotTakerTest
             new MapLookupExtractorFactory(ImmutableMap.of("key", "value"), true)
         )
     );
-    List<LookupBean> lookupBeanList1 = Lists.newArrayList(lookupBean1);
+    List<LookupBean> lookupBeanList1 = Collections.singletonList(lookupBean1);
     lookupSnapshotTaker.takeSnapshot(TIER1, lookupBeanList1);
-    List<LookupBean> lookupBeanList2 = Lists.newArrayList(lookupBean2);
+    List<LookupBean> lookupBeanList2 = Collections.singletonList(lookupBean2);
     lookupSnapshotTaker.takeSnapshot(TIER2, lookupBeanList2);
     Assert.assertEquals(lookupBeanList1, lookupSnapshotTaker.pullExistingSnapshot(TIER1));
     Assert.assertEquals(lookupBeanList2, lookupSnapshotTaker.pullExistingSnapshot(TIER2));
@@ -114,7 +113,7 @@ public class LookupSnapshotTakerTest
             )
         )
     );
-    List<LookupBean> lookupBeanList = Lists.newArrayList(lookupBean);
+    List<LookupBean> lookupBeanList = Collections.singletonList(lookupBean);
 
     expectedException.expect(ISE.class);
     expectedException.expectMessage("Exception during serialization of lookups");
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 72a2945..6cee954 100644
--- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java
+++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java
@@ -487,13 +487,8 @@ public class RealtimeManagerTest
           .builder()
           .setDataSource(QueryRunnerTestHelper.dataSource)
           .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-          .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-          .setAggregatorSpecs(
-              Arrays.asList(
-                  QueryRunnerTestHelper.rowsCount,
-                  new LongSumAggregatorFactory("idx", "index")
-              )
-          )
+          .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+          .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
           .setGranularity(QueryRunnerTestHelper.dayGran)
           .build();
       plumber.setRunners(ImmutableMap.of(query.getIntervals().get(0), runner));
@@ -565,13 +560,8 @@ public class RealtimeManagerTest
           .builder()
           .setDataSource(QueryRunnerTestHelper.dataSource)
           .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
-          .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-          .setAggregatorSpecs(
-              Arrays.asList(
-                  QueryRunnerTestHelper.rowsCount,
-                  new LongSumAggregatorFactory("idx", "index")
-              )
-          )
+          .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+          .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
           .setGranularity(QueryRunnerTestHelper.dayGran)
           .build();
       plumber.setRunners(ImmutableMap.of(query.getIntervals().get(0), runner));
@@ -677,13 +667,8 @@ public class RealtimeManagerTest
                     descriptor_26_28_1,
                     descriptor_28_29_1
                 )))
-        .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias")))
-        .setAggregatorSpecs(
-            Arrays.asList(
-                QueryRunnerTestHelper.rowsCount,
-                new LongSumAggregatorFactory("idx", "index")
-            )
-        )
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index"))
         .setGranularity(QueryRunnerTestHelper.dayGran)
         .build();
 
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 45db9b1..4cf56e8 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
@@ -19,7 +19,6 @@
 
 package io.druid.segment.realtime.plumber;
 
-import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.client.ImmutableSegmentLoadInfo;
@@ -36,6 +35,7 @@ import org.joda.time.Duration;
 import org.joda.time.Interval;
 import org.junit.Test;
 
+import java.util.Collections;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 public class CoordinatorBasedSegmentHandoffNotifierTest
@@ -73,15 +73,14 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
     CoordinatorClient coordinatorClient = EasyMock.createMock(CoordinatorClient.class);
     EasyMock.expect(coordinatorClient.fetchServerView("test_ds", interval, true))
             .andReturn(
-                Lists.newArrayList(
+                Collections.singletonList(
                     new ImmutableSegmentLoadInfo(
                         segment,
-                        Sets.newHashSet(
-                            createRealtimeServerMetadata("a1")
-                        )
+                        Sets.newHashSet(createRealtimeServerMetadata("a1"))
                     )
                 )
-            ).anyTimes();
+            )
+            .anyTimes();
     EasyMock.replay(coordinatorClient);
     CoordinatorBasedSegmentHandoffNotifier notifier = new CoordinatorBasedSegmentHandoffNotifier(
         "test_ds",
@@ -90,14 +89,9 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
     );
     final AtomicBoolean callbackCalled = new AtomicBoolean(false);
     notifier.registerSegmentHandoffCallback(
-        descriptor, MoreExecutors.sameThreadExecutor(), new Runnable()
-        {
-          @Override
-          public void run()
-          {
-            callbackCalled.set(true);
-          }
-        }
+        descriptor,
+        MoreExecutors.sameThreadExecutor(),
+        () -> callbackCalled.set(true)
     );
     notifier.checkForSegmentHandoffs();
     // callback should have registered
@@ -130,15 +124,14 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
     CoordinatorClient coordinatorClient = EasyMock.createMock(CoordinatorClient.class);
     EasyMock.expect(coordinatorClient.fetchServerView("test_ds", interval, true))
             .andReturn(
-                Lists.newArrayList(
+                Collections.singletonList(
                     new ImmutableSegmentLoadInfo(
                         segment,
-                        Sets.newHashSet(
-                            createHistoricalServerMetadata("a1")
-                        )
+                        Sets.newHashSet(createHistoricalServerMetadata("a1"))
                     )
                 )
-            ).anyTimes();
+            )
+            .anyTimes();
     EasyMock.replay(coordinatorClient);
     CoordinatorBasedSegmentHandoffNotifier notifier = new CoordinatorBasedSegmentHandoffNotifier(
         "test_ds",
@@ -147,14 +140,9 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
     );
 
     notifier.registerSegmentHandoffCallback(
-        descriptor, MoreExecutors.sameThreadExecutor(), new Runnable()
-        {
-          @Override
-          public void run()
-          {
-            callbackCalled.set(true);
-          }
-        }
+        descriptor,
+        MoreExecutors.sameThreadExecutor(),
+        () -> callbackCalled.set(true)
     );
     Assert.assertEquals(1, notifier.getHandOffCallbacks().size());
     Assert.assertTrue(notifier.getHandOffCallbacks().containsKey(descriptor));
@@ -173,7 +161,7 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
     );
     Assert.assertFalse(
         CoordinatorBasedSegmentHandoffNotifier.isHandOffComplete(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new ImmutableSegmentLoadInfo(
                     createSegment(interval, "v1", 2),
                     Sets.newHashSet(createHistoricalServerMetadata("a"))
@@ -185,7 +173,7 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
 
     Assert.assertTrue(
         CoordinatorBasedSegmentHandoffNotifier.isHandOffComplete(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new ImmutableSegmentLoadInfo(
                     createSegment(interval, "v2", 2),
                     Sets.newHashSet(createHistoricalServerMetadata("a"))
@@ -197,7 +185,7 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
 
     Assert.assertTrue(
         CoordinatorBasedSegmentHandoffNotifier.isHandOffComplete(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new ImmutableSegmentLoadInfo(
                     createSegment(interval, "v1", 2),
                     Sets.newHashSet(createHistoricalServerMetadata("a"))
@@ -217,7 +205,7 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
     );
     Assert.assertTrue(
         CoordinatorBasedSegmentHandoffNotifier.isHandOffComplete(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new ImmutableSegmentLoadInfo(
                     createSegment(interval, "v1", 2),
                     Sets.newHashSet(createHistoricalServerMetadata("a"))
@@ -229,7 +217,7 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
 
     Assert.assertFalse(
         CoordinatorBasedSegmentHandoffNotifier.isHandOffComplete(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new ImmutableSegmentLoadInfo(
                     createSegment(interval, "v1", 2),
                     Sets.newHashSet(createRealtimeServerMetadata("a"))
@@ -248,7 +236,7 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
     );
     Assert.assertTrue(
         CoordinatorBasedSegmentHandoffNotifier.isHandOffComplete(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new ImmutableSegmentLoadInfo(
                     createSegment(interval, "v1", 1),
                     Sets.newHashSet(createHistoricalServerMetadata("a"))
@@ -260,7 +248,7 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
 
     Assert.assertFalse(
         CoordinatorBasedSegmentHandoffNotifier.isHandOffComplete(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new ImmutableSegmentLoadInfo(
                     createSegment(interval, "v1", 1),
                     Sets.newHashSet(createHistoricalServerMetadata("a"))
@@ -278,41 +266,25 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
 
     Assert.assertFalse(
         CoordinatorBasedSegmentHandoffNotifier.isHandOffComplete(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new ImmutableSegmentLoadInfo(
-                    createSegment(
-                        Intervals.of(
-                            "2011-04-01/2011-04-02"
-                        ), "v1", 1
-                    ),
+                    createSegment(Intervals.of("2011-04-01/2011-04-02"), "v1", 1),
                     Sets.newHashSet(createHistoricalServerMetadata("a"))
                 )
             ),
-            new SegmentDescriptor(
-                Intervals.of(
-                    "2011-04-01/2011-04-03"
-                ), "v1", 1
-            )
+            new SegmentDescriptor(Intervals.of("2011-04-01/2011-04-03"), "v1", 1)
         )
     );
 
     Assert.assertTrue(
         CoordinatorBasedSegmentHandoffNotifier.isHandOffComplete(
-            Lists.newArrayList(
+            Collections.singletonList(
                 new ImmutableSegmentLoadInfo(
-                    createSegment(
-                        Intervals.of(
-                            "2011-04-01/2011-04-04"
-                        ), "v1", 1
-                    ),
+                    createSegment(Intervals.of("2011-04-01/2011-04-04"), "v1", 1),
                     Sets.newHashSet(createHistoricalServerMetadata("a"))
                 )
             ),
-            new SegmentDescriptor(
-                Intervals.of(
-                    "2011-04-02/2011-04-03"
-                ), "v1", 1
-            )
+            new SegmentDescriptor(Intervals.of("2011-04-02/2011-04-03"), "v1", 1)
         )
     );
   }
diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java
index f089e3b..f8951f7 100644
--- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java
+++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java
@@ -500,14 +500,17 @@ public class DruidCoordinatorRuleRunnerTest
     EasyMock.expectLastCall().times(1);
     EasyMock.replay(emitter);
 
-    EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn(
-        Lists.newArrayList(
-            new IntervalLoadRule(
-                Intervals.of("2012-01-02T00:00:00.000Z/2012-01-03T00:00:00.000Z"),
-                ImmutableMap.of("normal", 1)
+    EasyMock
+        .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject()))
+        .andReturn(
+            Collections.singletonList(
+                new IntervalLoadRule(
+                    Intervals.of("2012-01-02T00:00:00.000Z/2012-01-03T00:00:00.000Z"),
+                    ImmutableMap.of("normal", 1)
+                )
             )
         )
-    ).atLeastOnce();
+        .atLeastOnce();
 
     EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).anyTimes();
     EasyMock.replay(databaseRuleManager, mockPeon);
@@ -554,9 +557,7 @@ public class DruidCoordinatorRuleRunnerTest
     EasyMock.expectLastCall().atLeastOnce();
     mockEmptyPeon();
 
-    EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(
-        createCoordinatorDynamicConfig()
-    ).anyTimes();
+    EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes();
     coordinator.removeSegment(EasyMock.anyObject());
     EasyMock.expectLastCall().atLeastOnce();
     EasyMock.replay(coordinator);
@@ -846,19 +847,11 @@ public class DruidCoordinatorRuleRunnerTest
         null,
         ImmutableMap.of(
             "hot",
-            Stream.of(
-                new ServerHolder(
-                    server1.toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))),
+            Stream.of(new ServerHolder(server1.toImmutableDruidServer(), mockPeon))
+                  .collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))),
             "normal",
-            Stream.of(
-                new ServerHolder(
-                    server2.toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            Stream.of(new ServerHolder(server2.toImmutableDruidServer(), mockPeon))
+                  .collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
         )
     );
 
@@ -893,14 +886,17 @@ public class DruidCoordinatorRuleRunnerTest
   public void testDropServerActuallyServesSegment()
   {
     mockCoordinator();
-    EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn(
-        Lists.newArrayList(
-            new IntervalLoadRule(
-                Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T01:00:00.000Z"),
-                ImmutableMap.of("normal", 0)
+    EasyMock
+        .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject()))
+        .andReturn(
+            Collections.singletonList(
+                new IntervalLoadRule(
+                    Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T01:00:00.000Z"),
+                    ImmutableMap.of("normal", 0)
+                )
             )
         )
-    ).atLeastOnce();
+        .atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
     DruidServer server1 = new DruidServer(
@@ -950,18 +946,9 @@ public class DruidCoordinatorRuleRunnerTest
         ImmutableMap.of(
             "normal",
             Stream.of(
-                new ServerHolder(
-                    server1.toImmutableDruidServer(),
-                    mockPeon
-                ),
-                new ServerHolder(
-                    server2.toImmutableDruidServer(),
-                    anotherMockPeon
-                ),
-                new ServerHolder(
-                    server3.toImmutableDruidServer(),
-                    anotherMockPeon
-                )
+                new ServerHolder(server1.toImmutableDruidServer(), mockPeon),
+                new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon),
+                new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon)
             ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
         )
     );
@@ -1007,14 +994,16 @@ public class DruidCoordinatorRuleRunnerTest
     EasyMock.expectLastCall().atLeastOnce();
     mockEmptyPeon();
 
-    EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn(
-        Lists.newArrayList(
-            new IntervalLoadRule(
-                Intervals.of("2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z"),
-                ImmutableMap.of("hot", 2)
+    EasyMock
+        .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject()))
+        .andReturn(
+            Collections.singletonList(new IntervalLoadRule(
+                    Intervals.of("2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z"),
+                    ImmutableMap.of("hot", 2)
+                )
             )
         )
-    ).atLeastOnce();
+        .atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
     DruidCluster druidCluster = new DruidCluster(
@@ -1115,14 +1104,16 @@ public class DruidCoordinatorRuleRunnerTest
   @Test
   public void testReplicantThrottleAcrossTiers()
   {
-    EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(
-        CoordinatorDynamicConfig.builder()
-                                .withReplicationThrottleLimit(7)
-                                .withReplicantLifetime(1)
-                                .withMaxSegmentsInNodeLoadingQueue(1000)
-                                .build()
-
-    ).atLeastOnce();
+    EasyMock
+        .expect(coordinator.getDynamicConfigs())
+        .andReturn(
+            CoordinatorDynamicConfig.builder()
+                                    .withReplicationThrottleLimit(7)
+                                    .withReplicantLifetime(1)
+                                    .withMaxSegmentsInNodeLoadingQueue(1000)
+                                    .build()
+        )
+        .atLeastOnce();
     coordinator.removeSegment(EasyMock.anyObject());
     EasyMock.expectLastCall().anyTimes();
     EasyMock.replay(coordinator);
@@ -1130,17 +1121,19 @@ public class DruidCoordinatorRuleRunnerTest
     EasyMock.expectLastCall().atLeastOnce();
     mockEmptyPeon();
 
-    EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn(
-        Lists.newArrayList(
-            new IntervalLoadRule(
-                Intervals.of("2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z"),
-                ImmutableMap.of(
-                    "hot", 1,
-                    DruidServer.DEFAULT_TIER, 1
+    EasyMock
+        .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject()))
+        .andReturn(
+            Collections.singletonList(new IntervalLoadRule(
+                    Intervals.of("2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z"),
+                    ImmutableMap.of(
+                        "hot", 1,
+                        DruidServer.DEFAULT_TIER, 1
+                    )
                 )
             )
         )
-    ).atLeastOnce();
+        .atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
     DruidCluster druidCluster = new DruidCluster(
@@ -1215,14 +1208,16 @@ public class DruidCoordinatorRuleRunnerTest
     EasyMock.expectLastCall().atLeastOnce();
     mockEmptyPeon();
 
-    EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn(
-        Lists.newArrayList(
-            new IntervalLoadRule(
-                Intervals.of("2012-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z"),
-                ImmutableMap.of("normal", 1)
+    EasyMock
+        .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject()))
+        .andReturn(
+            Collections.singletonList(new IntervalLoadRule(
+                    Intervals.of("2012-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z"),
+                    ImmutableMap.of("normal", 1)
+                )
             )
         )
-    ).atLeastOnce();
+        .atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
     DataSegment overFlowSegment = new DataSegment(
@@ -1342,9 +1337,7 @@ public class DruidCoordinatorRuleRunnerTest
     mockEmptyPeon();
 
     EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn(
-        Lists.newArrayList(
-            new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 1))
-        )).atLeastOnce();
+        Collections.singletonList(new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 1)))).atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
     DruidCluster druidCluster = new DruidCluster(
diff --git a/server/src/test/java/io/druid/server/coordinator/rules/BroadcastDistributionRuleSerdeTest.java b/server/src/test/java/io/druid/server/coordinator/rules/BroadcastDistributionRuleSerdeTest.java
index 6caa686..d0367a1 100644
--- a/server/src/test/java/io/druid/server/coordinator/rules/BroadcastDistributionRuleSerdeTest.java
+++ b/server/src/test/java/io/druid/server/coordinator/rules/BroadcastDistributionRuleSerdeTest.java
@@ -31,6 +31,7 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
@@ -67,7 +68,7 @@ public class BroadcastDistributionRuleSerdeTest
   @Test
   public void testSerde() throws IOException
   {
-    final List<Rule> rules = Lists.newArrayList(testRule);
+    final List<Rule> rules = Collections.singletonList(testRule);
     final String json = MAPPER.writeValueAsString(rules);
     final List<Rule> fromJson = MAPPER.readValue(json, new TypeReference<List<Rule>>(){});
     assertEquals(rules, fromJson);
diff --git a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java
index a66a155..0fcb14b 100644
--- a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java
+++ b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java
@@ -19,7 +19,6 @@
 
 package io.druid.server.http.security;
 
-import com.google.common.collect.Lists;
 import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.jackson.DefaultObjectMapper;
@@ -38,11 +37,12 @@ import javax.servlet.FilterChain;
 import javax.servlet.ServletOutputStream;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+import java.util.Collections;
 import java.util.List;
 
 public class PreResponseAuthorizationCheckFilterTest
 {
-  private static List<Authenticator> authenticators = Lists.newArrayList(new AllowAllAuthenticator());
+  private static List<Authenticator> authenticators = Collections.singletonList(new AllowAllAuthenticator());
 
   @Rule
   public ExpectedException expectedException = ExpectedException.none();
diff --git a/server/src/test/java/io/druid/timeline/partition/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/timeline/partition/HashBasedNumberedShardSpecTest.java
index 7bc6647..80d38f31 100644
--- a/server/src/test/java/io/druid/timeline/partition/HashBasedNumberedShardSpecTest.java
+++ b/server/src/test/java/io/druid/timeline/partition/HashBasedNumberedShardSpecTest.java
@@ -34,6 +34,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 public class HashBasedNumberedShardSpecTest
@@ -155,7 +156,7 @@ public class HashBasedNumberedShardSpecTest
         ImmutableList.of("visitor_id", "cnt"),
         ImmutableMap.of("visitor_id", "v1", "cnt", 10)
     );
-    Assert.assertEquals(ImmutableList.of(Lists.newArrayList("v1")), shardSpec1.getGroupKey(time.getMillis(), inputRow));
+    Assert.assertEquals(ImmutableList.of(Collections.singletonList("v1")), shardSpec1.getGroupKey(time.getMillis(), inputRow));
 
     final HashBasedNumberedShardSpec shardSpec2 = new HashBasedNumberedShardSpec(
         1,
@@ -167,9 +168,9 @@ public class HashBasedNumberedShardSpecTest
         time.getMillis(),
         ImmutableMap.of(
             "cnt",
-            Lists.newArrayList(10),
+            Collections.singletonList(10),
             "visitor_id",
-            Lists.newArrayList("v1")
+            Collections.singletonList("v1")
         )
     ).toString(), shardSpec2.getGroupKey(time.getMillis(), inputRow).toString());
   }
diff --git a/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java b/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java
index 981e694..9021ddb 100644
--- a/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java
+++ b/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java
@@ -20,7 +20,6 @@
 package io.druid.cli;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.Lists;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
 import com.google.inject.Key;
@@ -42,6 +41,7 @@ import org.eclipse.jetty.servlet.DefaultServlet;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
 
+import java.util.Collections;
 import java.util.List;
 
 /**
@@ -60,9 +60,7 @@ class MiddleManagerJettyServerInitializer implements JettyServerInitializer
     this.authConfig = authConfig;
   }
 
-  private static List<String> UNSECURED_PATHS = Lists.newArrayList(
-      "/status/health"
-  );
+  private static List<String> UNSECURED_PATHS = Collections.singletonList("/status/health");
 
   @Override
   public void initialize(Server server, Injector injector)
diff --git a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java
index 26d0b27..2b7fc65 100644
--- a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java
+++ b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java
@@ -22,7 +22,6 @@ package io.druid.cli;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
 import com.google.inject.Key;
@@ -46,6 +45,7 @@ import org.eclipse.jetty.servlet.FilterHolder;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
@@ -54,9 +54,7 @@ import java.util.Set;
 public class QueryJettyServerInitializer implements JettyServerInitializer
 {
   private static final Logger log = new Logger(QueryJettyServerInitializer.class);
-  private static List<String> UNSECURED_PATHS = Lists.newArrayList(
-      "/status/health"
-  );
+  private static List<String> UNSECURED_PATHS = Collections.singletonList("/status/health");
 
   private final List<Handler> extensionHandlers;
 
diff --git a/sql/src/main/java/io/druid/sql/calcite/schema/InformationSchema.java b/sql/src/main/java/io/druid/sql/calcite/schema/InformationSchema.java
index b73e76b..de39a22 100644
--- a/sql/src/main/java/io/druid/sql/calcite/schema/InformationSchema.java
+++ b/sql/src/main/java/io/druid/sql/calcite/schema/InformationSchema.java
@@ -27,7 +27,6 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import com.google.inject.Inject;
 import io.druid.segment.column.ValueType;
 import io.druid.server.security.AuthenticationResult;
@@ -56,6 +55,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 
 import javax.annotation.Nullable;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
 
@@ -106,7 +106,7 @@ public class InformationSchema extends AbstractSchema
       .build();
   private static final RelDataTypeSystem TYPE_SYSTEM = RelDataTypeSystem.DEFAULT;
   private static final Function<String, Iterable<ResourceAction>> DRUID_TABLE_RA_GENERATOR = datasourceName -> {
-    return Lists.newArrayList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName));
+    return Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName));
   };
 
   private final SchemaPlus rootSchema;
diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java
index 462af22..0fd471e 100644
--- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java
@@ -2423,12 +2423,8 @@ public class CalciteQueryTest extends CalciteTestBase
                         .setInterval(QSS(Filtration.eternity()))
                         .setGranularity(Granularities.ALL)
                         .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
-                        .setAggregatorSpecs(
-                            ImmutableList.of(
-                                new FloatMinAggregatorFactory("a0", "m1"),
-                                new FloatMaxAggregatorFactory("a1", "m1")
-                            )
-                        )
+                        .setAggregatorSpecs(new FloatMinAggregatorFactory("a0", "m1"),
+                                            new FloatMaxAggregatorFactory("a1", "m1"))
                         .setPostAggregatorSpecs(ImmutableList.of(EXPRESSION_POST_AGG("p0", "(\"a0\" + \"a1\")")))
                         .setLimitSpec(
                             new DefaultLimitSpec(
@@ -2469,12 +2465,8 @@ public class CalciteQueryTest extends CalciteTestBase
                         .setInterval(QSS(Filtration.eternity()))
                         .setGranularity(Granularities.ALL)
                         .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
-                        .setAggregatorSpecs(
-                            ImmutableList.of(
-                                new FloatMinAggregatorFactory("a0", "m1"),
-                                new FloatMaxAggregatorFactory("a1", "m1")
-                            )
-                        )
+                        .setAggregatorSpecs(new FloatMinAggregatorFactory("a0", "m1"),
+                                            new FloatMaxAggregatorFactory("a1", "m1"))
                         .setPostAggregatorSpecs(
                             ImmutableList.of(
                                 EXPRESSION_POST_AGG("p0", "(\"a0\" + \"a1\")")


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