You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by fj...@apache.org on 2019/07/31 23:15:21 UTC

[incubator-druid] branch master updated: GroupBy array-based result rows. (#8196)

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

fjy 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 77297f4  GroupBy array-based result rows. (#8196)
77297f4 is described below

commit 77297f4e6f2e9d617c96cd46852bb5a772961e85
Author: Gian Merlino <gi...@gmail.com>
AuthorDate: Wed Jul 31 16:15:12 2019 -0700

    GroupBy array-based result rows. (#8196)
    
    * GroupBy array-based result rows.
    
    Fixes #8118; see that proposal for details.
    
    Other than the GroupBy changes, the main other "interesting" classes are:
    
    - ResultRow: The array-based result type.
    - BaseQuery: T is no longer required to be Comparable.
    - QueryToolChest: Adds "decorateObjectMapper" to enable query-aware serialization
      and deserialization of result rows (necessary due to their positional nature).
    - QueryResource: Uses the new decoration functionality.
    - DirectDruidClient: Also uses the new decoration functionality.
    - QueryMaker (in Druid SQL): Modifications to read ResultRows.
    
    These classes weren't changed, but got some new javadocs:
    
    - BySegmentQueryRunner
    - FinalizeResultsQueryRunner
    - Query
    
    * Adjustments for TC stuff.
---
 .../benchmark/GroupByTypeInterfaceBenchmark.java   |  116 +-
 .../druid/benchmark/query/GroupByBenchmark.java    |   88 +-
 .../benchmark/query/SqlVsNativeBenchmark.java      |    6 +-
 .../DistinctCountGroupByQueryTest.java             |    8 +-
 .../aggregator/MomentsSketchAggregatorTest.java    |   24 +-
 .../query/movingaverage/MovingAverageQuery.java    |   74 +-
 .../movingaverage/MovingAverageQueryRunner.java    |   10 +-
 .../movingaverage/MovingAverageQueryTest.java      |   17 +-
 .../queryTests/basicGroupByMovingAverage.yaml      |   17 +-
 .../queryTests/basicGroupByMovingAverage2.yaml     |   14 +-
 .../resources/queryTests/missingGroupByValues.yaml |   24 +-
 .../resources/queryTests/sortingAveragersAsc.yaml  |   29 +-
 .../resources/queryTests/sortingAveragersDesc.yaml |   29 +-
 .../sortingWithNonMovingAndMovingAvgMetric.yaml    |   24 +-
 .../queryTests/sortingWithNonMovingAvgMetric.yaml  |   24 +-
 .../tdigestsketch/TDigestSketchAggregatorTest.java |   26 +-
 .../TimestampGroupByAggregationTest.java           |   13 +-
 .../druid/segment/MapVirtualColumnGroupByTest.java |   11 +-
 .../datasketches/hll/HllSketchAggregatorTest.java  |   50 +-
 .../quantiles/DoublesSketchAggregatorTest.java     |   60 +-
 .../datasketches/theta/SketchAggregationTest.java  |   81 +-
 .../theta/SketchAggregationWithSimpleDataTest.java |   30 +-
 .../theta/oldapi/OldApiSketchAggregationTest.java  |   70 +-
 .../tuple/ArrayOfDoublesSketchAggregationTest.java |  108 +-
 .../bloom/BloomFilterGroupByQueryTest.java         |    6 +-
 .../ApproximateHistogramAggregationTest.java       |    7 +-
 .../ApproximateHistogramGroupByQueryTest.java      |   13 +-
 .../FixedBucketsHistogramAggregationTest.java      |    7 +-
 .../FixedBucketsHistogramGroupByQueryTest.java     |   13 +-
 .../variance/VarianceGroupByQueryTest.java         |   29 +-
 .../aggregation/variance/VarianceTestHelper.java   |   11 +-
 .../indexing/common/task/RealtimeIndexTask.java    |   16 +-
 .../java/org/apache/druid/query/BaseQuery.java     |   11 +-
 .../apache/druid/query/BySegmentQueryRunner.java   |    8 +
 .../java/org/apache/druid/query/CacheStrategy.java |   25 +-
 .../druid/query/FinalizeResultsQueryRunner.java    |   26 +-
 .../druid/query/GroupByMergedQueryRunner.java      |    4 +-
 .../main/java/org/apache/druid/query/Query.java    |   13 +
 .../org/apache/druid/query/QueryToolChest.java     |   14 +
 .../apache/druid/query/groupby/GroupByQuery.java   |  360 +-
 .../druid/query/groupby/GroupByQueryConfig.java    |    1 +
 .../druid/query/groupby/GroupByQueryHelper.java    |   94 +-
 .../query/groupby/GroupByQueryQueryToolChest.java  |  383 +-
 .../query/groupby/GroupByQueryRunnerFactory.java   |   30 +-
 .../org/apache/druid/query/groupby/ResultRow.java  |  218 ++
 .../groupby/RowBasedColumnSelectorFactory.java     |  109 +-
 .../groupby/epinephelinae/GroupByBinaryFnV2.java   |   63 +-
 .../epinephelinae/GroupByMergingQueryRunnerV2.java |   82 +-
 .../epinephelinae/GroupByQueryEngineV2.java        |  103 +-
 .../groupby/epinephelinae/GroupByRowProcessor.java |  165 +-
 .../epinephelinae/RowBasedGrouperHelper.java       |  322 +-
 ...uildingStringGroupByColumnSelectorStrategy.java |   11 +-
 .../DoubleGroupByColumnSelectorStrategy.java       |    6 +-
 .../column/FloatGroupByColumnSelectorStrategy.java |    6 +-
 .../column/GroupByColumnSelectorPlus.java          |   19 +-
 .../column/GroupByColumnSelectorStrategy.java      |   40 +-
 .../column/LongGroupByColumnSelectorStrategy.java  |    6 +-
 ...NullableValueGroupByColumnSelectorStrategy.java |    8 +-
 .../StringGroupByColumnSelectorStrategy.java       |   25 +-
 .../vector/DoubleGroupByVectorColumnSelector.java  |    8 +-
 .../vector/FloatGroupByVectorColumnSelector.java   |    8 +-
 .../vector/GroupByVectorColumnSelector.java        |    7 +-
 .../vector/LongGroupByVectorColumnSelector.java    |    8 +-
 ...ngleValueStringGroupByVectorColumnSelector.java |    8 +-
 .../epinephelinae/vector/VectorGroupByEngine.java  |   50 +-
 .../query/groupby/having/AlwaysHavingSpec.java     |   13 +-
 .../druid/query/groupby/having/AndHavingSpec.java  |   22 +-
 .../query/groupby/having/DimFilterHavingSpec.java  |  180 +-
 .../having/DimensionSelectorHavingSpec.java        |   22 +-
 .../query/groupby/having/EqualToHavingSpec.java    |   19 +-
 .../groupby/having/GreaterThanHavingSpec.java      |   19 +-
 .../druid/query/groupby/having/HavingSpec.java     |   23 +-
 .../druid/query/groupby/having/HavingSpecUtil.java |   13 +
 .../query/groupby/having/LessThanHavingSpec.java   |   19 +-
 .../query/groupby/having/NeverHavingSpec.java      |   13 +-
 .../druid/query/groupby/having/NotHavingSpec.java  |   21 +-
 .../druid/query/groupby/having/OrHavingSpec.java   |   22 +-
 .../query/groupby/orderby/DefaultLimitSpec.java    |  140 +-
 .../druid/query/groupby/orderby/LimitSpec.java     |   23 +-
 .../druid/query/groupby/orderby/NoopLimitSpec.java |   18 +-
 .../query/groupby/strategy/GroupByStrategy.java    |  133 +-
 .../query/groupby/strategy/GroupByStrategyV1.java  |   74 +-
 .../query/groupby/strategy/GroupByStrategyV2.java  |  382 +-
 .../timeseries/TimeseriesQueryQueryToolChest.java  |    7 +-
 .../druid/query/topn/TopNQueryQueryToolChest.java  |    7 +-
 .../segment/incremental/IncrementalIndex.java      |    2 +-
 .../druid/segment/transform/TransformSpec.java     |   11 -
 .../druid/segment/transform/Transformer.java       |    2 +-
 .../druid/query/MultiValuedDimensionTest.java      |  224 +-
 .../query/aggregation/AggregationTestHelper.java   |   40 +-
 .../hyperloglog/HyperUniquesAggregationTest.java   |   18 +-
 .../FinalizingFieldAccessPostAggregatorTest.java   |   12 +-
 ...GroupByLimitPushDownInsufficientBufferTest.java |   71 +-
 .../GroupByLimitPushDownMultiNodeMergeTest.java    |   73 +-
 .../query/groupby/GroupByMultiSegmentTest.java     |   26 +-
 .../query/groupby/GroupByQueryMergeBufferTest.java |    7 +-
 .../groupby/GroupByQueryQueryToolChestTest.java    |  188 +-
 .../groupby/GroupByQueryRunnerFactoryTest.java     |   12 +-
 .../groupby/GroupByQueryRunnerFailureTest.java     |    7 +-
 .../query/groupby/GroupByQueryRunnerTest.java      | 3920 ++++++++++----------
 .../groupby/GroupByQueryRunnerTestHelper.java      |   51 +-
 .../druid/query/groupby/GroupByQueryTest.java      |    9 +-
 .../groupby/GroupByTimeseriesQueryRunnerTest.java  |   14 +-
 .../query/groupby/NestedQueryPushDownTest.java     |   87 +-
 .../apache/druid/query/groupby/ResultRowTest.java} |   27 +-
 .../groupby/having/DimFilterHavingSpecTest.java    |   47 +-
 .../having/DimensionSelectorHavingSpecTest.java    |    7 +-
 .../druid/query/groupby/having/HavingSpecTest.java |   29 +-
 .../groupby/orderby/DefaultLimitSpecTest.java      |  197 +-
 .../java/org/apache/druid/segment/TestHelper.java  |   70 +-
 .../druid/segment/filter/BaseFilterTest.java       |    2 +-
 .../virtual/DummyStringVirtualColumnTest.java      |   32 +-
 .../virtual/ExpressionVirtualColumnTest.java       |    8 +-
 .../org/apache/druid/client/DirectDruidClient.java |   10 +-
 .../appenderator/SinkQuerySegmentWalker.java       |    8 +-
 .../org/apache/druid/server/QueryLifecycle.java    |    9 +
 .../org/apache/druid/server/QueryResource.java     |   66 +-
 .../druid/client/CachingClusteredClientTest.java   |   81 +-
 .../apache/druid/sql/calcite/rel/QueryMaker.java   |   33 +-
 .../druid/sql/calcite/util/CalciteTests.java       |   73 +-
 120 files changed, 5503 insertions(+), 4436 deletions(-)

diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
index e4cf5ac..ccd219d 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
@@ -34,7 +34,6 @@ import org.apache.druid.collections.DefaultBlockingPool;
 import org.apache.druid.collections.NonBlockingPool;
 import org.apache.druid.collections.StupidPool;
 import org.apache.druid.data.input.InputRow;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.granularity.Granularity;
@@ -58,6 +57,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryEngine;
 import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
 import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
 import org.apache.druid.query.groupby.strategy.GroupByStrategyV1;
 import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
@@ -108,7 +108,7 @@ import java.util.concurrent.TimeUnit;
 public class GroupByTypeInterfaceBenchmark
 {
   private static final SegmentId Q_INDEX_SEGMENT_ID = SegmentId.dummy("qIndex");
-  
+
   @Param({"4"})
   private int numSegments;
 
@@ -137,7 +137,7 @@ public class GroupByTypeInterfaceBenchmark
   private IncrementalIndex anIncrementalIndex;
   private List<QueryableIndex> queryableIndexes;
 
-  private QueryRunnerFactory<Row, GroupByQuery> factory;
+  private QueryRunnerFactory<ResultRow, GroupByQuery> factory;
 
   private BenchmarkSchemaInfo schemaInfo;
   private GroupByQuery stringQuery;
@@ -190,8 +190,10 @@ public class GroupByTypeInterfaceBenchmark
           .builder()
           .setDataSource("blah")
           .setQuerySegmentSpec(intervalSpec)
-          .setDimensions(new DefaultDimensionSpec("metLongUniform", null),
-                         new DefaultDimensionSpec("metFloatNormal", null))
+          .setDimensions(
+              new DefaultDimensionSpec("metLongUniform", null),
+              new DefaultDimensionSpec("metFloatNormal", null)
+          )
           .setAggregatorSpecs(
               queryAggs
           )
@@ -459,15 +461,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexStringOnly(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -477,15 +479,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexLongOnly(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -495,15 +497,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexFloatOnly(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -513,15 +515,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexNumericOnly(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -531,15 +533,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexNumericThenString(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
 
@@ -551,7 +553,7 @@ public class GroupByTypeInterfaceBenchmark
 
     results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -562,15 +564,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexLongThenString(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
 
@@ -582,7 +584,7 @@ public class GroupByTypeInterfaceBenchmark
 
     results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -592,15 +594,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexLongThenFloat(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
 
@@ -612,7 +614,7 @@ public class GroupByTypeInterfaceBenchmark
 
     results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -622,15 +624,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexStringThenNumeric(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
 
@@ -642,7 +644,7 @@ public class GroupByTypeInterfaceBenchmark
 
     results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -652,15 +654,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexStringThenLong(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
 
@@ -672,7 +674,7 @@ public class GroupByTypeInterfaceBenchmark
 
     results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -682,15 +684,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexStringTwice(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
 
@@ -702,7 +704,7 @@ public class GroupByTypeInterfaceBenchmark
 
     results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -712,15 +714,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexLongTwice(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
 
@@ -732,7 +734,7 @@ public class GroupByTypeInterfaceBenchmark
 
     results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -743,15 +745,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexFloatTwice(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
 
@@ -763,7 +765,7 @@ public class GroupByTypeInterfaceBenchmark
 
     results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -773,15 +775,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexFloatThenLong(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
 
@@ -793,7 +795,7 @@ public class GroupByTypeInterfaceBenchmark
 
     results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
@@ -803,15 +805,15 @@ public class GroupByTypeInterfaceBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndexFloatThenString(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         Q_INDEX_SEGMENT_ID,
         new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
     );
 
-    List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
+    List<ResultRow> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
 
@@ -823,7 +825,7 @@ public class GroupByTypeInterfaceBenchmark
 
     results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
 
-    for (Row result : results) {
+    for (ResultRow result : results) {
       blackhole.consume(result);
     }
   }
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
index 9076ff9..fe1e3e2 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
@@ -35,7 +35,6 @@ import org.apache.druid.collections.DefaultBlockingPool;
 import org.apache.druid.collections.NonBlockingPool;
 import org.apache.druid.collections.StupidPool;
 import org.apache.druid.data.input.InputRow;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.common.concurrent.Execs;
 import org.apache.druid.java.util.common.granularity.Granularities;
@@ -64,9 +63,13 @@ import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryEngine;
 import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
 import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
+import org.apache.druid.query.groupby.ResultRow;
+import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
 import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
 import org.apache.druid.query.groupby.strategy.GroupByStrategyV1;
 import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
+import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
 import org.apache.druid.query.spec.QuerySegmentSpec;
 import org.apache.druid.segment.IncrementalIndexSegment;
@@ -109,9 +112,9 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
 @State(Scope.Benchmark)
-@Fork(value = 1)
-@Warmup(iterations = 15)
-@Measurement(iterations = 30)
+@Fork(value = 2)
+@Warmup(iterations = 10)
+@Measurement(iterations = 25)
 public class GroupByBenchmark
 {
   @Param({"4"})
@@ -148,7 +151,7 @@ public class GroupByBenchmark
   private IncrementalIndex anIncrementalIndex;
   private List<QueryableIndex> queryableIndexes;
 
-  private QueryRunnerFactory<Row, GroupByQuery> factory;
+  private QueryRunnerFactory<ResultRow, GroupByQuery> factory;
 
   private BenchmarkSchemaInfo schemaInfo;
   private GroupByQuery query;
@@ -197,6 +200,34 @@ public class GroupByBenchmark
       basicQueries.put("A", queryA);
     }
 
+    { // basic.sorted
+      QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
+      List<AggregatorFactory> queryAggs = new ArrayList<>();
+      queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
+      GroupByQuery queryA = GroupByQuery
+          .builder()
+          .setDataSource("blah")
+          .setQuerySegmentSpec(intervalSpec)
+          .setDimensions(new DefaultDimensionSpec("dimSequential", null), new DefaultDimensionSpec("dimZipf", null))
+          .setAggregatorSpecs(queryAggs)
+          .setGranularity(Granularity.fromString(queryGranularity))
+          .setLimitSpec(
+              new DefaultLimitSpec(
+                  Collections.singletonList(
+                      new OrderByColumnSpec(
+                          "sumLongSequential",
+                          OrderByColumnSpec.Direction.DESCENDING,
+                          StringComparators.NUMERIC
+                      )
+                  ),
+                  100
+              )
+          )
+          .build();
+
+      basicQueries.put("sorted", queryA);
+    }
+
     { // basic.nested
       QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
       List<AggregatorFactory> queryAggs = new ArrayList<>();
@@ -562,14 +593,14 @@ public class GroupByBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleIncrementalIndex(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         SegmentId.dummy("incIndex"),
         new IncrementalIndexSegment(anIncrementalIndex, SegmentId.dummy("incIndex"))
     );
 
-    final Sequence<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
-    final Row lastRow = results.accumulate(
+    final Sequence<ResultRow> results = GroupByBenchmark.runQuery(factory, runner, query);
+    final ResultRow lastRow = results.accumulate(
         null,
         (accumulated, in) -> in
     );
@@ -582,14 +613,14 @@ public class GroupByBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void querySingleQueryableIndex(Blackhole blackhole)
   {
-    QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+    QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
         factory,
         SegmentId.dummy("qIndex"),
         new QueryableIndexSegment(queryableIndexes.get(0), SegmentId.dummy("qIndex"))
     );
 
-    final Sequence<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
-    final Row lastRow = results.accumulate(
+    final Sequence<ResultRow> results = GroupByBenchmark.runQuery(factory, runner, query);
+    final ResultRow lastRow = results.accumulate(
         null,
         (accumulated, in) -> in
     );
@@ -600,18 +631,18 @@ public class GroupByBenchmark
   @Benchmark
   @BenchmarkMode(Mode.AverageTime)
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
-  public void queryMultiQueryableIndex(Blackhole blackhole)
+  public void queryMultiQueryableIndexX(Blackhole blackhole)
   {
-    QueryToolChest<Row, GroupByQuery> toolChest = factory.getToolchest();
-    QueryRunner<Row> theRunner = new FinalizeResultsQueryRunner<>(
+    QueryToolChest<ResultRow, GroupByQuery> toolChest = factory.getToolchest();
+    QueryRunner<ResultRow> theRunner = new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             factory.mergeRunners(executorService, makeMultiRunners())
         ),
         (QueryToolChest) toolChest
     );
 
-    Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
-    List<Row> results = queryResult.toList();
+    Sequence<ResultRow> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
+    List<ResultRow> results = queryResult.toList();
     blackhole.consume(results);
   }
 
@@ -620,8 +651,8 @@ public class GroupByBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void queryMultiQueryableIndexWithSpilling(Blackhole blackhole)
   {
-    QueryToolChest<Row, GroupByQuery> toolChest = factory.getToolchest();
-    QueryRunner<Row> theRunner = new FinalizeResultsQueryRunner<>(
+    QueryToolChest<ResultRow, GroupByQuery> toolChest = factory.getToolchest();
+    QueryRunner<ResultRow> theRunner = new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             factory.mergeRunners(executorService, makeMultiRunners())
         ),
@@ -631,8 +662,8 @@ public class GroupByBenchmark
     final GroupByQuery spillingQuery = query.withOverriddenContext(
         ImmutableMap.of("bufferGrouperMaxSize", 4000)
     );
-    Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(spillingQuery), ResponseContext.createEmpty());
-    List<Row> results = queryResult.toList();
+    Sequence<ResultRow> queryResult = theRunner.run(QueryPlus.wrap(spillingQuery), ResponseContext.createEmpty());
+    List<ResultRow> results = queryResult.toList();
     blackhole.consume(results);
   }
 
@@ -641,12 +672,13 @@ public class GroupByBenchmark
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
   public void queryMultiQueryableIndexWithSerde(Blackhole blackhole)
   {
-    QueryToolChest<Row, GroupByQuery> toolChest = factory.getToolchest();
-    QueryRunner<Row> theRunner = new FinalizeResultsQueryRunner<>(
+    QueryToolChest<ResultRow, GroupByQuery> toolChest = factory.getToolchest();
+    //noinspection unchecked
+    QueryRunner<ResultRow> theRunner = new FinalizeResultsQueryRunner<>(
         toolChest.mergeResults(
             new SerializingQueryRunner<>(
                 new DefaultObjectMapper(new SmileFactory()),
-                Row.class,
+                ResultRow.class,
                 toolChest.mergeResults(
                     factory.mergeRunners(executorService, makeMultiRunners())
                 )
@@ -655,17 +687,17 @@ public class GroupByBenchmark
         (QueryToolChest) toolChest
     );
 
-    Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
-    List<Row> results = queryResult.toList();
+    Sequence<ResultRow> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
+    List<ResultRow> results = queryResult.toList();
     blackhole.consume(results);
   }
 
-  private List<QueryRunner<Row>> makeMultiRunners()
+  private List<QueryRunner<ResultRow>> makeMultiRunners()
   {
-    List<QueryRunner<Row>> runners = new ArrayList<>();
+    List<QueryRunner<ResultRow>> runners = new ArrayList<>();
     for (int i = 0; i < numSegments; i++) {
       String segmentName = "qIndex" + i;
-      QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
+      QueryRunner<ResultRow> runner = QueryBenchmarkUtil.makeQueryRunner(
           factory,
           SegmentId.dummy(segmentName),
           new QueryableIndexSegment(queryableIndexes.get(i), SegmentId.dummy(segmentName))
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java
index 47ae6ca..8021a99 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java
@@ -22,7 +22,6 @@ package org.apache.druid.benchmark.query;
 import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo;
 import org.apache.druid.benchmark.datagen.BenchmarkSchemas;
 import org.apache.druid.benchmark.datagen.SegmentGenerator;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.granularity.Granularities;
@@ -35,6 +34,7 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.query.context.ResponseContext;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.server.security.AuthTestUtils;
 import org.apache.druid.server.security.AuthenticationResult;
@@ -151,8 +151,8 @@ public class SqlVsNativeBenchmark
   @OutputTimeUnit(TimeUnit.MILLISECONDS)
   public void queryNative(Blackhole blackhole)
   {
-    final Sequence<Row> resultSequence = QueryPlus.wrap(groupByQuery).run(walker, ResponseContext.createEmpty());
-    final Row lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);
+    final Sequence<ResultRow> resultSequence = QueryPlus.wrap(groupByQuery).run(walker, ResponseContext.createEmpty());
+    final ResultRow lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);
     blackhole.consume(lastRow);
   }
 
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
index dc4f1da..96e7148 100644
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
+++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
@@ -22,7 +22,6 @@ package org.apache.druid.query.aggregation.distinctcount;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import org.apache.druid.data.input.MapBasedInputRow;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.io.Closer;
@@ -34,6 +33,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTestHelper;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
 import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
 import org.apache.druid.segment.IncrementalIndexSegment;
@@ -130,20 +130,22 @@ public class DistinctCountGroupByQueryTest
         .build();
     final Segment incrementalIndexSegment = new IncrementalIndexSegment(index, null);
 
-    Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(
+    Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(
         factory,
         factory.createRunner(incrementalIndexSegment),
         query
     );
 
-    List<Row> expectedResults = Arrays.asList(
+    List<ResultRow> expectedResults = Arrays.asList(
         GroupByQueryRunnerTestHelper.createExpectedRow(
+            query,
             "1970-01-01T00:00:00.000Z",
             client_type, "iphone",
             "UV", 2L,
             "rows", 2L
         ),
         GroupByQueryRunnerTestHelper.createExpectedRow(
+            query,
             "1970-01-01T00:00:00.000Z",
             client_type, "android",
             "UV", 1L,
diff --git a/extensions-contrib/momentsketch/src/test/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentsSketchAggregatorTest.java b/extensions-contrib/momentsketch/src/test/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentsSketchAggregatorTest.java
index 922075f..0299a0d 100644
--- a/extensions-contrib/momentsketch/src/test/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentsSketchAggregatorTest.java
+++ b/extensions-contrib/momentsketch/src/test/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentsSketchAggregatorTest.java
@@ -21,7 +21,6 @@ package org.apache.druid.query.aggregation.momentsketch.aggregator;
 
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.initialization.DruidModule;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.common.granularity.Granularities;
@@ -31,6 +30,7 @@ import org.apache.druid.query.aggregation.momentsketch.MomentSketchModule;
 import org.apache.druid.query.aggregation.momentsketch.MomentSketchWrapper;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -90,7 +90,7 @@ public class MomentsSketchAggregatorTest
   @Test
   public void buildingSketchesAtIngestionTime() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("doubles_build_data.tsv").getFile()),
         String.join(
             "\n",
@@ -133,28 +133,28 @@ public class MomentsSketchAggregatorTest
             "}"
         )
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    double[] quantilesArray = (double[]) row.getRaw("quantiles");
+    ResultRow row = results.get(0);
+    double[] quantilesArray = (double[]) row.get(1); // "quantiles"
     Assert.assertEquals(0, quantilesArray[0], 0.05);
     Assert.assertEquals(.5, quantilesArray[1], 0.05);
     Assert.assertEquals(1.0, quantilesArray[2], 0.05);
 
-    Double minValue = (Double) row.getRaw("min");
+    Double minValue = (Double) row.get(2); // "min"
     Assert.assertEquals(0.0011, minValue, 0.0001);
 
-    Double maxValue = (Double) row.getRaw("max");
+    Double maxValue = (Double) row.get(3); // "max"
     Assert.assertEquals(0.9969, maxValue, 0.0001);
 
-    MomentSketchWrapper sketchObject = (MomentSketchWrapper) row.getRaw("sketch");
+    MomentSketchWrapper sketchObject = (MomentSketchWrapper) row.get(0); // "sketch"
     Assert.assertEquals(400.0, sketchObject.getPowerSums()[0], 1e-10);
   }
 
   @Test
   public void buildingSketchesAtQueryTime() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("doubles_build_data.tsv").getFile()),
         String.join(
             "\n",
@@ -191,11 +191,11 @@ public class MomentsSketchAggregatorTest
         )
     );
 
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
+    ResultRow row = results.get(0);
 
-    MomentSketchWrapper sketchObject = (MomentSketchWrapper) row.getRaw("sketch");
+    MomentSketchWrapper sketchObject = (MomentSketchWrapper) row.get(0); // "sketch"
     // 9 total products since we pre-sum the values.
     Assert.assertEquals(9.0, sketchObject.getPowerSums()[0], 1e-10);
   }
diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java
index 38fc1eb..46689d0 100644
--- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java
+++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java
@@ -27,8 +27,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.google.common.base.Function;
 import com.google.common.base.Functions;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.IAE;
@@ -42,6 +43,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.PostAggregator;
 import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.having.HavingSpec;
 import org.apache.druid.query.groupby.orderby.LimitSpec;
 import org.apache.druid.query.groupby.orderby.NoopLimitSpec;
@@ -67,7 +70,6 @@ public class MovingAverageQuery extends BaseQuery<Row>
   private final LimitSpec limitSpec;
   private final HavingSpec havingSpec;
   private final DimFilter dimFilter;
-  private final Function<Sequence<Row>, Sequence<Row>> limitFn;
   private final Granularity granularity;
   private final List<DimensionSpec> dimensions;
   private final List<AggregatorFactory> aggregatorSpecs;
@@ -75,6 +77,16 @@ public class MovingAverageQuery extends BaseQuery<Row>
   private final List<AveragerFactory<?, ?>> averagerSpecs;
   private final List<PostAggregator> postAveragerSpecs;
 
+  /**
+   * This GroupByQuery is used by {@link #applyLimit(Sequence)} to convert between Rows and ResultRows.
+   */
+  private final GroupByQuery groupByQueryForLimitSpec;
+
+  /**
+   * This Function is used by {@link #applyLimit(Sequence)} to apply having and limit specs.
+   */
+  private final Function<Sequence<ResultRow>, Sequence<ResultRow>> limitFn;
+
   @JsonCreator
   public MovingAverageQuery(
       @JsonProperty("dataSource") DataSource dataSource,
@@ -94,7 +106,10 @@ public class MovingAverageQuery extends BaseQuery<Row>
     super(dataSource, querySegmentSpec, false, context);
 
     //TBD: Implement null awareness to respect the contract of this flag.
-    Preconditions.checkArgument(NullHandling.replaceWithDefault(), "movingAverage does not support druid.generic.useDefaultValueForNull=false");
+    Preconditions.checkArgument(
+        NullHandling.replaceWithDefault(),
+        "movingAverage does not support druid.generic.useDefaultValueForNull=false"
+    );
 
     this.dimFilter = dimFilter;
     this.granularity = granularity;
@@ -120,41 +135,29 @@ public class MovingAverageQuery extends BaseQuery<Row>
       combinedAggregatorSpecs.add(new AveragerFactoryWrapper(avg, ""));
     }
 
-    Function<Sequence<Row>, Sequence<Row>> postProcFn =
-        this.limitSpec.build(
-            this.dimensions,
-            combinedAggregatorSpecs,
-            this.postAggregatorSpecs,
-            this.granularity,
-            getContextSortByDimsFirst()
-        );
+    this.groupByQueryForLimitSpec = GroupByQuery
+        .builder()
+        .setDataSource(dataSource)
+        .setInterval(getQuerySegmentSpec())
+        .setDimensions(this.dimensions)
+        .setAggregatorSpecs(combinedAggregatorSpecs)
+        .setPostAggregatorSpecs(
+            ImmutableList.copyOf(Iterables.concat(this.postAggregatorSpecs, this.postAveragerSpecs))
+        )
+        .setGranularity(this.granularity)
+        .overrideContext(ImmutableMap.of(GroupByQuery.CTX_KEY_SORT_BY_DIMS_FIRST, true))
+        .build();
+
+    Function<Sequence<ResultRow>, Sequence<ResultRow>> postProcFn = this.limitSpec.build(groupByQueryForLimitSpec);
 
     if (havingSpec != null) {
       postProcFn = Functions.compose(
           postProcFn,
-          new Function<Sequence<Row>, Sequence<Row>>()
-          {
-            @Override
-            public Sequence<Row> apply(Sequence<Row> input)
-            {
-              return Sequences.filter(
-                  input,
-                  new Predicate<Row>()
-                  {
-                    @Override
-                    public boolean apply(Row input)
-                    {
-                      return MovingAverageQuery.this.havingSpec.eval(input);
-                    }
-                  }
-              );
-            }
-          }
+          sequence -> Sequences.filter(sequence, MovingAverageQuery.this.havingSpec::eval)
       );
     }
 
     this.limitFn = postProcFn;
-
   }
 
   private static void verifyOutputNames(
@@ -200,7 +203,8 @@ public class MovingAverageQuery extends BaseQuery<Row>
       List<PostAggregator> postAveragerSpecs,
       HavingSpec havingSpec,
       LimitSpec orderBySpec,
-      Function<Sequence<Row>, Sequence<Row>> limitFn,
+      GroupByQuery groupByQueryForLimitSpec,
+      Function<Sequence<ResultRow>, Sequence<ResultRow>> limitFn,
       Map<String, Object> context
   )
   {
@@ -215,6 +219,7 @@ public class MovingAverageQuery extends BaseQuery<Row>
     this.postAveragerSpecs = postAveragerSpecs;
     this.havingSpec = havingSpec;
     this.limitSpec = orderBySpec;
+    this.groupByQueryForLimitSpec = groupByQueryForLimitSpec;
     this.limitFn = limitFn;
   }
 
@@ -307,6 +312,7 @@ public class MovingAverageQuery extends BaseQuery<Row>
         postAveragerSpecs,
         havingSpec,
         limitSpec,
+        groupByQueryForLimitSpec,
         limitFn,
         computeOverridenContext(contextOverride)
     );
@@ -327,6 +333,7 @@ public class MovingAverageQuery extends BaseQuery<Row>
         postAveragerSpecs,
         havingSpec,
         limitSpec,
+        groupByQueryForLimitSpec,
         limitFn,
         getContext()
     );
@@ -347,6 +354,7 @@ public class MovingAverageQuery extends BaseQuery<Row>
         postAveragerSpecs,
         havingSpec,
         limitSpec,
+        groupByQueryForLimitSpec,
         limitFn,
         getContext()
     );
@@ -366,6 +374,7 @@ public class MovingAverageQuery extends BaseQuery<Row>
         postAveragerSpecs,
         havingSpec,
         limitSpec,
+        groupByQueryForLimitSpec,
         limitFn,
         getContext()
     );
@@ -373,6 +382,7 @@ public class MovingAverageQuery extends BaseQuery<Row>
 
   public Sequence<Row> applyLimit(Sequence<Row> results)
   {
-    return limitFn.apply(results);
+    return limitFn.apply(results.map(row -> ResultRow.fromLegacyRow(row, groupByQueryForLimitSpec)))
+                  .map(row -> row.toMapBasedRow(groupByQueryForLimitSpec));
   }
 }
diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java
index 53c4422..645a3b1 100644
--- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java
+++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java
@@ -40,6 +40,7 @@ import org.apache.druid.query.TableDataSource;
 import org.apache.druid.query.UnionDataSource;
 import org.apache.druid.query.context.ResponseContext;
 import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.movingaverage.averagers.AveragerFactory;
 import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
 import org.apache.druid.query.timeseries.TimeseriesQuery;
@@ -129,7 +130,7 @@ public class MovingAverageQueryRunner implements QueryRunner<Row>
       gbqResponseContext.put(QUERY_FAIL_TIME, System.currentTimeMillis() + QueryContexts.getTimeout(gbq));
       gbqResponseContext.put(QUERY_TOTAL_BYTES_GATHERED, new AtomicLong());
 
-      Sequence<Row> results = gbq.getRunner(walker).run(QueryPlus.wrap(gbq), gbqResponseContext);
+      Sequence<ResultRow> results = gbq.getRunner(walker).run(QueryPlus.wrap(gbq), gbqResponseContext);
       try {
         // use localhost for remote address
         requestLogger.logNativeQuery(RequestLogLine.forNative(
@@ -148,7 +149,7 @@ public class MovingAverageQueryRunner implements QueryRunner<Row>
         throw Throwables.propagate(e);
       }
 
-      resultsSeq = results;
+      resultsSeq = results.map(row -> row.toMapBasedRow(gbq));
     } else {
       // no dimensions, so optimize this as a TimeSeries
       TimeseriesQuery tsq = new TimeseriesQuery(
@@ -201,7 +202,8 @@ public class MovingAverageQueryRunner implements QueryRunner<Row>
             maq.getAveragerSpecs(),
             maq.getPostAggregatorSpecs(),
             maq.getAggregatorSpecs()
-        ));
+        )
+    );
 
     // Apply any postAveragers
     Sequence<Row> movingAvgResultsWithPostAveragers =
@@ -216,7 +218,7 @@ public class MovingAverageQueryRunner implements QueryRunner<Row>
         );
 
     // Apply any having, sorting, and limits
-    movingAvgResults = ((MovingAverageQuery) maq).applyLimit(movingAvgResults);
+    movingAvgResults = maq.applyLimit(movingAvgResults);
 
     return movingAvgResults;
 
diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java
index 625e01f..762f975 100644
--- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java
+++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java
@@ -60,6 +60,7 @@ import org.apache.druid.query.Result;
 import org.apache.druid.query.RetryQueryRunnerConfig;
 import org.apache.druid.query.SegmentDescriptor;
 import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.movingaverage.test.TestConfig;
 import org.apache.druid.query.timeseries.TimeseriesQuery;
 import org.apache.druid.query.timeseries.TimeseriesResultValue;
@@ -248,18 +249,16 @@ public class MovingAverageQueryTest
     timeseriesResults.clear();
 
     if (getGroupByResultJson() != null) {
-      groupByResults.addAll(jsonMapper.readValue(getGroupByResultJson(), new TypeReference<List<Row>>()
-      {
-      }));
+      groupByResults.addAll(jsonMapper.readValue(getGroupByResultJson(), new TypeReference<List<ResultRow>>() {}));
     }
 
     if (getTimeseriesResultJson() != null) {
-      timeseriesResults.addAll(jsonMapper.readValue(
-          getTimeseriesResultJson(),
-          new TypeReference<List<Result<TimeseriesResultValue>>>()
-          {
-          }
-      ));
+      timeseriesResults.addAll(
+          jsonMapper.readValue(
+              getTimeseriesResultJson(),
+              new TypeReference<List<Result<TimeseriesResultValue>>>() {}
+          )
+      );
     }
   }
 
diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage.yaml
index a3d2d16..fba40e8 100644
--- a/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage.yaml
+++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage.yaml
@@ -31,8 +31,7 @@ query:
     - type: constant
       name: seconds_per_minute
       value: 60.0
-  postAveragers: [
-  ]
+  postAveragers: []
 expectedOutput:
 - version: v1
   timestamp: 2017-01-02T00:00Z
@@ -43,15 +42,5 @@ expectedOutput:
     trailing7DayAvgTimeSpent: 3.0
 intermediateResults:
   groupBy:
-  - version: v1
-    timestamp: 2017-01-01T00:00Z
-    event:
-      gender: m
-      timespent_secs: 120.0
-      timeSpent: 2.0
-  - version: v1
-    timestamp: 2017-01-02T00:00Z
-    event:
-      gender: m
-      timespent_secs: 240.0
-      timeSpent: 4.0
+  - [1483228800000, m, 120.0, 2.0]
+  - [1483315200000, m, 240.0, 4.0]
diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage2.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage2.yaml
index a3d2d16..ac42208 100644
--- a/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage2.yaml
+++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage2.yaml
@@ -43,15 +43,5 @@ expectedOutput:
     trailing7DayAvgTimeSpent: 3.0
 intermediateResults:
   groupBy:
-  - version: v1
-    timestamp: 2017-01-01T00:00Z
-    event:
-      gender: m
-      timespent_secs: 120.0
-      timeSpent: 2.0
-  - version: v1
-    timestamp: 2017-01-02T00:00Z
-    event:
-      gender: m
-      timespent_secs: 240.0
-      timeSpent: 4.0
+  - [1483228800000, m, 120.0, 2.0]
+  - [1483315200000, m, 240.0, 4.0]
diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml
index c4ab5a4..c4bb0c5 100644
--- a/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml
+++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml
@@ -55,24 +55,6 @@ expectedOutput:
     someSum: 0.0
 intermediateResults:
   groupBy:
-  - version: v1
-    timestamp: 2017-01-01T00:00Z
-    event:
-      gender: m
-      timespent_secs: 120
-      timeSpent: 2.0
-      someSum: 5.0
-  - version: v1
-    timestamp: 2017-01-01T00:00Z
-    event:
-      gender: f
-      timespent_secs: 120
-      timeSpent: 2.0
-      someSum: 2.0
-  - version: v1
-    timestamp: 2017-01-02T00:00Z
-    event:
-      gender: m
-      timespent_secs: 240
-      timeSpent: 4.0
-      someSum: 3.0
+  - [1483228800000, m, 120, 5.0, 2.0]
+  - [1483228800000, f, 120, 2.0, 2.0]
+  - [1483315200000, m, 240, 3.0, 4.0]
diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersAsc.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersAsc.yaml
index ba685ff..161861c 100644
--- a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersAsc.yaml
+++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersAsc.yaml
@@ -54,28 +54,7 @@ expectedOutput:
     trailing7DayAvgTimeSpent: 6.0
 intermediateResults:
   groupBy:
-  - version: v1
-    timestamp: 2017-01-01T00:00Z
-    event:
-      gender: m
-      timespent_secs: 120.0
-      timeSpent: 2.0
-  - version: v1
-    timestamp: 2017-01-01T00:00Z
-    event:
-      gender: f
-      timespent_secs: 240.0
-      timeSpent: 4.0
-  - version: v1
-    timestamp: 2017-01-02T00:00Z
-    event:
-      gender: m
-      timespent_secs: 240.0
-      timeSpent: 4.0
-  - version: v1
-    timestamp: 2017-01-02T00:00Z
-    event:
-      gender: f
-      timespent_secs: 480.0
-      timeSpent: 8.0
-            
\ No newline at end of file
+  - [1483228800000, m, 120.0, 2.0]
+  - [1483228800000, f, 240.0, 4.0]
+  - [1483315200000, m, 240.0, 4.0]
+  - [1483315200000, f, 480.0, 8.0]
diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersDesc.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersDesc.yaml
index 59f75bc..08bf53c 100644
--- a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersDesc.yaml
+++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersDesc.yaml
@@ -55,28 +55,7 @@ expectedOutput:
     trailing7DayAvgTimeSpent: 3.0
 intermediateResults:
   groupBy:
-  - version: v1
-    timestamp: 2017-01-01T00:00Z
-    event:
-      gender: m
-      timespent_secs: 120.0
-      timeSpent: 2.0
-  - version: v1
-    timestamp: 2017-01-01T00:00Z
-    event:
-      gender: f
-      timespent_secs: 240.0
-      timeSpent: 4.0
-  - version: v1
-    timestamp: 2017-01-02T00:00Z
-    event:
-      gender: m
-      timespent_secs: 240.0
-      timeSpent: 4.0
-  - version: v1
-    timestamp: 2017-01-02T00:00Z
-    event:
-      gender: f
-      timespent_secs: 480.0
-      timeSpent: 8.0
-            
\ No newline at end of file
+  - [1483228800000, m, 120.0, 2.0]
+  - [1483228800000, f, 240.0, 4.0]
+  - [1483315200000, m, 240.0, 4.0]
+  - [1483315200000, f, 480.0, 8.0]
diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAndMovingAvgMetric.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAndMovingAvgMetric.yaml
index c7d7ddc..4b438c4 100644
--- a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAndMovingAvgMetric.yaml
+++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAndMovingAvgMetric.yaml
@@ -61,24 +61,6 @@ expectedOutput:
     trailing7DayAvgTotalPageViews: 2.142857142857143
 intermediateResults:
   groupBy:
-  - version: v1
-    timestamp: 2017-01-01T00:00Z
-    event:
-      gender: m
-      addPageViews: 5.0
-      pageViews: 10.0
-      totalPageViews: 15.0
-  - version: v1
-    timestamp: 2017-01-01T00:00Z
-    event:
-      gender: f
-      addPageViews: 6.0
-      pageViews: 12.0
-      totalPageViews: 18.0
-  - version: v1
-    timestamp: 2017-01-02T00:00Z
-    event:
-      gender: f
-      addPageViews: 1.0
-      pageViews: 2.0
-      totalPageViews: 3.0
\ No newline at end of file
+  - [1483228800000, m, 5.0, 10.0, 15.0]
+  - [1483228800000, f, 6.0, 12.0, 18.0]
+  - [1483315200000, f, 1.0, 2.0, 3.0]
diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAvgMetric.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAvgMetric.yaml
index 89ae941..0e8c635 100644
--- a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAvgMetric.yaml
+++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAvgMetric.yaml
@@ -59,24 +59,6 @@ expectedOutput:
     trailing7DayAvgTotalPageViews: 2.142857142857143
 intermediateResults:
   groupBy:
-  - version: v1
-    timestamp: 2017-01-01T00:00Z
-    event:
-      gender: m
-      addPageViews: 5.0
-      pageViews: 10.0
-      totalPageViews: 15.0
-  - version: v1
-    timestamp: 2017-01-01T00:00Z
-    event:
-      gender: f
-      addPageViews: 6.0
-      pageViews: 12.0
-      totalPageViews: 18.0
-  - version: v1
-    timestamp: 2017-01-02T00:00Z
-    event:
-      gender: f
-      addPageViews: 1.0
-      pageViews: 2.0
-      totalPageViews: 3.0
\ No newline at end of file
+  - [1483228800000, m, 5.0, 10.0, 15.0]
+  - [1483228800000, f, 6.0, 12.0, 18.0]
+  - [1483315200000, f, 1.0, 2.0, 3.0]
diff --git a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorTest.java b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorTest.java
index 88adde0..67773b1 100644
--- a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorTest.java
+++ b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorTest.java
@@ -20,7 +20,6 @@
 package org.apache.druid.query.aggregation.tdigestsketch;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
@@ -28,6 +27,7 @@ import org.apache.druid.query.aggregation.AggregationTestHelper;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -102,7 +102,7 @@ public class TDigestSketchAggregatorTest
   @Test
   public void buildingSketchesAtIngestionTime() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("doubles_build_data.tsv").getFile()),
         String.join(
             "\n",
@@ -144,12 +144,12 @@ public class TDigestSketchAggregatorTest
             "}"
         )
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
+    ResultRow row = results.get(0);
 
     // post agg
-    Object quantilesObject = row.getRaw("quantiles");
+    Object quantilesObject = row.get(1); // "quantiles"
     Assert.assertTrue(quantilesObject instanceof double[]);
     double[] quantiles = (double[]) quantilesObject;
     Assert.assertEquals(0, quantiles[0], 0.05); // min value
@@ -160,7 +160,7 @@ public class TDigestSketchAggregatorTest
   @Test
   public void buildingSketchesAtQueryTime() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("doubles_build_data.tsv").getFile()),
         String.join(
             "\n",
@@ -199,13 +199,13 @@ public class TDigestSketchAggregatorTest
             "}"
         )
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
+    ResultRow row = results.get(0);
 
 
     // post agg
-    Object quantilesObject = row.getRaw("quantiles");
+    Object quantilesObject = row.get(1); // "quantiles"
     Assert.assertTrue(quantilesObject instanceof double[]);
     double[] quantiles = (double[]) quantilesObject;
     Assert.assertEquals(0, quantiles[0], 0.05); // min value
@@ -216,7 +216,7 @@ public class TDigestSketchAggregatorTest
   @Test
   public void testIngestingSketches() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("doubles_sketch_data.tsv").getFile()),
         String.join(
             "\n",
@@ -264,12 +264,12 @@ public class TDigestSketchAggregatorTest
             "}"
         )
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
+    ResultRow row = results.get(0);
 
     // post agg
-    Object quantilesObject = row.getRaw("quantiles");
+    Object quantilesObject = row.get(1); // "quantiles"
     Assert.assertTrue(quantilesObject instanceof double[]);
     double[] quantiles = (double[]) quantilesObject;
     Assert.assertEquals(0, quantiles[0], 0.05); // min value
diff --git a/extensions-contrib/time-min-max/src/test/java/org/apache/druid/query/aggregation/TimestampGroupByAggregationTest.java b/extensions-contrib/time-min-max/src/test/java/org/apache/druid/query/aggregation/TimestampGroupByAggregationTest.java
index 1a11653..5a40641 100644
--- a/extensions-contrib/time-min-max/src/test/java/org/apache/druid/query/aggregation/TimestampGroupByAggregationTest.java
+++ b/extensions-contrib/time-min-max/src/test/java/org/apache/druid/query/aggregation/TimestampGroupByAggregationTest.java
@@ -21,13 +21,13 @@ package org.apache.druid.query.aggregation;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import org.apache.druid.data.input.MapBasedRow;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.easymock.EasyMock;
 import org.joda.time.DateTime;
@@ -175,7 +175,7 @@ public class TimestampGroupByAggregationTest
         "  ]\n" +
         "}";
     ZipFile zip = new ZipFile(new File(this.getClass().getClassLoader().getResource("druid.sample.tsv.zip").toURI()));
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         zip.getInputStream(zip.getEntry("druid.sample.tsv")),
         recordParser,
         aggregator,
@@ -185,8 +185,11 @@ public class TimestampGroupByAggregationTest
         groupBy
     );
 
-    List<Row> results = seq.toList();
+    int groupByFieldNumber = ((GroupByQuery) helper.readQuery(groupBy)).getResultRowPositionLookup()
+                                                                       .getInt(groupByField);
+
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(36, results.size());
-    Assert.assertEquals(expected, ((MapBasedRow) results.get(0)).getEvent().get(groupByField));
+    Assert.assertEquals(expected, results.get(0).get(groupByFieldNumber));
   }
 }
diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java
index 320f7d3..01631e3 100644
--- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java
+++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java
@@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableList;
 import org.apache.druid.collections.DefaultBlockingPool;
 import org.apache.druid.collections.StupidPool;
 import org.apache.druid.data.input.MapBasedRow;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.Intervals;
@@ -39,6 +38,7 @@ import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
 import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
 import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
 import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
@@ -53,13 +53,14 @@ import org.junit.rules.ExpectedException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
+import java.util.stream.Collectors;
 
 public class MapVirtualColumnGroupByTest
 {
   @Rule
   public ExpectedException expectedException = ExpectedException.none();
 
-  private QueryRunner<Row> runner;
+  private QueryRunner<ResultRow> runner;
 
   @Before
   public void setup() throws IOException
@@ -161,14 +162,14 @@ public class MapVirtualColumnGroupByTest
         null
     );
 
-    final List<Row> result = runner.run(QueryPlus.wrap(query)).toList();
-    final List<Row> expected = ImmutableList.of(
+    final List<ResultRow> result = runner.run(QueryPlus.wrap(query)).toList();
+    final List<ResultRow> expected = ImmutableList.of(
         new MapBasedRow(
             DateTimes.of("2011-01-12T00:00:00.000Z"),
             MapVirtualColumnTestBase.mapOf("count", 1L, "params.key3", "value3")
         ),
         new MapBasedRow(DateTimes.of("2011-01-12T00:00:00.000Z"), MapVirtualColumnTestBase.mapOf("count", 2L))
-    );
+    ).stream().map(row -> ResultRow.fromLegacyRow(row, query)).collect(Collectors.toList());
 
     Assert.assertEquals(expected, result);
   }
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java
index 1dc64cc..e2c55be 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java
@@ -22,12 +22,12 @@ package org.apache.druid.query.aggregation.datasketches.hll;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.aggregation.AggregationTestHelper;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -73,7 +73,7 @@ public class HllSketchAggregatorTest
   @Test
   public void ingestSketches() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("hll/hll_sketches.tsv").getFile()),
         buildParserJson(
             Arrays.asList("dim", "multiDim"),
@@ -85,16 +85,16 @@ public class HllSketchAggregatorTest
         200, // maxRowCount
         buildGroupByQueryJson("HLLSketchMerge", "sketch", !ROUND)
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    Assert.assertEquals(200, (double) row.getMetric("sketch"), 0.1);
+    ResultRow row = results.get(0);
+    Assert.assertEquals(200, (double) row.get(0), 0.1);
   }
 
   @Test
   public void buildSketchesAtIngestionTime() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("hll/hll_raw.tsv").getFile()),
         buildParserJson(
             Collections.singletonList("dim"),
@@ -106,16 +106,16 @@ public class HllSketchAggregatorTest
         200, // maxRowCount
         buildGroupByQueryJson("HLLSketchMerge", "sketch", !ROUND)
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    Assert.assertEquals(200, (double) row.getMetric("sketch"), 0.1);
+    ResultRow row = results.get(0);
+    Assert.assertEquals(200, (double) row.get(0), 0.1);
   }
 
   @Test
   public void buildSketchesAtQueryTime() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("hll/hll_raw.tsv").getFile()),
         buildParserJson(
             Arrays.asList("dim", "multiDim", "id"),
@@ -127,16 +127,16 @@ public class HllSketchAggregatorTest
         200, // maxRowCount
         buildGroupByQueryJson("HLLSketchBuild", "id", !ROUND)
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    Assert.assertEquals(200, (double) row.getMetric("sketch"), 0.1);
+    ResultRow row = results.get(0);
+    Assert.assertEquals(200, (double) row.get(0), 0.1);
   }
 
   @Test
   public void buildSketchesAtQueryTimeMultiValue() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("hll/hll_raw.tsv").getFile()),
         buildParserJson(
             Arrays.asList("dim", "multiDim", "id"),
@@ -148,16 +148,16 @@ public class HllSketchAggregatorTest
         200, // maxRowCount
         buildGroupByQueryJson("HLLSketchBuild", "multiDim", !ROUND)
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    Assert.assertEquals(14, (double) row.getMetric("sketch"), 0.1);
+    ResultRow row = results.get(0);
+    Assert.assertEquals(14, (double) row.get(0), 0.1);
   }
 
   @Test
   public void roundBuildSketch() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("hll/hll_raw.tsv").getFile()),
         buildParserJson(
             Arrays.asList("dim", "multiDim", "id"),
@@ -169,16 +169,16 @@ public class HllSketchAggregatorTest
         200, // maxRowCount
         buildGroupByQueryJson("HLLSketchBuild", "id", ROUND)
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    Assert.assertEquals(200L, (long) row.getMetric("sketch"));
+    ResultRow row = results.get(0);
+    Assert.assertEquals(200L, (long) row.get(0));
   }
 
   @Test
   public void roundMergeSketch() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("hll/hll_sketches.tsv").getFile()),
         buildParserJson(
             Arrays.asList("dim", "multiDim"),
@@ -190,10 +190,10 @@ public class HllSketchAggregatorTest
         200, // maxRowCount
         buildGroupByQueryJson("HLLSketchMerge", "sketch", ROUND)
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    Assert.assertEquals(200L, (long) row.getMetric("sketch"));
+    ResultRow row = results.get(0);
+    Assert.assertEquals(200L, (long) row.get(0));
   }
 
   private static String buildParserJson(List<String> dimensions, List<String> columns)
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java
index a29f649..2a2cc85 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java
@@ -20,7 +20,6 @@
 package org.apache.druid.query.aggregation.datasketches.quantiles;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
@@ -28,6 +27,7 @@ import org.apache.druid.query.aggregation.AggregationTestHelper;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Rule;
@@ -115,7 +115,7 @@ public class DoublesSketchAggregatorTest
   @Test
   public void ingestingSketches() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("quantiles/doubles_sketch_data.tsv").getFile()),
         String.join(
             "\n",
@@ -162,22 +162,22 @@ public class DoublesSketchAggregatorTest
             "}"
         )
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
+    ResultRow row = results.get(0);
 
-    Object nonExistentSketchObject = row.getRaw("non_existent_sketch");
+    Object nonExistentSketchObject = row.get(1);
     Assert.assertTrue(nonExistentSketchObject instanceof Long);
     long nonExistentSketchValue = (long) nonExistentSketchObject;
     Assert.assertEquals(0, nonExistentSketchValue);
 
-    Object sketchObject = row.getRaw("sketch");
+    Object sketchObject = row.get(0);
     Assert.assertTrue(sketchObject instanceof Long);
     long sketchValue = (long) sketchObject;
     Assert.assertEquals(400, sketchValue);
 
     // post agg
-    Object quantilesObject = row.getRaw("quantiles");
+    Object quantilesObject = row.get(2);
     Assert.assertTrue(quantilesObject instanceof double[]);
     double[] quantiles = (double[]) quantilesObject;
     Assert.assertEquals(0, quantiles[0], 0.05); // min value
@@ -185,7 +185,7 @@ public class DoublesSketchAggregatorTest
     Assert.assertEquals(1, quantiles[2], 0.05); // max value
 
     // post agg
-    Object histogramObject = row.getRaw("histogram");
+    Object histogramObject = row.get(3);
     Assert.assertTrue(histogramObject instanceof double[]);
     double[] histogram = (double[]) histogramObject;
     for (final double bin : histogram) {
@@ -197,7 +197,7 @@ public class DoublesSketchAggregatorTest
   @Test
   public void buildingSketchesAtIngestionTime() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()),
         String.join(
             "\n",
@@ -238,17 +238,17 @@ public class DoublesSketchAggregatorTest
             "}"
         )
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
+    ResultRow row = results.get(0);
 
-    Object sketchObject = row.getRaw("sketch");
+    Object sketchObject = row.get(0);
     Assert.assertTrue(sketchObject instanceof Long);
     long sketchValue = (long) sketchObject;
     Assert.assertEquals(400, sketchValue);
 
     // post agg
-    Object quantilesObject = row.getRaw("quantiles");
+    Object quantilesObject = row.get(2);
     Assert.assertTrue(quantilesObject instanceof double[]);
     double[] quantiles = (double[]) quantilesObject;
     Assert.assertEquals(0, quantiles[0], 0.05); // min value
@@ -256,7 +256,7 @@ public class DoublesSketchAggregatorTest
     Assert.assertEquals(1, quantiles[2], 0.05); // max value
 
     // post agg
-    Object histogramObject = row.getRaw("histogram");
+    Object histogramObject = row.get(3);
     Assert.assertTrue(histogramObject instanceof double[]);
     double[] histogram = (double[]) histogramObject;
     Assert.assertEquals(4, histogram.length);
@@ -268,7 +268,7 @@ public class DoublesSketchAggregatorTest
   @Test
   public void buildingSketchesAtQueryTime() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()),
         String.join(
             "\n",
@@ -309,22 +309,22 @@ public class DoublesSketchAggregatorTest
             "}"
         )
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
+    ResultRow row = results.get(0);
 
-    Object sketchObject = row.getRaw("sketch");
+    Object sketchObject = row.get(0);
     Assert.assertTrue(sketchObject instanceof Long);
     long sketchValue = (long) sketchObject;
     Assert.assertEquals(400, sketchValue);
 
     // post agg
-    Object quantileObject = row.getRaw("quantile");
+    Object quantileObject = row.get(1);
     Assert.assertTrue(quantileObject instanceof Double);
     Assert.assertEquals(0.5, (double) quantileObject, 0.05); // median value
 
     // post agg
-    Object quantilesObject = row.getRaw("quantiles");
+    Object quantilesObject = row.get(2);
     Assert.assertTrue(quantilesObject instanceof double[]);
     double[] quantiles = (double[]) quantilesObject;
     Assert.assertEquals(0, quantiles[0], 0.05); // min value
@@ -332,7 +332,7 @@ public class DoublesSketchAggregatorTest
     Assert.assertEquals(1, quantiles[2], 0.05); // max value
 
     // post agg
-    Object histogramObject = row.getRaw("histogram");
+    Object histogramObject = row.get(3);
     Assert.assertTrue(histogramObject instanceof double[]);
     double[] histogram = (double[]) histogramObject;
     for (final double bin : histogram) {
@@ -344,7 +344,7 @@ public class DoublesSketchAggregatorTest
   @Test
   public void queryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()),
         String.join(
             "\n",
@@ -385,22 +385,22 @@ public class DoublesSketchAggregatorTest
             "}"
         )
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
+    ResultRow row = results.get(0);
 
-    Object sketchObject = row.getRaw("sketch");
+    Object sketchObject = row.get(0);
     Assert.assertTrue(sketchObject instanceof Long);
     long sketchValue = (long) sketchObject;
     Assert.assertEquals(400, sketchValue);
 
     // post agg
-    Object quantileObject = row.getRaw("quantile");
+    Object quantileObject = row.get(1);
     Assert.assertTrue(quantileObject instanceof Double);
     Assert.assertEquals(0.5, (double) quantileObject, 0.05); // median value
 
     // post agg
-    Object quantilesObject = row.getRaw("quantiles");
+    Object quantilesObject = row.get(2);
     Assert.assertTrue(quantilesObject instanceof double[]);
     double[] quantiles = (double[]) quantilesObject;
     Assert.assertEquals(0, quantiles[0], 0.05); // min value
@@ -408,7 +408,7 @@ public class DoublesSketchAggregatorTest
     Assert.assertEquals(1, quantiles[2], 0.05); // max value
 
     // post agg
-    Object histogramObject = row.getRaw("histogram");
+    Object histogramObject = row.get(3);
     Assert.assertTrue(histogramObject instanceof double[]);
     double[] histogram = (double[]) histogramObject;
     for (final double bin : histogram) {
@@ -420,7 +420,7 @@ public class DoublesSketchAggregatorTest
   @Test
   public void timeSeriesQueryInputAsFloat() throws Exception
   {
-    Sequence<Row> seq = timeSeriesHelper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = timeSeriesHelper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()),
         String.join(
             "\n",
@@ -460,7 +460,7 @@ public class DoublesSketchAggregatorTest
             "}"
         )
     );
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
   }
 }
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java
index a2a4bad..980a093 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java
@@ -30,16 +30,18 @@ import com.yahoo.sketches.theta.Sketches;
 import com.yahoo.sketches.theta.Union;
 import com.yahoo.sketches.theta.UpdateSketch;
 import org.apache.druid.data.input.MapBasedRow;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
 import org.apache.druid.query.aggregation.AggregationTestHelper;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.PostAggregator;
 import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.epinephelinae.GrouperTestUtil;
 import org.apache.druid.query.groupby.epinephelinae.TestColumnSelectorFactory;
 import org.junit.After;
@@ -58,8 +60,10 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.List;
+import java.util.stream.Collectors;
 
 /**
+ *
  */
 @RunWith(Parameterized.class)
 public class SketchAggregationTest
@@ -98,34 +102,45 @@ public class SketchAggregationTest
   @Test
   public void testSketchDataIngestAndGpByQuery() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    final String groupByQueryString = readFileFromClasspathAsString("sketch_test_data_group_by_query.json");
+    final GroupByQuery groupByQuery = (GroupByQuery) helper.getObjectMapper()
+                                                           .readValue(groupByQueryString, Query.class);
+
+    final Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(SketchAggregationTest.class.getClassLoader().getResource("sketch_test_data.tsv").getFile()),
         readFileFromClasspathAsString("sketch_test_data_record_parser.json"),
         readFileFromClasspathAsString("sketch_test_data_aggregators.json"),
         0,
         Granularities.NONE,
         1000,
-        readFileFromClasspathAsString("sketch_test_data_group_by_query.json")
+        groupByQueryString
     );
 
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
     Assert.assertEquals(
-        new MapBasedRow(
-            DateTimes.of("2014-10-19T00:00:00.000Z"),
-            ImmutableMap
-                .<String, Object>builder()
-                .put("sids_sketch_count", 50.0)
-                .put("sids_sketch_count_with_err",
-                    new SketchEstimateWithErrorBounds(50.0, 50.0, 50.0, 2))
-                .put("sketchEstimatePostAgg", 50.0)
-                .put("sketchEstimatePostAggWithErrorBounds",
-                    new SketchEstimateWithErrorBounds(50.0, 50.0, 50.0, 2))
-                .put("sketchUnionPostAggEstimate", 50.0)
-                .put("sketchIntersectionPostAggEstimate", 50.0)
-                .put("sketchAnotBPostAggEstimate", 0.0)
-                .put("non_existing_col_validation", 0.0)
-                .build()
+        ResultRow.fromLegacyRow(
+            new MapBasedRow(
+                DateTimes.of("2014-10-19T00:00:00.000Z"),
+                ImmutableMap
+                    .<String, Object>builder()
+                    .put("sids_sketch_count", 50.0)
+                    .put(
+                        "sids_sketch_count_with_err",
+                        new SketchEstimateWithErrorBounds(50.0, 50.0, 50.0, 2)
+                    )
+                    .put("sketchEstimatePostAgg", 50.0)
+                    .put(
+                        "sketchEstimatePostAggWithErrorBounds",
+                        new SketchEstimateWithErrorBounds(50.0, 50.0, 50.0, 2)
+                    )
+                    .put("sketchUnionPostAggEstimate", 50.0)
+                    .put("sketchIntersectionPostAggEstimate", 50.0)
+                    .put("sketchAnotBPostAggEstimate", 0.0)
+                    .put("non_existing_col_validation", 0.0)
+                    .build()
+            ),
+            groupByQuery
         ),
         results.get(0)
     );
@@ -134,7 +149,11 @@ public class SketchAggregationTest
   @Test
   public void testThetaCardinalityOnSimpleColumn() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    final String groupByQueryString = readFileFromClasspathAsString("simple_test_data_group_by_query.json");
+    final GroupByQuery groupByQuery = (GroupByQuery) helper.getObjectMapper()
+                                                           .readValue(groupByQueryString, Query.class);
+
+    final Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(SketchAggregationTest.class.getClassLoader().getResource("simple_test_data.tsv").getFile()),
         readFileFromClasspathAsString("simple_test_data_record_parser2.json"),
         "["
@@ -146,10 +165,10 @@ public class SketchAggregationTest
         0,
         Granularities.NONE,
         1000,
-        readFileFromClasspathAsString("simple_test_data_group_by_query.json")
+        groupByQueryString
     );
 
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(5, results.size());
     Assert.assertEquals(
         ImmutableList.of(
@@ -218,7 +237,7 @@ public class SketchAggregationTest
                     .put("non_existing_col_validation", 0.0)
                     .build()
             )
-        ),
+        ).stream().map(row -> ResultRow.fromLegacyRow(row, groupByQuery)).collect(Collectors.toList()),
         results
     );
   }
@@ -284,7 +303,7 @@ public class SketchAggregationTest
             2
         )
     );
-    
+
     assertPostAggregatorSerde(
         new SketchEstimatePostAggregator(
             "name",
@@ -308,7 +327,7 @@ public class SketchAggregationTest
             )
         )
     );
-    
+
     assertPostAggregatorSerde(
         new SketchSetPostAggregator(
             "name",
@@ -357,17 +376,21 @@ public class SketchAggregationTest
   @Test
   public void testRetentionDataIngestAndGpByQuery() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    final String groupByQueryString = readFileFromClasspathAsString("retention_test_data_group_by_query.json");
+    final GroupByQuery groupByQuery = (GroupByQuery) helper.getObjectMapper()
+                                                           .readValue(groupByQueryString, Query.class);
+
+    final Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("retention_test_data.tsv").getFile()),
         readFileFromClasspathAsString("simple_test_data_record_parser.json"),
         readFileFromClasspathAsString("simple_test_data_aggregators.json"),
         0,
         Granularities.NONE,
         5,
-        readFileFromClasspathAsString("retention_test_data_group_by_query.json")
+        groupByQueryString
     );
 
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
     Assert.assertEquals(
         ImmutableList.of(
@@ -385,7 +408,7 @@ public class SketchAggregationTest
                     .put("non_existing_col_validation", 0.0)
                     .build()
             )
-        ),
+        ).stream().map(row -> ResultRow.fromLegacyRow(row, groupByQuery)).collect(Collectors.toList()),
         results
     );
   }
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java
index 0ee0e49..8581eb1 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java
@@ -24,14 +24,16 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 import com.google.common.io.Files;
 import org.apache.druid.data.input.MapBasedRow;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
 import org.apache.druid.query.Result;
 import org.apache.druid.query.aggregation.AggregationTestHelper;
+import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.select.SelectResultValue;
 import org.apache.druid.query.timeseries.TimeseriesResultValue;
 import org.apache.druid.query.topn.DimensionAndMetricValueExtractor;
@@ -52,6 +54,7 @@ import java.util.Collection;
 import java.util.List;
 
 /**
+ *
  */
 @RunWith(Parameterized.class)
 public class SketchAggregationWithSimpleDataTest
@@ -127,13 +130,17 @@ public class SketchAggregationWithSimpleDataTest
             tempFolder
         )
     ) {
+      final String groupByQueryString = readFileFromClasspathAsString("simple_test_data_group_by_query.json");
+      final GroupByQuery groupByQuery = (GroupByQuery) gpByQueryAggregationTestHelper
+          .getObjectMapper()
+          .readValue(groupByQueryString, Query.class);
 
-      Sequence seq = gpByQueryAggregationTestHelper.runQueryOnSegments(
+      Sequence<ResultRow> seq = gpByQueryAggregationTestHelper.runQueryOnSegments(
           ImmutableList.of(s1, s2),
-          readFileFromClasspathAsString("simple_test_data_group_by_query.json")
+          groupByQueryString
       );
 
-      List<Row> results = seq.toList();
+      List<MapBasedRow> results = seq.map(row -> row.toMapBasedRow(groupByQuery)).toList();
       Assert.assertEquals(5, results.size());
       Assert.assertEquals(
           ImmutableList.of(
@@ -279,9 +286,12 @@ public class SketchAggregationWithSimpleDataTest
     Result<SelectResultValue> result = (Result<SelectResultValue>) Iterables.getOnlyElement(seq.toList());
     Assert.assertEquals(DateTimes.of("2014-10-20T00:00:00.000Z"), result.getTimestamp());
     Assert.assertEquals(100, result.getValue().getEvents().size());
-    Assert.assertEquals("AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=", result.getValue().getEvents().get(0).getEvent().get("pty_country"));
+    Assert.assertEquals(
+        "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=",
+        result.getValue().getEvents().get(0).getEvent().get("pty_country")
+    );
   }
-  
+
   @Test
   public void testTopNQueryWithSketchConstant() throws Exception
   {
@@ -294,9 +304,9 @@ public class SketchAggregationWithSimpleDataTest
         ImmutableList.of(s1, s2),
         readFileFromClasspathAsString("topn_query_sketch_const.json")
     );
-    
+
     Result<TopNResultValue> result = (Result<TopNResultValue>) Iterables.getOnlyElement(seq.toList());
-    
+
     Assert.assertEquals(DateTimes.of("2014-10-20T00:00:00.000Z"), result.getTimestamp());
 
     DimensionAndMetricValueExtractor value1 = Iterables.get(result.getValue().getValue(), 0);
@@ -307,7 +317,7 @@ public class SketchAggregationWithSimpleDataTest
     Assert.assertEquals(1.0, value1.getDoubleMetric("sketchIntersectionPostAggEstimate"), 0.01);
     Assert.assertEquals(37.0, value1.getDoubleMetric("sketchAnotBPostAggEstimate"), 0.01);
     Assert.assertEquals("product_3", value1.getDimensionValue("product"));
-    
+
     DimensionAndMetricValueExtractor value2 = Iterables.get(result.getValue().getValue(), 1);
     Assert.assertEquals(42.0, value2.getDoubleMetric("sketch_count"), 0.01);
     Assert.assertEquals(42.0, value2.getDoubleMetric("sketchEstimatePostAgg"), 0.01);
@@ -316,7 +326,7 @@ public class SketchAggregationWithSimpleDataTest
     Assert.assertEquals(2.0, value2.getDoubleMetric("sketchIntersectionPostAggEstimate"), 0.01);
     Assert.assertEquals(40.0, value2.getDoubleMetric("sketchAnotBPostAggEstimate"), 0.01);
     Assert.assertEquals("product_1", value2.getDimensionValue("product"));
-    
+
     DimensionAndMetricValueExtractor value3 = Iterables.get(result.getValue().getValue(), 2);
     Assert.assertEquals(42.0, value3.getDoubleMetric("sketch_count"), 0.01);
     Assert.assertEquals(42.0, value3.getDoubleMetric("sketchEstimatePostAgg"), 0.01);
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java
index 9155061..66e0b88 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java
@@ -28,13 +28,16 @@ import org.apache.druid.data.input.MapBasedRow;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
 import org.apache.druid.query.aggregation.AggregationTestHelper;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.PostAggregator;
 import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder;
 import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.epinephelinae.GrouperTestUtil;
 import org.apache.druid.query.groupby.epinephelinae.TestColumnSelectorFactory;
 import org.junit.After;
@@ -53,6 +56,7 @@ import java.util.Collection;
 import java.util.List;
 
 /**
+ *
  */
 @RunWith(Parameterized.class)
 public class OldApiSketchAggregationTest
@@ -93,30 +97,37 @@ public class OldApiSketchAggregationTest
   @Test
   public void testSimpleDataIngestAndQuery() throws Exception
   {
-    Sequence seq = helper.createIndexAndRunQueryOnSegment(
+    final String groupByQueryString = readFileFromClasspathAsString("oldapi/old_simple_test_data_group_by_query.json");
+    final GroupByQuery groupByQuery = (GroupByQuery) helper.getObjectMapper()
+                                                           .readValue(groupByQueryString, Query.class);
+
+    final Sequence seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("simple_test_data.tsv").getFile()),
         readFileFromClasspathAsString("simple_test_data_record_parser.json"),
         readFileFromClasspathAsString("oldapi/old_simple_test_data_aggregators.json"),
         0,
         Granularities.NONE,
         1000,
-        readFileFromClasspathAsString("oldapi/old_simple_test_data_group_by_query.json")
+        groupByQueryString
     );
 
     List results = seq.toList();
     Assert.assertEquals(1, results.size());
     Assert.assertEquals(
-        new MapBasedRow(
-            DateTimes.of("2014-10-19T00:00:00.000Z"),
-            ImmutableMap
-                .<String, Object>builder()
-                .put("sketch_count", 50.0)
-                .put("sketchEstimatePostAgg", 50.0)
-                .put("sketchUnionPostAggEstimate", 50.0)
-                .put("sketchIntersectionPostAggEstimate", 50.0)
-                .put("sketchAnotBPostAggEstimate", 0.0)
-                .put("non_existing_col_validation", 0.0)
-                .build()
+        ResultRow.fromLegacyRow(
+            new MapBasedRow(
+                DateTimes.of("2014-10-19T00:00:00.000Z"),
+                ImmutableMap
+                    .<String, Object>builder()
+                    .put("sketch_count", 50.0)
+                    .put("sketchEstimatePostAgg", 50.0)
+                    .put("sketchUnionPostAggEstimate", 50.0)
+                    .put("sketchIntersectionPostAggEstimate", 50.0)
+                    .put("sketchAnotBPostAggEstimate", 0.0)
+                    .put("non_existing_col_validation", 0.0)
+                    .build()
+            ),
+            groupByQuery
         ),
         results.get(0)
     );
@@ -125,30 +136,37 @@ public class OldApiSketchAggregationTest
   @Test
   public void testSketchDataIngestAndQuery() throws Exception
   {
-    Sequence seq = helper.createIndexAndRunQueryOnSegment(
+    final String groupByQueryString = readFileFromClasspathAsString("oldapi/old_sketch_test_data_group_by_query.json");
+    final GroupByQuery groupByQuery = (GroupByQuery) helper.getObjectMapper()
+                                                           .readValue(groupByQueryString, Query.class);
+
+    final Sequence seq = helper.createIndexAndRunQueryOnSegment(
         new File(OldApiSketchAggregationTest.class.getClassLoader().getResource("sketch_test_data.tsv").getFile()),
         readFileFromClasspathAsString("sketch_test_data_record_parser.json"),
         readFileFromClasspathAsString("oldapi/old_sketch_test_data_aggregators.json"),
         0,
         Granularities.NONE,
         1000,
-        readFileFromClasspathAsString("oldapi/old_sketch_test_data_group_by_query.json")
+        groupByQueryString
     );
 
     List results = seq.toList();
     Assert.assertEquals(1, results.size());
     Assert.assertEquals(
-        new MapBasedRow(
-            DateTimes.of("2014-10-19T00:00:00.000Z"),
-            ImmutableMap
-                .<String, Object>builder()
-                .put("sids_sketch_count", 50.0)
-                .put("sketchEstimatePostAgg", 50.0)
-                .put("sketchUnionPostAggEstimate", 50.0)
-                .put("sketchIntersectionPostAggEstimate", 50.0)
-                .put("sketchAnotBPostAggEstimate", 0.0)
-                .put("non_existing_col_validation", 0.0)
-                .build()
+        ResultRow.fromLegacyRow(
+            new MapBasedRow(
+                DateTimes.of("2014-10-19T00:00:00.000Z"),
+                ImmutableMap
+                    .<String, Object>builder()
+                    .put("sids_sketch_count", 50.0)
+                    .put("sketchEstimatePostAgg", 50.0)
+                    .put("sketchUnionPostAggEstimate", 50.0)
+                    .put("sketchIntersectionPostAggEstimate", 50.0)
+                    .put("sketchAnotBPostAggEstimate", 0.0)
+                    .put("non_existing_col_validation", 0.0)
+                    .build()
+            ),
+            groupByQuery
         ),
         results.get(0)
     );
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregationTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregationTest.java
index bb1e70c..cc599d6 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregationTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregationTest.java
@@ -20,13 +20,13 @@
 package org.apache.druid.query.aggregation.datasketches.tuple;
 
 import com.yahoo.sketches.quantiles.DoublesSketch;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.initialization.DruidModule;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.aggregation.AggregationTestHelper;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Rule;
@@ -77,7 +77,7 @@ public class ArrayOfDoublesSketchAggregationTest
   @Test
   public void ingestingSketches() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("tuple/array_of_doubles_sketch_data.tsv").getFile()),
         String.join("\n",
             "{",
@@ -138,17 +138,17 @@ public class ArrayOfDoublesSketchAggregationTest
             "  ],",
             "  \"intervals\": [\"2015-01-01T00:00:00.000Z/2015-01-31T00:00:00.000Z\"]",
             "}"));
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    Assert.assertEquals(0, (double) row.getMetric("non_existing_sketch"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("sketch"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("estimate"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("union"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("intersection"), 0);
-    Assert.assertEquals(0, (double) row.getRaw("anotb"), 0);
+    ResultRow row = results.get(0);
+    Assert.assertEquals("non_existing_sketch", 0, (double) row.get(1), 0);
+    Assert.assertEquals("sketch", 40.0, (double) row.get(0), 0);
+    Assert.assertEquals("estimate", 40.0, (double) row.get(2), 0);
+    Assert.assertEquals("union", 40.0, (double) row.get(4), 0);
+    Assert.assertEquals("intersection", 40.0, (double) row.get(5), 0);
+    Assert.assertEquals("anotb", 0, (double) row.get(6), 0);
 
-    Object obj = row.getRaw("quantiles-sketch");
+    Object obj = row.get(3); // quantiles-sketch
     Assert.assertTrue(obj instanceof DoublesSketch);
     DoublesSketch ds = (DoublesSketch) obj;
     Assert.assertEquals(40, ds.getN());
@@ -159,7 +159,7 @@ public class ArrayOfDoublesSketchAggregationTest
   @Test
   public void ingestingSketchesTwoValues() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("tuple/array_of_doubles_sketch_data_two_values.tsv")
             .getFile()),
         String.join("\n",
@@ -225,23 +225,23 @@ public class ArrayOfDoublesSketchAggregationTest
             "  ],",
             "  \"intervals\": [\"2015-01-01T00:00:00.000Z/2015-01-31T00:00:00.000Z\"]",
             "}"));
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    Assert.assertEquals(40.0, (double) row.getRaw("sketch"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("estimate"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("union"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("intersection"), 0);
-    Assert.assertEquals(0, (double) row.getRaw("anotb"), 0);
+    ResultRow row = results.get(0);
+    Assert.assertEquals("sketch", 40.0, (double) row.get(0), 0);
+    Assert.assertEquals("estimate", 40.0, (double) row.get(1), 0);
+    Assert.assertEquals("union", 40.0, (double) row.get(3), 0);
+    Assert.assertEquals("intersection", 40.0, (double) row.get(4), 0);
+    Assert.assertEquals("anotb", 0, (double) row.get(5), 0);
 
-    Object meansObj = row.getRaw("means");
+    Object meansObj = row.get(6); // means
     Assert.assertTrue(meansObj instanceof double[]);
     double[] means = (double[]) meansObj;
     Assert.assertEquals(2, means.length);
     Assert.assertEquals(1.0, means[0], 0);
     Assert.assertEquals(2.0, means[1], 0);
 
-    Object quantilesObj = row.getRaw("quantiles-sketch");
+    Object quantilesObj = row.get(2); // quantiles-sketch
     Assert.assertTrue(quantilesObj instanceof DoublesSketch);
     DoublesSketch ds = (DoublesSketch) quantilesObj;
     Assert.assertEquals(40, ds.getN());
@@ -252,7 +252,7 @@ public class ArrayOfDoublesSketchAggregationTest
   @Test
   public void buildingSketchesAtIngestionTime() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("tuple/array_of_doubles_build_data.tsv").getFile()),
         String.join("\n",
             "{",
@@ -311,16 +311,16 @@ public class ArrayOfDoublesSketchAggregationTest
             "  ],",
             "  \"intervals\": [\"2015-01-01T00:00:00.000Z/2015-01-31T00:00:00.000Z\"]",
             "}"));
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    Assert.assertEquals(40.0, (double) row.getRaw("sketch"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("estimate"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("union"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("intersection"), 0);
-    Assert.assertEquals(0, (double) row.getRaw("anotb"), 0);
+    ResultRow row = results.get(0);
+    Assert.assertEquals("sketch", 40.0, (double) row.get(0), 0);
+    Assert.assertEquals("estimate", 40.0, (double) row.get(1), 0);
+    Assert.assertEquals("union", 40.0, (double) row.get(3), 0);
+    Assert.assertEquals("intersection", 40.0, (double) row.get(4), 0);
+    Assert.assertEquals("anotb", 0, (double) row.get(5), 0);
 
-    Object obj = row.getRaw("quantiles-sketch");
+    Object obj = row.get(2);  // quantiles-sketch
     Assert.assertTrue(obj instanceof DoublesSketch);
     DoublesSketch ds = (DoublesSketch) obj;
     Assert.assertEquals(40, ds.getN());
@@ -331,7 +331,7 @@ public class ArrayOfDoublesSketchAggregationTest
   @Test
   public void buildingSketchesAtIngestionTimeTwoValues() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(
             this.getClass().getClassLoader().getResource("tuple/array_of_doubles_build_data_two_values.tsv").getFile()),
         String.join("\n",
@@ -399,23 +399,23 @@ public class ArrayOfDoublesSketchAggregationTest
             "  ],",
             "  \"intervals\": [\"2015-01-01T00:00:00.000Z/2015-01-31T00:00:00.000Z\"]",
             "}"));
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    Assert.assertEquals(40.0, (double) row.getRaw("sketch"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("estimate"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("union"), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("intersection"), 0);
-    Assert.assertEquals(0, (double) row.getRaw("anotb"), 0);
+    ResultRow row = results.get(0);
+    Assert.assertEquals("sketch", 40.0, (double) row.get(0), 0);
+    Assert.assertEquals("estimate", 40.0, (double) row.get(1), 0);
+    Assert.assertEquals("union", 40.0, (double) row.get(3), 0);
+    Assert.assertEquals("intersection", 40.0, (double) row.get(4), 0);
+    Assert.assertEquals("anotb", 0, (double) row.get(5), 0);
 
-    Object meansObj = row.getRaw("means");
+    Object meansObj = row.get(6); // means
     Assert.assertTrue(meansObj instanceof double[]);
     double[] means = (double[]) meansObj;
     Assert.assertEquals(2, means.length);
     Assert.assertEquals(1.0, means[0], 0);
     Assert.assertEquals(2.0, means[1], 0);
 
-    Object obj = row.getRaw("quantiles-sketch");
+    Object obj = row.get(2); // quantiles-sketch
     Assert.assertTrue(obj instanceof DoublesSketch);
     DoublesSketch ds = (DoublesSketch) obj;
     Assert.assertEquals(40, ds.getN());
@@ -426,7 +426,7 @@ public class ArrayOfDoublesSketchAggregationTest
   @Test
   public void buildingSketchesAtQueryTime() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("tuple/array_of_doubles_build_data.tsv").getFile()),
         String.join("\n",
             "{",
@@ -486,17 +486,17 @@ public class ArrayOfDoublesSketchAggregationTest
             "  ],",
             "  \"intervals\": [\"2015-01-01T00:00:00.000Z/2015-01-31T00:00:00.000Z\"]",
             "}"));
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    Assert.assertEquals(40.0, new Double(row.getRaw("cnt").toString()), 0);
-    Assert.assertEquals(40.0, (double) row.getRaw("sketch"), 0);
-    Assert.assertEquals(40.0, new Double(row.getRaw("estimate").toString()), 0);
-    Assert.assertEquals(40.0, new Double(row.getRaw("union").toString()), 0);
-    Assert.assertEquals(40.0, new Double(row.getRaw("intersection").toString()), 0);
-    Assert.assertEquals(0, new Double(row.getRaw("anotb").toString()), 0);
+    ResultRow row = results.get(0);
+    Assert.assertEquals("cnt", 40.0, new Double(row.get(1).toString()), 0);
+    Assert.assertEquals("sketch", 40.0, (double) row.get(0), 0);
+    Assert.assertEquals("estimate", 40.0, new Double(row.get(2).toString()), 0);
+    Assert.assertEquals("union", 40.0, new Double(row.get(4).toString()), 0);
+    Assert.assertEquals("intersection", 40.0, new Double(row.get(5).toString()), 0);
+    Assert.assertEquals("anotb", 0, new Double(row.get(6).toString()), 0);
 
-    Object obj = row.getRaw("quantiles-sketch");
+    Object obj = row.get(3); // quantiles-sketch
     Assert.assertTrue(obj instanceof DoublesSketch);
     DoublesSketch ds = (DoublesSketch) obj;
     Assert.assertEquals(40, ds.getN());
@@ -509,7 +509,7 @@ public class ArrayOfDoublesSketchAggregationTest
   @Test
   public void buildingSketchesAtQueryTimeAndTTest() throws Exception
   {
-    Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("tuple/bucket_test_data.tsv").getFile()),
         String.join("\n",
             "{",
@@ -558,10 +558,10 @@ public class ArrayOfDoublesSketchAggregationTest
             "  ],",
             "  \"intervals\": [\"2017-01-01T00:00:00.000Z/2017-01-31T00:00:00.000Z\"]",
             "}"));
-    List<Row> results = seq.toList();
+    List<ResultRow> results = seq.toList();
     Assert.assertEquals(1, results.size());
-    Row row = results.get(0);
-    Object obj = row.getRaw("p-value");
+    ResultRow row = results.get(0);
+    Object obj = row.get(2); // p-value
     Assert.assertTrue(obj instanceof double[]);
     double[] array = (double[]) obj;
     Assert.assertEquals(1, array.length);
diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java
index 5661e7b..003dc27 100644
--- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java
+++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java
@@ -30,8 +30,10 @@ import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.aggregation.AggregationTestHelper;
 import org.apache.druid.query.filter.BloomKFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
 import org.apache.druid.segment.TestHelper;
 import org.junit.After;
@@ -241,7 +243,7 @@ public class BloomFilterGroupByQueryTest
                        + "  }"
                        + "}";
 
-    Sequence seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         this.getClass().getClassLoader().getResourceAsStream("sample.data.tsv"),
         parseSpec,
         metricSpec,
@@ -251,6 +253,6 @@ public class BloomFilterGroupByQueryTest
         query
     );
 
-    return (MapBasedRow) seq.toList().get(0);
+    return seq.toList().get(0).toMapBasedRow((GroupByQuery) helper.readQuery(query));
   }
 }
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java
index 36103b9..01cef7d 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java
@@ -25,8 +25,10 @@ import org.apache.druid.data.input.MapBasedRow;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.aggregation.AggregationTestHelper;
+import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Rule;
@@ -41,6 +43,7 @@ import java.util.Collection;
 import java.util.List;
 
 /**
+ *
  */
 @RunWith(Parameterized.class)
 public class ApproximateHistogramAggregationTest
@@ -141,7 +144,7 @@ public class ApproximateHistogramAggregationTest
                    + "\"intervals\": [ \"1970/2050\" ]"
                    + "}";
 
-    Sequence seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         this.getClass().getClassLoader().getResourceAsStream("sample.data.tsv"),
         parseSpec,
         metricSpec,
@@ -151,6 +154,6 @@ public class ApproximateHistogramAggregationTest
         query
     );
 
-    return (MapBasedRow) seq.toList().get(0);
+    return seq.toList().get(0).toMapBasedRow((GroupByQuery) helper.readQuery(query));
   }
 }
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java
index f5323e0..245fde2 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java
@@ -32,6 +32,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTestHelper;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
 import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
 import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
@@ -124,7 +125,7 @@ public class ApproximateHistogramGroupByQueryTest
       );
       final GroupByQueryRunnerFactory factory = factoryAndCloser.lhs;
       resourceCloser.register(factoryAndCloser.rhs);
-      for (QueryRunner<Row> runner : QueryRunnerTestHelper.makeQueryRunners(factory)) {
+      for (QueryRunner<ResultRow> runner : QueryRunnerTestHelper.makeQueryRunners(factory)) {
         final String testName = StringUtils.format(
             "config=%s, runner=%s",
             config.toString(),
@@ -187,8 +188,9 @@ public class ApproximateHistogramGroupByQueryTest
         )
         .build();
 
-    List<Row> expectedResults = Collections.singletonList(
+    List<ResultRow> expectedResults = Collections.singletonList(
         GroupByQueryRunnerTestHelper.createExpectedRow(
+            query,
             "1970-01-01T00:00:00.000Z",
             "marketalias", "upfront",
             "rows", 186L,
@@ -210,7 +212,7 @@ public class ApproximateHistogramGroupByQueryTest
         )
     );
 
-    Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
+    Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
     TestHelper.assertExpectedObjects(expectedResults, results, "approx-histo");
   }
 
@@ -247,8 +249,9 @@ public class ApproximateHistogramGroupByQueryTest
         )
         .build();
 
-    List<Row> expectedResults = Collections.singletonList(
+    List<ResultRow> expectedResults = Collections.singletonList(
         GroupByQueryRunnerTestHelper.createExpectedRow(
+            query,
             "1970-01-01T00:00:00.000Z",
             "marketalias", "upfront",
             "rows", 186L,
@@ -256,7 +259,7 @@ public class ApproximateHistogramGroupByQueryTest
         )
     );
 
-    Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
+    Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
     TestHelper.assertExpectedObjects(expectedResults, results, "approx-histo");
   }
 }
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregationTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregationTest.java
index 1d81737..5fbb694 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregationTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregationTest.java
@@ -25,8 +25,10 @@ import org.apache.druid.data.input.MapBasedRow;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.aggregation.AggregationTestHelper;
+import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Rule;
@@ -41,6 +43,7 @@ import java.util.Collection;
 import java.util.List;
 
 /**
+ *
  */
 @RunWith(Parameterized.class)
 public class FixedBucketsHistogramAggregationTest
@@ -146,7 +149,7 @@ public class FixedBucketsHistogramAggregationTest
                    + "\"intervals\": [ \"1970/2050\" ]"
                    + "}";
 
-    Sequence seq = helper.createIndexAndRunQueryOnSegment(
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
         this.getClass().getClassLoader().getResourceAsStream("sample.data.tsv"),
         parseSpec,
         metricSpec,
@@ -156,6 +159,6 @@ public class FixedBucketsHistogramAggregationTest
         query
     );
 
-    return (MapBasedRow) seq.toList().get(0);
+    return seq.toList().get(0).toMapBasedRow((GroupByQuery) helper.readQuery(query));
   }
 }
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java
index d7761af..929a346 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java
@@ -32,6 +32,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTestHelper;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
 import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
 import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
@@ -124,7 +125,7 @@ public class FixedBucketsHistogramGroupByQueryTest
       );
       final GroupByQueryRunnerFactory factory = factoryAndCloser.lhs;
       resourceCloser.register(factoryAndCloser.rhs);
-      for (QueryRunner<Row> runner : QueryRunnerTestHelper.makeQueryRunners(factory)) {
+      for (QueryRunner<ResultRow> runner : QueryRunnerTestHelper.makeQueryRunners(factory)) {
         final String testName = StringUtils.format(
             "config=%s, runner=%s",
             config.toString(),
@@ -188,8 +189,9 @@ public class FixedBucketsHistogramGroupByQueryTest
         )
         .build();
 
-    List<Row> expectedResults = Collections.singletonList(
+    List<ResultRow> expectedResults = Collections.singletonList(
         GroupByQueryRunnerTestHelper.createExpectedRow(
+            query,
             "1970-01-01T00:00:00.000Z",
             "marketalias", "upfront",
             "rows", 186L,
@@ -211,7 +213,7 @@ public class FixedBucketsHistogramGroupByQueryTest
         )
     );
 
-    Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
+    Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
     TestHelper.assertExpectedObjects(expectedResults, results, "fixed-histo");
   }
 
@@ -248,8 +250,9 @@ public class FixedBucketsHistogramGroupByQueryTest
         )
         .build();
 
-    List<Row> expectedResults = Collections.singletonList(
+    List<ResultRow> expectedResults = Collections.singletonList(
         GroupByQueryRunnerTestHelper.createExpectedRow(
+            query,
             "1970-01-01T00:00:00.000Z",
             "marketalias", "upfront",
             "rows", 186L,
@@ -257,7 +260,7 @@ public class FixedBucketsHistogramGroupByQueryTest
         )
     );
 
-    Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
+    Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
     TestHelper.assertExpectedObjects(expectedResults, results, "fixed-histo");
   }
 }
diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java
index b302aef..d769eff 100644
--- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java
+++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java
@@ -32,6 +32,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTestHelper;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.having.GreaterThanHavingSpec;
 import org.apache.druid.query.groupby.having.OrHavingSpec;
 import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
@@ -106,7 +107,7 @@ public class VarianceGroupByQueryTest
     VarianceTestHelper.RowBuilder builder =
         new VarianceTestHelper.RowBuilder(new String[]{"alias", "index_stddev", "index_var"});
 
-    List<Row> expectedResults = builder
+    List<ResultRow> expectedResults = builder
         .add("2011-04-01", "automotive", 0d, 0d)
         .add("2011-04-01", "business", 0d, 0d)
         .add("2011-04-01", "entertainment", 0d, 0d)
@@ -126,9 +127,9 @@ public class VarianceGroupByQueryTest
         .add("2011-04-02", "premium", 621.3898134843073d, 386125.30030206224d)
         .add("2011-04-02", "technology", 0d, 0d)
         .add("2011-04-02", "travel", 0d, 0d)
-        .build();
+        .build(query);
 
-    Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
+    Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
     TestHelper.assertExpectedObjects(expectedResults, results, "variance");
   }
 
@@ -152,7 +153,7 @@ public class VarianceGroupByQueryTest
     VarianceTestHelper.RowBuilder builder =
         new VarianceTestHelper.RowBuilder(new String[]{"alias", "rows", "idx", "index_stddev", "index_var"});
 
-    List<Row> expectedResults = builder
+    List<ResultRow> expectedResults = builder
         .add("2011-04-01", "automotive", 1L, 135L, 0d, 0d)
         .add("2011-04-01", "business", 1L, 118L, 0d, 0d)
         .add("2011-04-01", "entertainment", 1L, 158L, 0d, 0d)
@@ -172,9 +173,9 @@ public class VarianceGroupByQueryTest
         .add("2011-04-02", "premium", 3L, 2505L, 621.3898134843073d, 386125.30030206224d)
         .add("2011-04-02", "technology", 1L, 97L, 0d, 0d)
         .add("2011-04-02", "travel", 1L, 126L, 0d, 0d)
-        .build();
+        .build(query);
 
-    Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
+    Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
     TestHelper.assertExpectedObjects(expectedResults, results, "groupBy");
   }
 
@@ -185,12 +186,6 @@ public class VarianceGroupByQueryTest
         new String[]{"alias", "rows", "index", "index_var", "index_stddev"}
     );
 
-    List<Row> expectedResults = expect
-        .add("2011-04-01", "automotive", 2L, 269L, 299.0009819048282, 17.29164485827847)
-        .add("2011-04-01", "mezzanine", 6L, 4420L, 254083.76447001836, 504.06722217380724)
-        .add("2011-04-01", "premium", 6L, 4416L, 252279.2020389339, 502.27403082275106)
-        .build();
-
     GroupByQuery query = GroupByQuery
         .builder()
         .setDataSource(QueryRunnerTestHelper.dataSource)
@@ -212,7 +207,13 @@ public class VarianceGroupByQueryTest
         )
         .build();
 
-    Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
+    List<ResultRow> expectedResults = expect
+        .add("2011-04-01", "automotive", 2L, 269L, 299.0009819048282, 17.29164485827847)
+        .add("2011-04-01", "mezzanine", 6L, 4420L, 254083.76447001836, 504.06722217380724)
+        .add("2011-04-01", "premium", 6L, 4416L, 252279.2020389339, 502.27403082275106)
+        .build(query);
+
+    Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
     TestHelper.assertExpectedObjects(expectedResults, results, "havingSpec");
 
     query = query.withLimitSpec(
@@ -228,7 +229,7 @@ public class VarianceGroupByQueryTest
     expectedResults = expect
         .add("2011-04-01", "automotive", 2L, 269L, 299.0009819048282, 17.29164485827847)
         .add("2011-04-01", "premium", 6L, 4416L, 252279.2020389339, 502.27403082275106)
-        .build();
+        .build(query);
 
     results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
     TestHelper.assertExpectedObjects(expectedResults, results, "limitSpec");
diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTestHelper.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTestHelper.java
index a46b6c3..e6e9f00 100644
--- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTestHelper.java
+++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTestHelper.java
@@ -20,7 +20,6 @@
 package org.apache.druid.query.aggregation.variance;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 import org.apache.druid.data.input.MapBasedRow;
 import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.DateTimes;
@@ -28,6 +27,8 @@ import org.apache.druid.query.QueryRunnerTestHelper;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.PostAggregator;
 import org.apache.druid.query.aggregation.stats.DruidStatsModule;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 import org.joda.time.DateTime;
 
 import java.util.ArrayList;
@@ -35,8 +36,10 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 
 /**
+ *
  */
 public class VarianceTestHelper extends QueryRunnerTestHelper
 {
@@ -83,17 +86,17 @@ public class VarianceTestHelper extends QueryRunnerTestHelper
       return this;
     }
 
-    public List<Row> build()
+    public List<ResultRow> build(final GroupByQuery query)
     {
       try {
-        return Lists.newArrayList(rows);
+        return rows.stream().map(row -> ResultRow.fromLegacyRow(row, query)).collect(Collectors.toList());
       }
       finally {
         rows.clear();
       }
     }
 
-    public Row build(final String timestamp, Object... values)
+    private Row build(final String timestamp, Object... values)
     {
       Preconditions.checkArgument(names.length == values.length);
 
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java
index c0c7c13..0dc3ee5 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java
@@ -46,12 +46,10 @@ import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.guava.CloseQuietly;
 import org.apache.druid.java.util.emitter.EmittingLogger;
-import org.apache.druid.query.FinalizeResultsQueryRunner;
+import org.apache.druid.query.NoopQueryRunner;
 import org.apache.druid.query.Query;
 import org.apache.druid.query.QueryRunner;
-import org.apache.druid.query.QueryRunnerFactory;
 import org.apache.druid.query.QueryRunnerFactoryConglomerate;
-import org.apache.druid.query.QueryToolChest;
 import org.apache.druid.segment.indexing.DataSchema;
 import org.apache.druid.segment.indexing.RealtimeIOConfig;
 import org.apache.druid.segment.indexing.RealtimeTuningConfig;
@@ -190,14 +188,12 @@ public class RealtimeIndexTask extends AbstractTask
   @Override
   public <T> QueryRunner<T> getQueryRunner(Query<T> query)
   {
-    if (plumber != null) {
-      QueryRunnerFactory<T, Query<T>> factory = queryRunnerFactoryConglomerate.findFactory(query);
-      QueryToolChest<T, Query<T>> toolChest = factory.getToolchest();
-
-      return new FinalizeResultsQueryRunner<T>(plumber.getQueryRunner(query), toolChest);
-    } else {
-      return null;
+    if (plumber == null) {
+      // Not yet initialized, no data yet, just return a noop runner.
+      return new NoopQueryRunner<>();
     }
+
+    return plumber.getQueryRunner(query);
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/query/BaseQuery.java b/processing/src/main/java/org/apache/druid/query/BaseQuery.java
index 1f4993c..cee1362 100644
--- a/processing/src/main/java/org/apache/druid/query/BaseQuery.java
+++ b/processing/src/main/java/org/apache/druid/query/BaseQuery.java
@@ -40,9 +40,10 @@ import java.util.Objects;
 import java.util.TreeMap;
 
 /**
+ *
  */
 @ExtensionPoint
-public abstract class BaseQuery<T extends Comparable<T>> implements Query<T>
+public abstract class BaseQuery<T> implements Query<T>
 {
   public static void checkInterrupted()
   {
@@ -216,10 +217,16 @@ public abstract class BaseQuery<T extends Comparable<T>> implements Query<T>
     return overridden;
   }
 
+  /**
+   * Default implementation of {@link Query#getResultOrdering()} that uses {@link Ordering#natural()}.
+   *
+   * If your query result type T is not Comparable, you must override this method.
+   */
   @Override
+  @SuppressWarnings("unchecked") // assumes T is Comparable; see method javadoc
   public Ordering<T> getResultOrdering()
   {
-    Ordering<T> retVal = Ordering.natural();
+    Ordering retVal = Ordering.natural();
     return descending ? retVal.reverse() : retVal;
   }
 
diff --git a/processing/src/main/java/org/apache/druid/query/BySegmentQueryRunner.java b/processing/src/main/java/org/apache/druid/query/BySegmentQueryRunner.java
index eced1f3..cd386c7 100644
--- a/processing/src/main/java/org/apache/druid/query/BySegmentQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/BySegmentQueryRunner.java
@@ -29,6 +29,14 @@ import java.util.Collections;
 import java.util.List;
 
 /**
+ * Query runner that wraps a base single-segment query runner, and wraps its results in a
+ * {@link BySegmentResultValueClass} object if the "bySegment" query context parameter is set. Otherwise, it
+ * delegates to the base runner without any behavior modification.
+ *
+ * Note that despite the type parameter "T", this runner may not actually return sequences with type T. They
+ * may really be of type {@code Result<BySegmentResultValue<T>>}, if "bySegment" is set. Downstream consumers
+ * of the returned sequence must be aware of this, and can use {@link QueryContexts#isBySegment(Query)} to
+ * know what to expect.
  */
 public class BySegmentQueryRunner<T> implements QueryRunner<T>
 {
diff --git a/processing/src/main/java/org/apache/druid/query/CacheStrategy.java b/processing/src/main/java/org/apache/druid/query/CacheStrategy.java
index f93a395..5d7d94f 100644
--- a/processing/src/main/java/org/apache/druid/query/CacheStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/CacheStrategy.java
@@ -22,13 +22,15 @@ package org.apache.druid.query;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.google.common.base.Function;
 import org.apache.druid.guice.annotations.ExtensionPoint;
+import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 
 import java.util.Iterator;
+import java.util.List;
 import java.util.concurrent.ExecutorService;
-import java.util.function.BiFunction;
 
 /**
+ *
  */
 @ExtensionPoint
 public interface CacheStrategy<T, CacheType, QueryType extends Query<T>>
@@ -113,19 +115,28 @@ public interface CacheStrategy<T, CacheType, QueryType extends Query<T>>
    * for dimension values (e.g., a Float would become Double).
    */
   static void fetchAggregatorsFromCache(
-      Iterator<AggregatorFactory> aggIter,
+      List<AggregatorFactory> aggregators,
       Iterator<Object> resultIter,
       boolean isResultLevelCache,
-      BiFunction<String, Object, Void> addToResultFunction
+      AddToResultFunction addToResultFunction
   )
   {
-    while (aggIter.hasNext() && resultIter.hasNext()) {
-      final AggregatorFactory factory = aggIter.next();
+    for (int i = 0; i < aggregators.size(); i++) {
+      final AggregatorFactory aggregator = aggregators.get(i);
+      if (!resultIter.hasNext()) {
+        throw new ISE("Ran out of objects while reading aggregators from cache!");
+      }
+
       if (isResultLevelCache) {
-        addToResultFunction.apply(factory.getName(), resultIter.next());
+        addToResultFunction.apply(aggregator.getName(), i, resultIter.next());
       } else {
-        addToResultFunction.apply(factory.getName(), factory.deserialize(resultIter.next()));
+        addToResultFunction.apply(aggregator.getName(), i, aggregator.deserialize(resultIter.next()));
       }
     }
   }
+
+  interface AddToResultFunction
+  {
+    void apply(String aggregatorName, int aggregatorIndex, Object object);
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/FinalizeResultsQueryRunner.java b/processing/src/main/java/org/apache/druid/query/FinalizeResultsQueryRunner.java
index ebb18f2..d63dcdd 100644
--- a/processing/src/main/java/org/apache/druid/query/FinalizeResultsQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/FinalizeResultsQueryRunner.java
@@ -30,6 +30,13 @@ import org.apache.druid.query.aggregation.MetricManipulatorFns;
 import org.apache.druid.query.context.ResponseContext;
 
 /**
+ * Query runner that applies {@link QueryToolChest#makePostComputeManipulatorFn(Query, MetricManipulationFn)} to the
+ * result stream. It is expected to be the last runner in the pipeline, after results are fully merged.
+ *
+ * Note that despite the type parameter "T", this runner may not actually return sequences with type T. This most
+ * commonly happens when an upstream {@link BySegmentQueryRunner} changes the result stream to type
+ * {@code Result<BySegmentResultValue<T>>}, in which case this class will retain the structure, but call the finalizer
+ * function on each result in the by-segment list (which may change their type from T to something else).
  */
 public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
 {
@@ -53,19 +60,19 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
     final boolean shouldFinalize = QueryContexts.isFinalize(query, true);
 
     final Query<T> queryToRun;
-    final Function<T, T> finalizerFn;
+    final Function<T, ?> finalizerFn;
     final MetricManipulationFn metricManipulationFn;
 
     if (shouldFinalize) {
       queryToRun = query.withOverriddenContext(ImmutableMap.of("finalize", false));
       metricManipulationFn = MetricManipulatorFns.finalizing();
-
     } else {
       queryToRun = query;
       metricManipulationFn = MetricManipulatorFns.identity();
     }
+
     if (isBySegment) {
-      finalizerFn = new Function<T, T>()
+      finalizerFn = new Function<T, Result<BySegmentResultValue<T>>>()
       {
         final Function<T, T> baseFinalizer = toolChest.makePostComputeManipulatorFn(
             query,
@@ -73,9 +80,9 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
         );
 
         @Override
-        @SuppressWarnings("unchecked")
-        public T apply(T input)
+        public Result<BySegmentResultValue<T>> apply(T input)
         {
+          //noinspection unchecked (input is not actually a T; see class-level javadoc)
           Result<BySegmentResultValueClass<T>> result = (Result<BySegmentResultValueClass<T>>) input;
 
           if (input == null) {
@@ -84,9 +91,9 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
 
           BySegmentResultValue<T> resultsClass = result.getValue();
 
-          return (T) new Result<>(
+          return new Result<>(
               result.getTimestamp(),
-              new BySegmentResultValueClass(
+              new BySegmentResultValueClass<>(
                   Lists.transform(resultsClass.getResults(), baseFinalizer),
                   resultsClass.getSegmentId(),
                   resultsClass.getInterval()
@@ -98,11 +105,10 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
       finalizerFn = toolChest.makePostComputeManipulatorFn(query, metricManipulationFn);
     }
 
-
-    return Sequences.map(
+    //noinspection unchecked (Technically unsound, but see class-level javadoc for rationale)
+    return (Sequence<T>) Sequences.map(
         baseRunner.run(queryPlus.withQuery(queryToRun), responseContext),
         finalizerFn
     );
-
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java b/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java
index 9c46d60..90e9f6f 100644
--- a/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java
@@ -85,9 +85,9 @@ public class GroupByMergedQueryRunner<T> implements QueryRunner<T>
     final boolean isSingleThreaded = querySpecificConfig.isSingleThreaded();
     final Pair<IncrementalIndex, Accumulator<IncrementalIndex, T>> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair(
         query,
+        null,
         querySpecificConfig,
-        bufferPool,
-        true
+        bufferPool
     );
     final Pair<Queue, Accumulator<Queue, T>> bySegmentAccumulatorPair = GroupByQueryHelper.createBySegmentAccumulatorPair();
     final boolean bySegment = QueryContexts.isBySegment(query);
diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java
index 36f88fe..9130309 100644
--- a/processing/src/main/java/org/apache/druid/query/Query.java
+++ b/processing/src/main/java/org/apache/druid/query/Query.java
@@ -35,6 +35,7 @@ import org.apache.druid.query.spec.QuerySegmentSpec;
 import org.apache.druid.query.timeboundary.TimeBoundaryQuery;
 import org.apache.druid.query.timeseries.TimeseriesQuery;
 import org.apache.druid.query.topn.TopNQuery;
+import org.apache.druid.segment.Segment;
 import org.joda.time.DateTimeZone;
 import org.joda.time.Duration;
 import org.joda.time.Interval;
@@ -42,6 +43,7 @@ import org.joda.time.Interval;
 import javax.annotation.Nullable;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ExecutorService;
 
 @ExtensionPoint
 @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "queryType")
@@ -99,6 +101,17 @@ public interface Query<T>
 
   boolean isDescending();
 
+  /**
+   * Comparator that represents the order in which results are generated from the
+   * {@link QueryRunnerFactory#createRunner(Segment)} and
+   * {@link QueryRunnerFactory#mergeRunners(ExecutorService, Iterable)} calls. This is used to combine streams of
+   * results from different sources; for example, it's used by historicals to combine streams from different segments,
+   * and it's used by the broker to combine streams from different historicals.
+   *
+   * Important note: sometimes, this ordering is used in a type-unsafe way to order @{code Result<BySegmentResultValue>}
+   * objects. Because of this, implementations should fall back to {@code Ordering.natural()} when they are given an
+   * object that is not of type T.
+   */
   Ordering<T> getResultOrdering();
 
   Query<T> withOverriddenContext(Map<String, Object> contextOverride);
diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java
index a8a20a6..4c760b0 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java
@@ -21,6 +21,7 @@ package org.apache.druid.query;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.JavaType;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.type.TypeFactory;
 import com.google.common.base.Function;
 import org.apache.druid.guice.annotations.ExtensionPoint;
@@ -76,6 +77,19 @@ public abstract class QueryToolChest<ResultType, QueryType extends Query<ResultT
   }
 
   /**
+   * Perform any per-query decoration of an {@link ObjectMapper} that enables it to read and write objects of the
+   * query's {@link ResultType}. It is used by QueryResource on the write side, and DirectDruidClient on the read side.
+   *
+   * For most queries, this is a no-op, but it can be useful for query types that support more than one result
+   * serialization format. Queries that implement this method must not modify the provided ObjectMapper, but instead
+   * must return a copy.
+   */
+  public ObjectMapper decorateObjectMapper(final ObjectMapper objectMapper, final QueryType query)
+  {
+    return objectMapper;
+  }
+
+  /**
    * This method wraps a QueryRunner.  The input QueryRunner, by contract, will provide a series of
    * ResultType objects in time order (ascending or descending).  This method should return a new QueryRunner that
    * potentially merges the stream of ordered ResultType objects.
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java
index 29ec5ec..0ba0f8d 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java
@@ -30,7 +30,11 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Ordering;
 import com.google.common.primitives.Longs;
-import org.apache.druid.data.input.Row;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
+import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.granularity.Granularities;
@@ -63,13 +67,13 @@ import org.apache.druid.segment.VirtualColumn;
 import org.apache.druid.segment.VirtualColumns;
 import org.apache.druid.segment.column.ColumnHolder;
 import org.apache.druid.segment.column.ValueType;
+import org.joda.time.DateTime;
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -78,15 +82,15 @@ import java.util.Set;
 import java.util.stream.Collectors;
 
 /**
+ *
  */
-public class GroupByQuery extends BaseQuery<Row>
+public class GroupByQuery extends BaseQuery<ResultRow>
 {
   public static final String CTX_KEY_SORT_BY_DIMS_FIRST = "sortByDimsFirst";
+  private static final String CTX_KEY_FUDGE_TIMESTAMP = "fudgeTimestamp";
 
-  private static final Comparator<Row> NON_GRANULAR_TIME_COMP = (Row lhs, Row rhs) -> Longs.compare(
-      lhs.getTimestampFromEpoch(),
-      rhs.getTimestampFromEpoch()
-  );
+  private static final Comparator<ResultRow> NON_GRANULAR_TIME_COMP =
+      (ResultRow lhs, ResultRow rhs) -> Longs.compare(lhs.getLong(0), rhs.getLong(0));
 
   public static Builder builder()
   {
@@ -106,14 +110,23 @@ public class GroupByQuery extends BaseQuery<Row>
   private final List<List<String>> subtotalsSpec;
 
   private final boolean applyLimitPushDown;
-  private final Function<Sequence<Row>, Sequence<Row>> postProcessingFn;
+  private final Function<Sequence<ResultRow>, Sequence<ResultRow>> postProcessingFn;
+  private final List<String> resultRowOrder;
+  private final Object2IntMap<String> resultRowPositionLookup;
+
+  /**
+   * This is set when we know that all rows will have the same timestamp, and allows us to not actually store
+   * and track it throughout the query execution process.
+   */
+  @Nullable
+  private final DateTime universalTimestamp;
 
   @JsonCreator
   public GroupByQuery(
       @JsonProperty("dataSource") DataSource dataSource,
       @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
       @JsonProperty("virtualColumns") VirtualColumns virtualColumns,
-      @JsonProperty("filter") DimFilter dimFilter,
+      @JsonProperty("filter") @Nullable DimFilter dimFilter,
       @JsonProperty("granularity") Granularity granularity,
       @JsonProperty("dimensions") List<DimensionSpec> dimensions,
       @JsonProperty("aggregations") List<AggregatorFactory> aggregatorSpecs,
@@ -141,22 +154,15 @@ public class GroupByQuery extends BaseQuery<Row>
     );
   }
 
-  private Function<Sequence<Row>, Sequence<Row>> makePostProcessingFn()
+  private Function<Sequence<ResultRow>, Sequence<ResultRow>> makePostProcessingFn()
   {
-    Function<Sequence<Row>, Sequence<Row>> postProcessingFn = limitSpec.build(
-        dimensions,
-        aggregatorSpecs,
-        postAggregatorSpecs,
-        getGranularity(),
-        getContextSortByDimsFirst()
-    );
+    Function<Sequence<ResultRow>, Sequence<ResultRow>> postProcessingFn = limitSpec.build(this);
 
     if (havingSpec != null) {
       postProcessingFn = Functions.compose(
           postProcessingFn,
-          (Sequence<Row> input) -> {
-            havingSpec.setRowSignature(GroupByQueryHelper.rowSignatureFor(GroupByQuery.this));
-            havingSpec.setAggregators(getAggregatorsMap(aggregatorSpecs));
+          (Sequence<ResultRow> input) -> {
+            havingSpec.setQuery(this);
             return Sequences.filter(input, havingSpec::eval);
           }
       );
@@ -179,7 +185,7 @@ public class GroupByQuery extends BaseQuery<Row>
       final @Nullable HavingSpec havingSpec,
       final LimitSpec limitSpec,
       final @Nullable List<List<String>> subtotalsSpec,
-      final @Nullable Function<Sequence<Row>, Sequence<Row>> postProcessingFn,
+      final @Nullable Function<Sequence<ResultRow>, Sequence<ResultRow>> postProcessingFn,
       final Map<String, Object> context
   )
   {
@@ -198,9 +204,12 @@ public class GroupByQuery extends BaseQuery<Row>
         this.aggregatorSpecs,
         postAggregatorSpecs == null ? ImmutableList.of() : postAggregatorSpecs
     );
+
+    this.universalTimestamp = computeUniversalTimestamp();
+    this.resultRowOrder = computeResultRowOrder();
+    this.resultRowPositionLookup = computeResultRowOrderLookup();
     this.havingSpec = havingSpec;
     this.limitSpec = LimitSpec.nullToNoopLimitSpec(limitSpec);
-
     this.subtotalsSpec = verifySubtotalsSpec(subtotalsSpec, this.dimensions);
 
     // Verify no duplicate names between dimensions, aggregators, and postAggregators.
@@ -297,11 +306,99 @@ public class GroupByQuery extends BaseQuery<Row>
 
   @JsonInclude(JsonInclude.Include.NON_NULL)
   @JsonProperty("subtotalsSpec")
+  @Nullable
   public List<List<String>> getSubtotalsSpec()
   {
     return subtotalsSpec;
   }
 
+  /**
+   * Returns a list of field names, of the same size as {@link #getResultRowSizeWithPostAggregators()}, in the
+   * order that they will appear in ResultRows for this query.
+   *
+   * @see ResultRow for documentation about the order that fields will be in
+   */
+  public List<String> getResultRowOrder()
+  {
+    return resultRowOrder;
+  }
+
+  /**
+   * Returns the size of ResultRows for this query when they do not include post-aggregators.
+   */
+  public int getResultRowSizeWithoutPostAggregators()
+  {
+    return getResultRowPostAggregatorStart();
+  }
+
+  /**
+   * Returns the size of ResultRows for this query when they include post-aggregators.
+   */
+  public int getResultRowSizeWithPostAggregators()
+  {
+    return resultRowOrder.size();
+  }
+
+  /**
+   * Returns a map that can be used to look up the position within ResultRows of certain field names. The map's
+   * {@link Object2IntMap#getInt(Object)} method will return -1 if the field is not found.
+   */
+  public Object2IntMap<String> getResultRowPositionLookup()
+  {
+    return resultRowPositionLookup;
+  }
+
+  /**
+   * If this query has a single universal timestamp, return it. Otherwise return null.
+   *
+   * This method will return a nonnull timestamp in the following two cases:
+   *
+   * 1) CTX_KEY_FUDGE_TIMESTAMP is set (in which case this timestamp will be returned).
+   * 2) Granularity is "ALL".
+   *
+   * If this method returns null, then {@link #getResultRowHasTimestamp()} will return true. The reverse is also true:
+   * if this method returns nonnull, then {@link #getResultRowHasTimestamp()} will return false.
+   */
+  @Nullable
+  public DateTime getUniversalTimestamp()
+  {
+    return universalTimestamp;
+  }
+
+  /**
+   * Returns true if ResultRows for this query include timestamps, false otherwise.
+   *
+   * @see #getUniversalTimestamp() for details about when timestamps are included in ResultRows
+   */
+  public boolean getResultRowHasTimestamp()
+  {
+    return universalTimestamp == null;
+  }
+
+  /**
+   * Returns the position of the first dimension in ResultRows for this query.
+   */
+  public int getResultRowDimensionStart()
+  {
+    return getResultRowHasTimestamp() ? 1 : 0;
+  }
+
+  /**
+   * Returns the position of the first aggregator in ResultRows for this query.
+   */
+  public int getResultRowAggregatorStart()
+  {
+    return getResultRowDimensionStart() + dimensions.size();
+  }
+
+  /**
+   * Returns the position of the first post-aggregator in ResultRows for this query.
+   */
+  public int getResultRowPostAggregatorStart()
+  {
+    return getResultRowAggregatorStart() + aggregatorSpecs.size();
+  }
+
   @Override
   public boolean hasFilters()
   {
@@ -309,6 +406,7 @@ public class GroupByQuery extends BaseQuery<Row>
   }
 
   @Override
+  @Nullable
   public DimFilter getFilter()
   {
     return dimFilter;
@@ -341,14 +439,14 @@ public class GroupByQuery extends BaseQuery<Row>
   @Override
   public Ordering getResultOrdering()
   {
-    final Ordering<Row> rowOrdering = getRowOrdering(false);
+    final Ordering<ResultRow> rowOrdering = getRowOrdering(false);
 
     return Ordering.from(
         (lhs, rhs) -> {
-          if (lhs instanceof Row) {
-            return rowOrdering.compare((Row) lhs, (Row) rhs);
+          if (lhs instanceof ResultRow) {
+            return rowOrdering.compare((ResultRow) lhs, (ResultRow) rhs);
           } else {
-            // Probably bySegment queries
+            //noinspection unchecked (Probably bySegment queries; see BySegmentQueryRunner for details)
             return ((Ordering) Comparators.naturalNullsFirst()).compare(lhs, rhs);
           }
         }
@@ -380,7 +478,35 @@ public class GroupByQuery extends BaseQuery<Row>
     return forcePushDown;
   }
 
-  public boolean determineApplyLimitPushDown()
+  private Object2IntMap<String> computeResultRowOrderLookup()
+  {
+    final Object2IntMap<String> indexes = new Object2IntOpenHashMap<>();
+    indexes.defaultReturnValue(-1);
+
+    int index = 0;
+    for (String columnName : resultRowOrder) {
+      indexes.put(columnName, index++);
+    }
+
+    return indexes;
+  }
+
+  private List<String> computeResultRowOrder()
+  {
+    final List<String> retVal = new ArrayList<>();
+
+    if (universalTimestamp == null) {
+      retVal.add(ColumnHolder.TIME_COLUMN_NAME);
+    }
+
+    dimensions.stream().map(DimensionSpec::getOutputName).forEach(retVal::add);
+    aggregatorSpecs.stream().map(AggregatorFactory::getName).forEach(retVal::add);
+    postAggregatorSpecs.stream().map(PostAggregator::getName).forEach(retVal::add);
+
+    return retVal;
+  }
+
+  private boolean determineApplyLimitPushDown()
   {
     if (subtotalsSpec != null) {
       return false;
@@ -426,14 +552,14 @@ public class GroupByQuery extends BaseQuery<Row>
    * limit/order spec (unlike non-push down case where the results always use the default natural ascending order),
    * so when merging these partial result streams, the merge needs to use the same ordering to get correct results.
    */
-  private Ordering<Row> getRowOrderingForPushDown(
+  private Ordering<ResultRow> getRowOrderingForPushDown(
       final boolean granular,
       final DefaultLimitSpec limitSpec
   )
   {
     final boolean sortByDimsFirst = getContextSortByDimsFirst();
 
-    final List<String> orderedFieldNames = new ArrayList<>();
+    final IntList orderedFieldNumbers = new IntArrayList();
     final Set<Integer> dimsInOrderBy = new HashSet<>();
     final List<Boolean> needsReverseList = new ArrayList<>();
     final List<ValueType> dimensionTypes = new ArrayList<>();
@@ -444,7 +570,7 @@ public class GroupByQuery extends BaseQuery<Row>
       int dimIndex = OrderByColumnSpec.getDimIndexForOrderBy(orderSpec, dimensions);
       if (dimIndex >= 0) {
         DimensionSpec dim = dimensions.get(dimIndex);
-        orderedFieldNames.add(dim.getOutputName());
+        orderedFieldNumbers.add(resultRowPositionLookup.getInt(dim.getOutputName()));
         dimsInOrderBy.add(dimIndex);
         needsReverseList.add(needsReverse);
         final ValueType type = dimensions.get(dimIndex).getOutputType();
@@ -455,7 +581,7 @@ public class GroupByQuery extends BaseQuery<Row>
 
     for (int i = 0; i < dimensions.size(); i++) {
       if (!dimsInOrderBy.contains(i)) {
-        orderedFieldNames.add(dimensions.get(i).getOutputName());
+        orderedFieldNumbers.add(resultRowPositionLookup.getInt(dimensions.get(i).getOutputName()));
         needsReverseList.add(false);
         final ValueType type = dimensions.get(i).getOutputType();
         dimensionTypes.add(type);
@@ -463,77 +589,60 @@ public class GroupByQuery extends BaseQuery<Row>
       }
     }
 
-    final Comparator<Row> timeComparator = getTimeComparator(granular);
+    final Comparator<ResultRow> timeComparator = getTimeComparator(granular);
 
     if (timeComparator == null) {
       return Ordering.from(
-          new Comparator<Row>()
-          {
-            @Override
-            public int compare(Row lhs, Row rhs)
-            {
-              return compareDimsForLimitPushDown(
-                  orderedFieldNames,
-                  needsReverseList,
-                  dimensionTypes,
-                  comparators,
-                  lhs,
-                  rhs
-              );
-            }
-          }
+          (lhs, rhs) -> compareDimsForLimitPushDown(
+              orderedFieldNumbers,
+              needsReverseList,
+              dimensionTypes,
+              comparators,
+              lhs,
+              rhs
+          )
       );
     } else if (sortByDimsFirst) {
       return Ordering.from(
-          new Comparator<Row>()
-          {
-            @Override
-            public int compare(Row lhs, Row rhs)
-            {
-              final int cmp = compareDimsForLimitPushDown(
-                  orderedFieldNames,
-                  needsReverseList,
-                  dimensionTypes,
-                  comparators,
-                  lhs,
-                  rhs
-              );
-              if (cmp != 0) {
-                return cmp;
-              }
-
-              return timeComparator.compare(lhs, rhs);
+          (lhs, rhs) -> {
+            final int cmp = compareDimsForLimitPushDown(
+                orderedFieldNumbers,
+                needsReverseList,
+                dimensionTypes,
+                comparators,
+                lhs,
+                rhs
+            );
+            if (cmp != 0) {
+              return cmp;
             }
+
+            return timeComparator.compare(lhs, rhs);
           }
       );
     } else {
       return Ordering.from(
-          new Comparator<Row>()
-          {
-            @Override
-            public int compare(Row lhs, Row rhs)
-            {
-              final int timeCompare = timeComparator.compare(lhs, rhs);
-
-              if (timeCompare != 0) {
-                return timeCompare;
-              }
-
-              return compareDimsForLimitPushDown(
-                  orderedFieldNames,
-                  needsReverseList,
-                  dimensionTypes,
-                  comparators,
-                  lhs,
-                  rhs
-              );
+          (lhs, rhs) -> {
+            final int timeCompare = timeComparator.compare(lhs, rhs);
+
+            if (timeCompare != 0) {
+              return timeCompare;
             }
+
+            return compareDimsForLimitPushDown(
+                orderedFieldNumbers,
+                needsReverseList,
+                dimensionTypes,
+                comparators,
+                lhs,
+                rhs
+            );
           }
       );
     }
   }
 
-  public Ordering<Row> getRowOrdering(final boolean granular)
+  public Ordering<ResultRow> getRowOrdering(final boolean granular)
   {
     if (applyLimitPushDown) {
       if (!DefaultLimitSpec.sortingOrderHasNonGroupingFields((DefaultLimitSpec) limitSpec, dimensions)) {
@@ -542,7 +651,7 @@ public class GroupByQuery extends BaseQuery<Row>
     }
 
     final boolean sortByDimsFirst = getContextSortByDimsFirst();
-    final Comparator<Row> timeComparator = getTimeComparator(granular);
+    final Comparator<ResultRow> timeComparator = getTimeComparator(granular);
 
     if (timeComparator == null) {
       return Ordering.from((lhs, rhs) -> compareDims(dimensions, lhs, rhs));
@@ -573,27 +682,36 @@ public class GroupByQuery extends BaseQuery<Row>
   }
 
   @Nullable
-  private Comparator<Row> getTimeComparator(boolean granular)
+  private Comparator<ResultRow> getTimeComparator(boolean granular)
   {
     if (Granularities.ALL.equals(getGranularity())) {
       return null;
-    } else if (granular) {
-      return (lhs, rhs) -> Longs.compare(
-          getGranularity().bucketStart(lhs.getTimestamp()).getMillis(),
-          getGranularity().bucketStart(rhs.getTimestamp()).getMillis()
-      );
     } else {
-      return NON_GRANULAR_TIME_COMP;
+      if (!getResultRowHasTimestamp()) {
+        // Sanity check (should never happen).
+        throw new ISE("Cannot do time comparisons!");
+      }
+
+      if (granular) {
+        return (lhs, rhs) -> Longs.compare(
+            getGranularity().bucketStart(DateTimes.utc(lhs.getLong(0))).getMillis(),
+            getGranularity().bucketStart(DateTimes.utc(rhs.getLong(0))).getMillis()
+        );
+      } else {
+        return NON_GRANULAR_TIME_COMP;
+      }
     }
   }
 
-  private static int compareDims(List<DimensionSpec> dimensions, Row lhs, Row rhs)
+  private int compareDims(List<DimensionSpec> dimensions, ResultRow lhs, ResultRow rhs)
   {
-    for (DimensionSpec dimension : dimensions) {
-      //noinspection unchecked
+    final int dimensionStart = getResultRowDimensionStart();
+
+    for (int i = 0; i < dimensions.size(); i++) {
+      DimensionSpec dimension = dimensions.get(i);
       final int dimCompare = DimensionHandlerUtils.compareObjectsAsType(
-          lhs.getRaw(dimension.getOutputName()),
-          rhs.getRaw(dimension.getOutputName()),
+          lhs.get(dimensionStart + i),
+          rhs.get(dimensionStart + i),
           dimension.getOutputType()
       );
       if (dimCompare != 0) {
@@ -604,23 +722,42 @@ public class GroupByQuery extends BaseQuery<Row>
     return 0;
   }
 
+  /**
+   * Computes the timestamp that will be returned by {@link #getUniversalTimestamp()}.
+   */
+  @Nullable
+  private DateTime computeUniversalTimestamp()
+  {
+    final String timestampStringFromContext = getContextValue(CTX_KEY_FUDGE_TIMESTAMP, "");
+    final Granularity granularity = getGranularity();
+
+    if (!timestampStringFromContext.isEmpty()) {
+      return DateTimes.utc(Long.parseLong(timestampStringFromContext));
+    } else if (Granularities.ALL.equals(granularity)) {
+      final DateTime timeStart = getIntervals().get(0).getStart();
+      return granularity.getIterable(new Interval(timeStart, timeStart.plus(1))).iterator().next().getStart();
+    } else {
+      return null;
+    }
+  }
+
   private static int compareDimsForLimitPushDown(
-      final List<String> fields,
+      final IntList fields,
       final List<Boolean> needsReverseList,
       final List<ValueType> dimensionTypes,
       final List<StringComparator> comparators,
-      Row lhs,
-      Row rhs
+      final ResultRow lhs,
+      final ResultRow rhs
   )
   {
     for (int i = 0; i < fields.size(); i++) {
-      final String fieldName = fields.get(i);
+      final int fieldNumber = fields.getInt(i);
       final StringComparator comparator = comparators.get(i);
       final ValueType dimensionType = dimensionTypes.get(i);
 
       final int dimCompare;
-      final Object lhsObj = lhs.getRaw(fieldName);
-      final Object rhsObj = rhs.getRaw(fieldName);
+      final Object lhsObj = lhs.get(fieldNumber);
+      final Object rhsObj = rhs.get(fieldNumber);
 
       if (ValueType.isNumeric(dimensionType)) {
         if (comparator.equals(StringComparators.NUMERIC)) {
@@ -648,7 +785,7 @@ public class GroupByQuery extends BaseQuery<Row>
    *
    * @return sequence of rows after applying havingSpec and limitSpec
    */
-  public Sequence<Row> postProcess(Sequence<Row> results)
+  public Sequence<ResultRow> postProcess(Sequence<ResultRow> results)
   {
     return postProcessingFn.apply(results);
   }
@@ -671,7 +808,7 @@ public class GroupByQuery extends BaseQuery<Row>
   }
 
   @Override
-  public Query<Row> withDataSource(DataSource dataSource)
+  public Query<ResultRow> withDataSource(DataSource dataSource)
   {
     return new Builder(this).setDataSource(dataSource).build();
   }
@@ -734,13 +871,6 @@ public class GroupByQuery extends BaseQuery<Row>
     }
   }
 
-  private static Map<String, AggregatorFactory> getAggregatorsMap(List<AggregatorFactory> aggregatorSpecs)
-  {
-    Map<String, AggregatorFactory> map = new HashMap<>(aggregatorSpecs.size());
-    aggregatorSpecs.forEach(v -> map.put(v.getName(), v));
-    return map;
-  }
-
   public static class Builder
   {
     @Nullable
@@ -774,7 +904,7 @@ public class GroupByQuery extends BaseQuery<Row>
     @Nullable
     private LimitSpec limitSpec = null;
     @Nullable
-    private Function<Sequence<Row>, Sequence<Row>> postProcessingFn;
+    private Function<Sequence<ResultRow>, Sequence<ResultRow>> postProcessingFn;
     private List<OrderByColumnSpec> orderByColumnSpecs = new ArrayList<>();
     private int limit = Integer.MAX_VALUE;
 
@@ -1020,7 +1150,7 @@ public class GroupByQuery extends BaseQuery<Row>
       return this;
     }
 
-    public Builder setHavingSpec(HavingSpec havingSpec)
+    public Builder setHavingSpec(@Nullable HavingSpec havingSpec)
     {
       this.havingSpec = havingSpec;
       this.postProcessingFn = null;
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java
index 13107fb..b272ab1 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java
@@ -31,6 +31,7 @@ public class GroupByQueryConfig
   public static final String CTX_KEY_APPLY_LIMIT_PUSH_DOWN = "applyLimitPushDown";
   public static final String CTX_KEY_FORCE_PUSH_DOWN_NESTED_QUERY = "forcePushDownNestedQuery";
   public static final String CTX_KEY_EXECUTING_NESTED_QUERY = "executingNestedQuery";
+  public static final String CTX_KEY_ARRAY_RESULT_ROWS = "resultAsArray";
   private static final String CTX_KEY_IS_SINGLE_THREADED = "groupByIsSingleThreaded";
   private static final String CTX_KEY_MAX_INTERMEDIATE_ROWS = "maxIntermediateRows";
   private static final String CTX_KEY_MAX_RESULTS = "maxResults";
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java
index 10f582c..28147fb 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java
@@ -40,6 +40,7 @@ import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.query.ResourceLimitExceededException;
 import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.PostAggregator;
 import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.segment.column.ValueType;
 import org.apache.druid.segment.incremental.IncrementalIndex;
@@ -47,6 +48,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema;
 import org.apache.druid.segment.incremental.IndexSizeExceededException;
 import org.joda.time.DateTime;
 
+import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
@@ -60,14 +62,15 @@ public class GroupByQueryHelper
 
   public static <T> Pair<IncrementalIndex, Accumulator<IncrementalIndex, T>> createIndexAccumulatorPair(
       final GroupByQuery query,
+      @Nullable final GroupByQuery subquery,
       final GroupByQueryConfig config,
-      NonBlockingPool<ByteBuffer> bufferPool,
-      final boolean combine
+      NonBlockingPool<ByteBuffer> bufferPool
   )
   {
     final GroupByQueryConfig querySpecificConfig = config.withOverrides(query);
     final Granularity gran = query.getGranularity();
     final DateTime timeStart = query.getIntervals().get(0).getStart();
+    final boolean combine = subquery == null;
 
     DateTime granTimeStart = timeStart;
     if (!(Granularities.ALL.equals(gran))) {
@@ -142,25 +145,30 @@ public class GroupByQueryHelper
       @Override
       public IncrementalIndex accumulate(IncrementalIndex accumulated, T in)
       {
+        final MapBasedRow mapBasedRow;
 
         if (in instanceof MapBasedRow) {
-          try {
-            MapBasedRow row = (MapBasedRow) in;
-            accumulated.add(
-                new MapBasedInputRow(
-                    row.getTimestamp(),
-                    dimensions,
-                    row.getEvent()
-                )
-            );
-          }
-          catch (IndexSizeExceededException e) {
-            throw new ResourceLimitExceededException(e.getMessage());
-          }
+          mapBasedRow = (MapBasedRow) in;
+        } else if (in instanceof ResultRow) {
+          final ResultRow row = (ResultRow) in;
+          mapBasedRow = row.toMapBasedRow(combine ? query : subquery);
         } else {
           throw new ISE("Unable to accumulate something of type [%s]", in.getClass());
         }
 
+        try {
+          accumulated.add(
+              new MapBasedInputRow(
+                  mapBasedRow.getTimestamp(),
+                  dimensions,
+                  mapBasedRow.getEvent()
+              )
+          );
+        }
+        catch (IndexSizeExceededException e) {
+          throw new ResourceLimitExceededException(e.getMessage());
+        }
+
         return accumulated;
       }
     };
@@ -189,39 +197,31 @@ public class GroupByQueryHelper
   // Used by GroupByStrategyV1
   public static IncrementalIndex makeIncrementalIndex(
       GroupByQuery query,
+      @Nullable GroupByQuery subquery,
       GroupByQueryConfig config,
       NonBlockingPool<ByteBuffer> bufferPool,
-      Sequence<Row> rows,
-      boolean combine
+      Sequence<ResultRow> rows
   )
   {
-    Pair<IncrementalIndex, Accumulator<IncrementalIndex, Row>> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair(
-        query,
-        config,
-        bufferPool,
-        combine
-    );
+    final Pair<IncrementalIndex, Accumulator<IncrementalIndex, ResultRow>> indexAccumulatorPair =
+        GroupByQueryHelper.createIndexAccumulatorPair(query, subquery, config, bufferPool);
 
     return rows.accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
   }
 
   // Used by GroupByStrategyV1
-  public static Sequence<Row> postAggregate(final GroupByQuery query, IncrementalIndex index)
+  public static Sequence<ResultRow> postAggregate(final GroupByQuery query, IncrementalIndex<?> index)
   {
     return Sequences.map(
         Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs(), query.isDescending())),
-        new Function<Row, Row>()
-        {
-          @Override
-          public Row apply(Row input)
-          {
-            final MapBasedRow row = (MapBasedRow) input;
-            return new MapBasedRow(
-                query.getGranularity()
-                     .toDateTime(row.getTimestampFromEpoch()),
-                row.getEvent()
-            );
+        input -> {
+          final ResultRow resultRow = toResultRow(query, input);
+
+          if (query.getResultRowHasTimestamp()) {
+            resultRow.set(0, query.getGranularity().toDateTime(resultRow.getLong(0)).getMillis());
           }
+
+          return resultRow;
         }
     );
   }
@@ -258,4 +258,28 @@ public class GroupByQueryHelper
     // Don't include post-aggregators since we don't know what types they are.
     return types.build();
   }
+
+  public static ResultRow toResultRow(final GroupByQuery query, final Row row)
+  {
+    final ResultRow resultRow = ResultRow.create(query.getResultRowSizeWithPostAggregators());
+    int i = 0;
+
+    if (query.getResultRowHasTimestamp()) {
+      resultRow.set(i++, row.getTimestampFromEpoch());
+    }
+
+    for (DimensionSpec dimensionSpec : query.getDimensions()) {
+      resultRow.set(i++, row.getRaw(dimensionSpec.getOutputName()));
+    }
+
+    for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) {
+      resultRow.set(i++, row.getRaw(aggregatorFactory.getName()));
+    }
+
+    for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) {
+      resultRow.set(i++, row.getRaw(postAggregator.getName()));
+    }
+
+    return resultRow;
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
index 1b58b20..b942660 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
@@ -19,19 +19,21 @@
 
 package org.apache.druid.query.groupby;
 
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonDeserializer;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.module.SimpleModule;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Functions;
-import com.google.common.base.Predicate;
-import com.google.common.collect.Collections2;
 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.common.collect.Maps;
 import com.google.inject.Inject;
-import org.apache.druid.data.input.MapBasedRow;
 import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.granularity.Granularity;
@@ -63,28 +65,27 @@ import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
 import org.apache.druid.segment.DimensionHandlerUtils;
 import org.joda.time.DateTime;
 
-import javax.annotation.Nullable;
+import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
+import java.util.BitSet;
 import java.util.Comparator;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.TreeMap;
 import java.util.function.BinaryOperator;
 
 /**
+ *
  */
-public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery>
+public class GroupByQueryQueryToolChest extends QueryToolChest<ResultRow, GroupByQuery>
 {
   private static final byte GROUPBY_QUERY = 0x14;
   private static final TypeReference<Object> OBJECT_TYPE_REFERENCE =
       new TypeReference<Object>()
       {
       };
-  private static final TypeReference<Row> TYPE_REFERENCE = new TypeReference<Row>()
+  private static final TypeReference<ResultRow> TYPE_REFERENCE = new TypeReference<ResultRow>()
   {
   };
   public static final String GROUP_BY_MERGE_KEY = "groupByMerge";
@@ -116,7 +117,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
   }
 
   @Override
-  public QueryRunner<Row> mergeResults(final QueryRunner<Row> runner)
+  public QueryRunner<ResultRow> mergeResults(final QueryRunner<ResultRow> runner)
   {
     return (queryPlus, responseContext) -> {
       if (QueryContexts.isBySegment(queryPlus.getQuery())) {
@@ -132,34 +133,48 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
   }
 
   @Override
-  public BinaryOperator<Row> createMergeFn(Query<Row> query)
+  public BinaryOperator<ResultRow> createMergeFn(Query<ResultRow> query)
   {
     return strategySelector.strategize((GroupByQuery) query).createMergeFn(query);
   }
 
   @Override
-  public Comparator<Row> createResultComparator(Query<Row> query)
+  public Comparator<ResultRow> createResultComparator(Query<ResultRow> query)
   {
     return strategySelector.strategize((GroupByQuery) query).createResultComparator(query);
   }
 
-  private Sequence<Row> initAndMergeGroupByResults(
+  private Sequence<ResultRow> initAndMergeGroupByResults(
       final GroupByQuery query,
-      QueryRunner<Row> runner,
+      QueryRunner<ResultRow> runner,
       ResponseContext context
   )
   {
     final GroupByStrategy groupByStrategy = strategySelector.strategize(query);
-    final GroupByQueryResource resource = groupByStrategy.prepareResource(query, false);
+    final GroupByQueryResource resource = groupByStrategy.prepareResource(query);
+    try {
+      final Sequence<ResultRow> mergedSequence = mergeGroupByResults(
+          groupByStrategy,
+          query,
+          resource,
+          runner,
+          context
+      );
 
-    return Sequences.withBaggage(mergeGroupByResults(groupByStrategy, query, resource, runner, context), resource);
+      return Sequences.withBaggage(mergedSequence, resource);
+    }
+    catch (Exception e) {
+      // Error creating the Sequence; release resources.
+      resource.close();
+      throw e;
+    }
   }
 
-  private Sequence<Row> mergeGroupByResults(
+  private Sequence<ResultRow> mergeGroupByResults(
       GroupByStrategy groupByStrategy,
       final GroupByQuery query,
       GroupByQueryResource resource,
-      QueryRunner<Row> runner,
+      QueryRunner<ResultRow> runner,
       ResponseContext context
   )
   {
@@ -169,11 +184,11 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
     return mergeGroupByResultsWithoutPushDown(groupByStrategy, query, resource, runner, context);
   }
 
-  private Sequence<Row> mergeGroupByResultsWithoutPushDown(
+  private Sequence<ResultRow> mergeGroupByResultsWithoutPushDown(
       GroupByStrategy groupByStrategy,
       GroupByQuery query,
       GroupByQueryResource resource,
-      QueryRunner<Row> runner,
+      QueryRunner<ResultRow> runner,
       ResponseContext context
   )
   {
@@ -204,7 +219,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
         throw new UnsupportedOperationException("Subqueries must be of type 'group by'");
       }
 
-      final Sequence<Row> subqueryResult = mergeGroupByResults(
+      final Sequence<ResultRow> subqueryResult = mergeGroupByResults(
           groupByStrategy,
           subquery.withOverriddenContext(
               ImmutableMap.of(
@@ -219,7 +234,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
           context
       );
 
-      final Sequence<Row> finalizingResults = finalizeSubqueryResults(subqueryResult, subquery);
+      final Sequence<ResultRow> finalizingResults = finalizeSubqueryResults(subqueryResult, subquery);
 
       if (query.getSubtotalsSpec() != null) {
         return groupByStrategy.processSubtotalsSpec(
@@ -250,16 +265,16 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
     }
   }
 
-  private Sequence<Row> mergeResultsWithNestedQueryPushDown(
+  private Sequence<ResultRow> mergeResultsWithNestedQueryPushDown(
       GroupByStrategy groupByStrategy,
       GroupByQuery query,
       GroupByQueryResource resource,
-      QueryRunner<Row> runner,
+      QueryRunner<ResultRow> runner,
       ResponseContext context
   )
   {
-    Sequence<Row> pushDownQueryResults = groupByStrategy.mergeResults(runner, query, context);
-    final Sequence<Row> finalizedResults = finalizeSubqueryResults(pushDownQueryResults, query);
+    Sequence<ResultRow> pushDownQueryResults = groupByStrategy.mergeResults(runner, query, context);
+    final Sequence<ResultRow> finalizedResults = finalizeSubqueryResults(pushDownQueryResults, query);
     GroupByQuery rewrittenQuery = rewriteNestedQueryForPushDown(query);
     return groupByStrategy.applyPostProcessing(groupByStrategy.processSubqueryResult(
         query,
@@ -288,9 +303,9 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
                 ));
   }
 
-  private Sequence<Row> finalizeSubqueryResults(Sequence<Row> subqueryResult, GroupByQuery subquery)
+  private Sequence<ResultRow> finalizeSubqueryResults(Sequence<ResultRow> subqueryResult, GroupByQuery subquery)
   {
-    final Sequence<Row> finalizingResults;
+    final Sequence<ResultRow> finalizingResults;
     if (QueryContexts.isFinalize(subquery, false)) {
       finalizingResults = new MappedSequence<>(
           subqueryResult,
@@ -322,7 +337,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
   }
 
   @Override
-  public Function<Row, Row> makePreComputeManipulatorFn(
+  public Function<ResultRow, ResultRow> makePreComputeManipulatorFn(
       final GroupByQuery query,
       final MetricManipulationFn fn
   )
@@ -331,116 +346,149 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
       return Functions.identity();
     }
 
-    return new Function<Row, Row>()
-    {
-      @Override
-      public Row apply(Row input)
-      {
-        if (input instanceof MapBasedRow) {
-          final MapBasedRow inputRow = (MapBasedRow) input;
-          final Map<String, Object> values = new HashMap<>(inputRow.getEvent());
-          for (AggregatorFactory agg : query.getAggregatorSpecs()) {
-            values.put(agg.getName(), fn.manipulate(agg, inputRow.getEvent().get(agg.getName())));
-          }
-          return new MapBasedRow(inputRow.getTimestamp(), values);
-        }
-        return input;
+    return row -> {
+      final ResultRow newRow = row.copy();
+      final List<AggregatorFactory> aggregatorSpecs = query.getAggregatorSpecs();
+      final int aggregatorStart = query.getResultRowAggregatorStart();
+
+      for (int i = 0; i < aggregatorSpecs.size(); i++) {
+        AggregatorFactory agg = aggregatorSpecs.get(i);
+        newRow.set(aggregatorStart + i, fn.manipulate(agg, row.get(aggregatorStart + i)));
       }
+
+      return newRow;
     };
   }
 
   @Override
-  public Function<Row, Row> makePostComputeManipulatorFn(
+  public Function<ResultRow, ResultRow> makePostComputeManipulatorFn(
       final GroupByQuery query,
       final MetricManipulationFn fn
   )
   {
-    final Set<String> optimizedDims = ImmutableSet.copyOf(
-        Iterables.transform(
-            extractionsToRewrite(query),
-            new Function<DimensionSpec, String>()
-            {
-              @Override
-              public String apply(DimensionSpec input)
-              {
-                return input.getOutputName();
-              }
-            }
-        )
-    );
-    final Function<Row, Row> preCompute = makePreComputeManipulatorFn(query, fn);
+    final BitSet optimizedDims = extractionsToRewrite(query);
+    final Function<ResultRow, ResultRow> preCompute = makePreComputeManipulatorFn(query, fn);
+
     if (optimizedDims.isEmpty()) {
       return preCompute;
     }
 
     // If we have optimizations that can be done at this level, we apply them here
 
-    final Map<String, ExtractionFn> extractionFnMap = new HashMap<>();
-    for (DimensionSpec dimensionSpec : query.getDimensions()) {
-      final String dimension = dimensionSpec.getOutputName();
-      if (optimizedDims.contains(dimension)) {
-        extractionFnMap.put(dimension, dimensionSpec.getExtractionFn());
+    final List<DimensionSpec> dimensions = query.getDimensions();
+    final List<ExtractionFn> extractionFns = new ArrayList<>(dimensions.size());
+    for (int i = 0; i < dimensions.size(); i++) {
+      final DimensionSpec dimensionSpec = dimensions.get(i);
+      final ExtractionFn extractionFnToAdd;
+
+      if (optimizedDims.get(i)) {
+        extractionFnToAdd = dimensionSpec.getExtractionFn();
+      } else {
+        extractionFnToAdd = null;
       }
+
+      extractionFns.add(extractionFnToAdd);
     }
 
-    return new Function<Row, Row>()
+    final int dimensionStart = query.getResultRowDimensionStart();
+    return row -> {
+      // preCompute.apply(row) will either return the original row, or create a copy.
+      ResultRow newRow = preCompute.apply(row);
+
+      //noinspection ObjectEquality (if preCompute made a copy, no need to make another copy)
+      if (newRow == row) {
+        newRow = row.copy();
+      }
+
+      for (int i = optimizedDims.nextSetBit(0); i >= 0; i = optimizedDims.nextSetBit(i + 1)) {
+        newRow.set(
+            dimensionStart + i,
+            extractionFns.get(i).apply(newRow.get(dimensionStart + i))
+        );
+      }
+
+      return newRow;
+    };
+  }
+
+  @Override
+  public TypeReference<ResultRow> getResultTypeReference()
+  {
+    return TYPE_REFERENCE;
+  }
+
+  @Override
+  public ObjectMapper decorateObjectMapper(final ObjectMapper objectMapper, final GroupByQuery query)
+  {
+    final boolean resultAsArray = query.getContextBoolean(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, false);
+
+    // Serializer that writes array- or map-based rows as appropriate, based on the "resultAsArray" setting.
+    final JsonSerializer<ResultRow> serializer = new JsonSerializer<ResultRow>()
     {
-      @Nullable
       @Override
-      public Row apply(Row input)
+      public void serialize(
+          final ResultRow resultRow,
+          final JsonGenerator jg,
+          final SerializerProvider serializers
+      ) throws IOException
       {
-        Row preRow = preCompute.apply(input);
-        if (preRow instanceof MapBasedRow) {
-          MapBasedRow preMapRow = (MapBasedRow) preRow;
-          Map<String, Object> event = new HashMap<>(preMapRow.getEvent());
-          for (String dim : optimizedDims) {
-            final Object eventVal = event.get(dim);
-            event.put(dim, extractionFnMap.get(dim).apply(eventVal));
-          }
-          return new MapBasedRow(preMapRow.getTimestamp(), event);
+        if (resultAsArray) {
+          jg.writeObject(resultRow.getArray());
         } else {
-          return preRow;
+          jg.writeObject(resultRow.toMapBasedRow(query));
         }
       }
     };
-  }
 
-  @Override
-  public TypeReference<Row> getResultTypeReference()
-  {
-    return TYPE_REFERENCE;
+    // Deserializer that can deserialize either array- or map-based rows.
+    final JsonDeserializer<ResultRow> deserializer = new JsonDeserializer<ResultRow>()
+    {
+      @Override
+      public ResultRow deserialize(final JsonParser jp, final DeserializationContext ctxt) throws IOException
+      {
+        if (jp.isExpectedStartObjectToken()) {
+          final Row row = jp.readValueAs(Row.class);
+          return ResultRow.fromLegacyRow(row, query);
+        } else {
+          return ResultRow.of(jp.readValueAs(Object[].class));
+        }
+      }
+    };
+
+    class GroupByResultRowModule extends SimpleModule
+    {
+      private GroupByResultRowModule()
+      {
+        addSerializer(ResultRow.class, serializer);
+        addDeserializer(ResultRow.class, deserializer);
+      }
+    }
+
+    final ObjectMapper newObjectMapper = objectMapper.copy();
+    newObjectMapper.registerModule(new GroupByResultRowModule());
+    return newObjectMapper;
   }
 
   @Override
-  public QueryRunner<Row> preMergeQueryDecoration(final QueryRunner<Row> runner)
+  public QueryRunner<ResultRow> preMergeQueryDecoration(final QueryRunner<ResultRow> runner)
   {
     return new SubqueryQueryRunner<>(
-        new QueryRunner<Row>()
+        new QueryRunner<ResultRow>()
         {
           @Override
-          public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
+          public Sequence<ResultRow> run(QueryPlus<ResultRow> queryPlus, ResponseContext responseContext)
           {
             GroupByQuery groupByQuery = (GroupByQuery) queryPlus.getQuery();
             if (groupByQuery.getDimFilter() != null) {
               groupByQuery = groupByQuery.withDimFilter(groupByQuery.getDimFilter().optimize());
             }
             final GroupByQuery delegateGroupByQuery = groupByQuery;
-            ArrayList<DimensionSpec> dimensionSpecs = new ArrayList<>();
-            Set<String> optimizedDimensions = ImmutableSet.copyOf(
-                Iterables.transform(
-                    extractionsToRewrite(delegateGroupByQuery),
-                    new Function<DimensionSpec, String>()
-                    {
-                      @Override
-                      public String apply(DimensionSpec input)
-                      {
-                        return input.getDimension();
-                      }
-                    }
-                )
-            );
-            for (DimensionSpec dimensionSpec : delegateGroupByQuery.getDimensions()) {
-              if (optimizedDimensions.contains(dimensionSpec.getDimension())) {
+            final List<DimensionSpec> dimensionSpecs = new ArrayList<>();
+            final BitSet optimizedDimensions = extractionsToRewrite(delegateGroupByQuery);
+            final List<DimensionSpec> dimensions = delegateGroupByQuery.getDimensions();
+            for (int i = 0; i < dimensions.size(); i++) {
+              final DimensionSpec dimensionSpec = dimensions.get(i);
+              if (optimizedDimensions.get(i)) {
                 dimensionSpecs.add(
                     new DefaultDimensionSpec(dimensionSpec.getDimension(), dimensionSpec.getOutputName())
                 );
@@ -465,9 +513,9 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
   }
 
   @Override
-  public CacheStrategy<Row, Object, GroupByQuery> getCacheStrategy(final GroupByQuery query)
+  public CacheStrategy<ResultRow, Object, GroupByQuery> getCacheStrategy(final GroupByQuery query)
   {
-    return new CacheStrategy<Row, Object, GroupByQuery>()
+    return new CacheStrategy<ResultRow, Object, GroupByQuery>()
     {
       private static final byte CACHE_STRATEGY_VERSION = 0x1;
       private final List<AggregatorFactory> aggs = query.getAggregatorSpecs();
@@ -521,93 +569,106 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
       }
 
       @Override
-      public Function<Row, Object> prepareForCache(boolean isResultLevelCache)
+      public Function<ResultRow, Object> prepareForCache(boolean isResultLevelCache)
       {
-        return new Function<Row, Object>()
+        final boolean resultRowHasTimestamp = query.getResultRowHasTimestamp();
+
+        return new Function<ResultRow, Object>()
         {
           @Override
-          public Object apply(Row input)
+          public Object apply(ResultRow resultRow)
           {
-            if (input instanceof MapBasedRow) {
-              final MapBasedRow row = (MapBasedRow) input;
-              final List<Object> retVal = Lists.newArrayListWithCapacity(1 + dims.size() + aggs.size());
-              retVal.add(row.getTimestamp().getMillis());
-              Map<String, Object> event = row.getEvent();
-              for (DimensionSpec dim : dims) {
-                retVal.add(event.get(dim.getOutputName()));
-              }
-              for (AggregatorFactory agg : aggs) {
-                retVal.add(event.get(agg.getName()));
-              }
-              if (isResultLevelCache) {
-                for (PostAggregator postAgg : query.getPostAggregatorSpecs()) {
-                  retVal.add(event.get(postAgg.getName()));
-                }
-              }
-              return retVal;
+            final List<Object> retVal = new ArrayList<>(1 + dims.size() + aggs.size());
+            int inPos = 0;
+
+            if (resultRowHasTimestamp) {
+              retVal.add(resultRow.getLong(inPos++));
+            } else {
+              retVal.add(query.getUniversalTimestamp().getMillis());
             }
 
-            throw new ISE("Don't know how to cache input rows of type[%s]", input.getClass());
+            for (int i = 0; i < dims.size(); i++) {
+              retVal.add(resultRow.get(inPos++));
+            }
+            for (int i = 0; i < aggs.size(); i++) {
+              retVal.add(resultRow.get(inPos++));
+            }
+            if (isResultLevelCache) {
+              for (int i = 0; i < query.getPostAggregatorSpecs().size(); i++) {
+                retVal.add(resultRow.get(inPos++));
+              }
+            }
+            return retVal;
           }
         };
       }
 
       @Override
-      public Function<Object, Row> pullFromCache(boolean isResultLevelCache)
+      public Function<Object, ResultRow> pullFromCache(boolean isResultLevelCache)
       {
-        return new Function<Object, Row>()
+        final boolean resultRowHasTimestamp = query.getResultRowHasTimestamp();
+        final int dimensionStart = query.getResultRowDimensionStart();
+        final int aggregatorStart = query.getResultRowAggregatorStart();
+        final int postAggregatorStart = query.getResultRowPostAggregatorStart();
+
+        return new Function<Object, ResultRow>()
         {
           private final Granularity granularity = query.getGranularity();
 
           @Override
-          public Row apply(Object input)
+          public ResultRow apply(Object input)
           {
             Iterator<Object> results = ((List<Object>) input).iterator();
 
             DateTime timestamp = granularity.toDateTime(((Number) results.next()).longValue());
 
-            final Map<String, Object> event = Maps.newLinkedHashMap();
-            Iterator<DimensionSpec> dimsIter = dims.iterator();
+            final int size = isResultLevelCache
+                             ? query.getResultRowSizeWithPostAggregators()
+                             : query.getResultRowSizeWithoutPostAggregators();
+
+            final ResultRow resultRow = ResultRow.create(size);
+
+            if (resultRowHasTimestamp) {
+              resultRow.set(0, timestamp.getMillis());
+            }
+
+            final Iterator<DimensionSpec> dimsIter = dims.iterator();
+            int dimPos = 0;
             while (dimsIter.hasNext() && results.hasNext()) {
               final DimensionSpec dimensionSpec = dimsIter.next();
 
               // Must convert generic Jackson-deserialized type into the proper type.
-              event.put(
-                  dimensionSpec.getOutputName(),
+              resultRow.set(
+                  dimensionStart + dimPos,
                   DimensionHandlerUtils.convertObjectToType(results.next(), dimensionSpec.getOutputType())
               );
             }
-            Iterator<AggregatorFactory> aggsIter = aggs.iterator();
 
             CacheStrategy.fetchAggregatorsFromCache(
-                aggsIter,
+                aggs,
                 results,
                 isResultLevelCache,
-                (aggName, aggValueObject) -> {
-                  event.put(aggName, aggValueObject);
-                  return null;
+                (aggName, aggPosition, aggValueObject) -> {
+                  resultRow.set(aggregatorStart + aggPosition, aggValueObject);
                 }
             );
 
             if (isResultLevelCache) {
               Iterator<PostAggregator> postItr = query.getPostAggregatorSpecs().iterator();
+              int postPos = 0;
               while (postItr.hasNext() && results.hasNext()) {
-                event.put(postItr.next().getName(), results.next());
+                resultRow.set(postAggregatorStart + postPos, results.next());
               }
             }
-            if (dimsIter.hasNext() || aggsIter.hasNext() || results.hasNext()) {
+            if (dimsIter.hasNext() || results.hasNext()) {
               throw new ISE(
-                  "Found left over objects while reading from cache!! dimsIter[%s] aggsIter[%s] results[%s]",
+                  "Found left over objects while reading from cache!! dimsIter[%s] results[%s]",
                   dimsIter.hasNext(),
-                  aggsIter.hasNext(),
                   results.hasNext()
               );
             }
 
-            return new MapBasedRow(
-                timestamp,
-                event
-            );
+            return resultRow;
           }
         };
       }
@@ -621,22 +682,22 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
    *
    * @param query The query to check for optimizations
    *
-   * @return A collection of DimensionsSpec which can be extracted at the last second upon query completion.
+   * @return The set of dimensions (as offsets into {@code query.getDimensions()}) which can be extracted at the last
+   * second upon query completion.
    */
-  public static Collection<DimensionSpec> extractionsToRewrite(GroupByQuery query)
+  private static BitSet extractionsToRewrite(GroupByQuery query)
   {
-    return Collections2.filter(
-        query.getDimensions(), new Predicate<DimensionSpec>()
-        {
-          @Override
-          public boolean apply(DimensionSpec input)
-          {
-            return input.getExtractionFn() != null
-                   && ExtractionFn.ExtractionType.ONE_TO_ONE.equals(
-                input.getExtractionFn().getExtractionType()
-            );
-          }
-        }
-    );
+    final BitSet retVal = new BitSet();
+
+    final List<DimensionSpec> dimensions = query.getDimensions();
+    for (int i = 0; i < dimensions.size(); i++) {
+      final DimensionSpec dimensionSpec = dimensions.get(i);
+      if (dimensionSpec.getExtractionFn() != null
+          && ExtractionFn.ExtractionType.ONE_TO_ONE.equals(dimensionSpec.getExtractionFn().getExtractionType())) {
+        retVal.set(i);
+      }
+    }
+
+    return retVal;
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
index 6d7c5ef..3fbeb25 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
@@ -23,7 +23,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.inject.Inject;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.Query;
@@ -39,8 +38,9 @@ import org.apache.druid.segment.StorageAdapter;
 import java.util.concurrent.ExecutorService;
 
 /**
+ *
  */
-public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupByQuery>
+public class GroupByQueryRunnerFactory implements QueryRunnerFactory<ResultRow, GroupByQuery>
 {
   private final GroupByStrategySelector strategySelector;
   private final GroupByQueryQueryToolChest toolChest;
@@ -56,38 +56,40 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
   }
 
   @Override
-  public QueryRunner<Row> createRunner(final Segment segment)
+  public QueryRunner<ResultRow> createRunner(final Segment segment)
   {
     return new GroupByQueryRunner(segment, strategySelector);
   }
 
   @Override
-  public QueryRunner<Row> mergeRunners(final ExecutorService exec, final Iterable<QueryRunner<Row>> queryRunners)
+  public QueryRunner<ResultRow> mergeRunners(
+      final ExecutorService exec,
+      final Iterable<QueryRunner<ResultRow>> queryRunners
+  )
   {
     // mergeRunners should take ListeningExecutorService at some point
     final ListeningExecutorService queryExecutor = MoreExecutors.listeningDecorator(exec);
 
-    return new QueryRunner<Row>()
+    return new QueryRunner<ResultRow>()
     {
       @Override
-      public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
+      public Sequence<ResultRow> run(QueryPlus<ResultRow> queryPlus, ResponseContext responseContext)
       {
-        QueryRunner<Row> rowQueryRunner = strategySelector.strategize((GroupByQuery) queryPlus.getQuery()).mergeRunners(
-            queryExecutor,
-            queryRunners
-        );
+        QueryRunner<ResultRow> rowQueryRunner = strategySelector
+            .strategize((GroupByQuery) queryPlus.getQuery())
+            .mergeRunners(queryExecutor, queryRunners);
         return rowQueryRunner.run(queryPlus, responseContext);
       }
     };
   }
 
   @Override
-  public QueryToolChest<Row, GroupByQuery> getToolchest()
+  public QueryToolChest<ResultRow, GroupByQuery> getToolchest()
   {
     return toolChest;
   }
 
-  private static class GroupByQueryRunner implements QueryRunner<Row>
+  private static class GroupByQueryRunner implements QueryRunner<ResultRow>
   {
     private final StorageAdapter adapter;
     private final GroupByStrategySelector strategySelector;
@@ -99,9 +101,9 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
     }
 
     @Override
-    public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
+    public Sequence<ResultRow> run(QueryPlus<ResultRow> queryPlus, ResponseContext responseContext)
     {
-      Query<Row> query = queryPlus.getQuery();
+      Query<ResultRow> query = queryPlus.getQuery();
       if (!(query instanceof GroupByQuery)) {
         throw new ISE("Got a [%s] which isn't a %s", query.getClass(), GroupByQuery.class);
       }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/ResultRow.java b/processing/src/main/java/org/apache/druid/query/groupby/ResultRow.java
new file mode 100644
index 0000000..b31228c
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/groupby/ResultRow.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.groupby;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonValue;
+import org.apache.druid.data.input.MapBasedRow;
+import org.apache.druid.data.input.Row;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.joda.time.DateTime;
+
+import javax.annotation.Nullable;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Used by {@link GroupByQuery} for results. Each row is positional, and has the following fields, in order:
+ *
+ * - Timestamp (optional; only if granularity != ALL)
+ * - Dimensions (in order)
+ * - Aggregators (in order)
+ * - Post-aggregators (optional; in order, if present)
+ *
+ * ResultRows may sometimes be created without space reserved for post-aggregators, in contexts where it is known
+ * that post-aggregators will not be computed.
+ *
+ * @see GroupByQuery#getResultRowOrder()
+ * @see GroupByQuery#getResultRowHasTimestamp()
+ * @see GroupByQuery#getUniversalTimestamp()
+ * @see GroupByQuery#getResultRowDimensionStart()
+ * @see GroupByQuery#getResultRowAggregatorStart()
+ * @see GroupByQuery#getResultRowPostAggregatorStart()
+ * @see GroupByQuery#getResultRowSizeWithPostAggregators()
+ * @see GroupByQuery#getResultRowSizeWithoutPostAggregators()
+ */
+public final class ResultRow
+{
+  private final Object[] row;
+
+  private ResultRow(final Object[] row)
+  {
+    this.row = row;
+  }
+
+  /**
+   * Create a row from an array of objects.
+   */
+  @JsonCreator
+  public static ResultRow of(final Object... row)
+  {
+    return new ResultRow(row);
+  }
+
+  /**
+   * Create a row of a certain size, initialized to all nulls.
+   */
+  public static ResultRow create(final int size)
+  {
+    return new ResultRow(new Object[size]);
+  }
+
+  /**
+   * Create a row based on a legacy {@link Row} that was generated by a given {@link GroupByQuery}. This is useful
+   * for deserializing rows that have come off the wire in the older format. (In the past, GroupBy query results
+   * were sequences of {@link Row}, not ResultRow.)
+   *
+   * @param row   legacy row
+   * @param query query corresponding to the output ResultRow
+   */
+  public static ResultRow fromLegacyRow(Row row, final GroupByQuery query)
+  {
+    // Can't be sure if we'll get result rows with or without postaggregations, so be safe.
+    final ResultRow resultRow = ResultRow.create(query.getResultRowSizeWithPostAggregators());
+
+    int i = 0;
+    if (query.getResultRowHasTimestamp()) {
+      resultRow.set(i++, row.getTimestamp().getMillis());
+    }
+
+    for (DimensionSpec dimensionSpec : query.getDimensions()) {
+      resultRow.set(i++, row.getRaw(dimensionSpec.getOutputName()));
+    }
+
+    for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) {
+      resultRow.set(i++, row.getRaw(aggregatorFactory.getName()));
+    }
+
+    for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) {
+      resultRow.set(i++, row.getRaw(postAggregator.getName()));
+    }
+
+    return resultRow;
+  }
+
+  /**
+   * Get the backing array for this row (not a copy).
+   */
+  @JsonValue
+  public Object[] getArray()
+  {
+    return row;
+  }
+
+  public void set(final int i, @Nullable final Object o)
+  {
+    row[i] = o;
+  }
+
+  @Nullable
+  public Object get(final int i)
+  {
+    return row[i];
+  }
+
+  public long getLong(final int i)
+  {
+    return ((Number) row[i]).longValue();
+  }
+
+  public int length()
+  {
+    return row.length;
+  }
+
+  /**
+   * Returns a copy of this row. The backing array will be copied as well.
+   */
+  public ResultRow copy()
+  {
+    final Object[] newArray = new Object[row.length];
+    System.arraycopy(row, 0, newArray, 0, row.length);
+    return new ResultRow(newArray);
+  }
+
+  /**
+   * Returns a Map representation of the data in this row. Does not include the timestamp.
+   */
+  public Map<String, Object> toMap(final GroupByQuery query)
+  {
+    final List<String> resultRowOrder = query.getResultRowOrder();
+    final Map<String, Object> map = new HashMap<>();
+
+    for (int i = query.getResultRowDimensionStart(); i < row.length; i++) {
+      final String columnName = resultRowOrder.get(i);
+
+      if (row[i] != null) {
+        map.put(columnName, row[i]);
+      }
+    }
+
+    return map;
+  }
+
+  /**
+   * Returns a {@link Row} representation of the data in this row.
+   */
+  public MapBasedRow toMapBasedRow(final GroupByQuery query)
+  {
+    // May be null, if so it'll get replaced later
+    final DateTime timestamp;
+
+    if (query.getResultRowHasTimestamp()) {
+      timestamp = query.getGranularity().toDateTime(getLong(0));
+    } else {
+      timestamp = query.getUniversalTimestamp();
+    }
+
+    return new MapBasedRow(timestamp, toMap(query));
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ResultRow resultRow = (ResultRow) o;
+    return Arrays.equals(row, resultRow.row);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Arrays.hashCode(row);
+  }
+
+  @Override
+  public String toString()
+  {
+    return "ResultRow{" +
+           "row=" + Arrays.toString(row) +
+           '}';
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/RowBasedColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/query/groupby/RowBasedColumnSelectorFactory.java
index 34e5596..b98dcde 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/RowBasedColumnSelectorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/RowBasedColumnSelectorFactory.java
@@ -20,10 +20,10 @@
 package org.apache.druid.query.groupby;
 
 import com.google.common.base.Predicate;
-import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableMap;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.data.input.Row;
+import org.apache.druid.data.input.Rows;
 import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.query.extraction.ExtractionFn;
 import org.apache.druid.query.filter.ValueMatcher;
@@ -47,35 +47,64 @@ import javax.annotation.Nullable;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.function.ToLongFunction;
 
-public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
+public class RowBasedColumnSelectorFactory<T> implements ColumnSelectorFactory
 {
-  private final Supplier<? extends Row> row;
+  public interface RowAdapter<T>
+  {
+    ToLongFunction<T> timestampFunction();
+
+    Function<T, Object> rawFunction(String columnName);
+  }
+
+  private final Supplier<T> supplier;
+  private final RowAdapter<T> adapter;
   private final Map<String, ValueType> rowSignature;
 
   private RowBasedColumnSelectorFactory(
-      final Supplier<? extends Row> row,
+      final Supplier<T> supplier,
+      final RowAdapter<T> adapter,
       @Nullable final Map<String, ValueType> rowSignature
   )
   {
-    this.row = row;
+    this.supplier = supplier;
+    this.adapter = adapter;
     this.rowSignature = rowSignature != null ? rowSignature : ImmutableMap.of();
   }
 
-  public static RowBasedColumnSelectorFactory create(
-      final Supplier<? extends Row> row,
-      @Nullable final Map<String, ValueType> rowSignature
+  public static <RowType extends Row> RowBasedColumnSelectorFactory create(
+      final Supplier<RowType> supplier,
+      @Nullable final Map<String, ValueType> signature
   )
   {
-    return new RowBasedColumnSelectorFactory(row, rowSignature);
+    final RowAdapter<RowType> adapter = new RowAdapter<RowType>()
+    {
+      @Override
+      public ToLongFunction<RowType> timestampFunction()
+      {
+        return Row::getTimestampFromEpoch;
+      }
+
+      @Override
+      public Function<RowType, Object> rawFunction(String columnName)
+      {
+        return r -> r.getRaw(columnName);
+      }
+    };
+
+    return new RowBasedColumnSelectorFactory<>(supplier, adapter, signature);
   }
 
-  public static RowBasedColumnSelectorFactory create(
-      final ThreadLocal<? extends Row> row,
-      @Nullable final Map<String, ValueType> rowSignature
+  public static <RowType> RowBasedColumnSelectorFactory create(
+      final RowAdapter<RowType> adapter,
+      final Supplier<RowType> supplier,
+      @Nullable final Map<String, ValueType> signature
   )
   {
-    return new RowBasedColumnSelectorFactory(row::get, rowSignature);
+    return new RowBasedColumnSelectorFactory<>(supplier, adapter, signature);
   }
 
   @Override
@@ -96,22 +125,26 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
         throw new UnsupportedOperationException("time dimension must provide an extraction function");
       }
 
+      final ToLongFunction<T> timestampFunction = adapter.timestampFunction();
+
       return new BaseSingleValueDimensionSelector()
       {
         @Override
         protected String getValue()
         {
-          return extractionFn.apply(row.get().getTimestampFromEpoch());
+          return extractionFn.apply(timestampFunction.applyAsLong(supplier.get()));
         }
 
         @Override
         public void inspectRuntimeShape(RuntimeShapeInspector inspector)
         {
-          inspector.visit("row", row);
+          inspector.visit("row", supplier);
           inspector.visit("extractionFn", extractionFn);
         }
       };
     } else {
+      final Function<T, Object> dimFunction = adapter.rawFunction(dimension);
+
       return new DimensionSelector()
       {
         private final RangeIndexedInts indexedInts = new RangeIndexedInts();
@@ -119,7 +152,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
         @Override
         public IndexedInts getRow()
         {
-          final List<String> dimensionValues = row.get().getDimension(dimension);
+          final List<String> dimensionValues = Rows.objectToStrings(dimFunction.apply(supplier.get()));
           indexedInts.setSize(dimensionValues != null ? dimensionValues.size() : 0);
           return indexedInts;
         }
@@ -133,7 +166,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
               @Override
               public boolean matches()
               {
-                final List<String> dimensionValues = row.get().getDimension(dimension);
+                final List<String> dimensionValues = Rows.objectToStrings(dimFunction.apply(supplier.get()));
                 if (dimensionValues == null || dimensionValues.isEmpty()) {
                   return value == null;
                 }
@@ -149,7 +182,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
               @Override
               public void inspectRuntimeShape(RuntimeShapeInspector inspector)
               {
-                inspector.visit("row", row);
+                inspector.visit("row", supplier);
               }
             };
           } else {
@@ -158,7 +191,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
               @Override
               public boolean matches()
               {
-                final List<String> dimensionValues = row.get().getDimension(dimension);
+                final List<String> dimensionValues = Rows.objectToStrings(dimFunction.apply(supplier.get()));
                 if (dimensionValues == null || dimensionValues.isEmpty()) {
                   return value == null;
                 }
@@ -174,7 +207,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
               @Override
               public void inspectRuntimeShape(RuntimeShapeInspector inspector)
               {
-                inspector.visit("row", row);
+                inspector.visit("row", supplier);
                 inspector.visit("extractionFn", extractionFn);
               }
             };
@@ -191,7 +224,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
               @Override
               public boolean matches()
               {
-                final List<String> dimensionValues = row.get().getDimension(dimension);
+                final List<String> dimensionValues = Rows.objectToStrings(dimFunction.apply(supplier.get()));
                 if (dimensionValues == null || dimensionValues.isEmpty()) {
                   return matchNull;
                 }
@@ -207,7 +240,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
               @Override
               public void inspectRuntimeShape(RuntimeShapeInspector inspector)
               {
-                inspector.visit("row", row);
+                inspector.visit("row", supplier);
                 inspector.visit("predicate", predicate);
               }
             };
@@ -217,7 +250,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
               @Override
               public boolean matches()
               {
-                final List<String> dimensionValues = row.get().getDimension(dimension);
+                final List<String> dimensionValues = Rows.objectToStrings(dimFunction.apply(supplier.get()));
                 if (dimensionValues == null || dimensionValues.isEmpty()) {
                   return matchNull;
                 }
@@ -233,7 +266,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
               @Override
               public void inspectRuntimeShape(RuntimeShapeInspector inspector)
               {
-                inspector.visit("row", row);
+                inspector.visit("row", supplier);
                 inspector.visit("predicate", predicate);
               }
             };
@@ -249,7 +282,9 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
         @Override
         public String lookupName(int id)
         {
-          final String value = NullHandling.emptyToNullIfNeeded(row.get().getDimension(dimension).get(id));
+          final String value = NullHandling.emptyToNullIfNeeded(
+              Rows.objectToStrings(dimFunction.apply(supplier.get())).get(id)
+          );
           return extractionFn == null ? value : extractionFn.apply(value);
         }
 
@@ -270,7 +305,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
         @Override
         public Object getObject()
         {
-          List<String> dimensionValues = row.get().getDimension(dimension);
+          List<String> dimensionValues = Rows.objectToStrings(dimFunction.apply(supplier.get()));
           if (dimensionValues == null) {
             return null;
           }
@@ -289,7 +324,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
         @Override
         public void inspectRuntimeShape(RuntimeShapeInspector inspector)
         {
-          inspector.visit("row", row);
+          inspector.visit("row", supplier);
           inspector.visit("extractionFn", extractionFn);
         }
       };
@@ -300,12 +335,14 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
   public ColumnValueSelector<?> makeColumnValueSelector(String columnName)
   {
     if (columnName.equals(ColumnHolder.TIME_COLUMN_NAME)) {
+      final ToLongFunction<T> timestampFunction = adapter.timestampFunction();
+
       class TimeLongColumnSelector implements LongColumnSelector
       {
         @Override
         public long getLong()
         {
-          return row.get().getTimestampFromEpoch();
+          return timestampFunction.applyAsLong(supplier.get());
         }
 
         @Override
@@ -318,23 +355,25 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
         @Override
         public void inspectRuntimeShape(RuntimeShapeInspector inspector)
         {
-          inspector.visit("row", row);
+          inspector.visit("row", supplier);
         }
       }
       return new TimeLongColumnSelector();
     } else {
+      final Function<T, Object> rawFunction = adapter.rawFunction(columnName);
+
       return new ColumnValueSelector()
       {
         @Override
         public boolean isNull()
         {
-          return row.get().getRaw(columnName) == null;
+          return rawFunction.apply(supplier.get()) == null;
         }
 
         @Override
         public double getDouble()
         {
-          Number metric = row.get().getMetric(columnName);
+          Number metric = Rows.objectToNumber(columnName, rawFunction.apply(supplier.get()));
           assert NullHandling.replaceWithDefault() || metric != null;
           return DimensionHandlerUtils.nullToZero(metric).doubleValue();
         }
@@ -342,7 +381,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
         @Override
         public float getFloat()
         {
-          Number metric = row.get().getMetric(columnName);
+          Number metric = Rows.objectToNumber(columnName, rawFunction.apply(supplier.get()));
           assert NullHandling.replaceWithDefault() || metric != null;
           return DimensionHandlerUtils.nullToZero(metric).floatValue();
         }
@@ -350,7 +389,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
         @Override
         public long getLong()
         {
-          Number metric = row.get().getMetric(columnName);
+          Number metric = Rows.objectToNumber(columnName, rawFunction.apply(supplier.get()));
           assert NullHandling.replaceWithDefault() || metric != null;
           return DimensionHandlerUtils.nullToZero(metric).longValue();
         }
@@ -359,7 +398,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
         @Override
         public Object getObject()
         {
-          return row.get().getRaw(columnName);
+          return rawFunction.apply(supplier.get());
         }
 
         @Override
@@ -371,7 +410,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
         @Override
         public void inspectRuntimeShape(RuntimeShapeInspector inspector)
         {
-          inspector.visit("row", row);
+          inspector.visit("row", supplier);
         }
       };
     }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java
index 6071adc..71342aa 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java
@@ -19,19 +19,25 @@
 
 package org.apache.druid.query.groupby.epinephelinae;
 
-import com.google.common.collect.Maps;
-import org.apache.druid.data.input.MapBasedRow;
-import org.apache.druid.data.input.Row;
+import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.granularity.AllGranularity;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.query.groupby.GroupByQuery;
-import org.joda.time.DateTime;
+import org.apache.druid.query.groupby.ResultRow;
 
-import java.util.Map;
+import javax.annotation.Nullable;
+import java.util.List;
 import java.util.function.BinaryOperator;
 
-public class GroupByBinaryFnV2 implements BinaryOperator<Row>
+/**
+ * Class that knows how to merge aggregator data from two groupBy {@link ResultRow} objects that have the same time
+ * and dimensions. This code runs on Brokers as well as data servers, like Historicals.
+ *
+ * Used by
+ * {@link org.apache.druid.query.groupby.strategy.GroupByStrategyV2#mergeResults}.
+ */
+public class GroupByBinaryFnV2 implements BinaryOperator<ResultRow>
 {
   private final GroupByQuery query;
 
@@ -41,7 +47,8 @@ public class GroupByBinaryFnV2 implements BinaryOperator<Row>
   }
 
   @Override
-  public Row apply(final Row arg1, final Row arg2)
+  @Nullable
+  public ResultRow apply(@Nullable final ResultRow arg1, @Nullable final ResultRow arg2)
   {
     if (arg1 == null) {
       return arg2;
@@ -49,35 +56,39 @@ public class GroupByBinaryFnV2 implements BinaryOperator<Row>
       return arg1;
     }
 
-    final Map<String, Object> newMap = Maps.newHashMapWithExpectedSize(
-        query.getDimensions().size() + query.getAggregatorSpecs().size()
-    );
+    final ResultRow newResult = ResultRow.create(query.getResultRowSizeWithoutPostAggregators());
+
+    // Add timestamp.
+    if (query.getResultRowHasTimestamp()) {
+      newResult.set(0, adjustTimestamp(arg1));
+    }
 
-    // Add dimensions
-    for (DimensionSpec dimension : query.getDimensions()) {
-      newMap.put(dimension.getOutputName(), arg1.getRaw(dimension.getOutputName()));
+    // Add dimensions.
+    final int dimensionStart = query.getResultRowDimensionStart();
+    final List<DimensionSpec> dimensions = query.getDimensions();
+    for (int i = 0; i < dimensions.size(); i++) {
+      final int rowIndex = dimensionStart + i;
+      newResult.set(rowIndex, arg1.get(rowIndex));
     }
 
-    // Add aggregations
-    for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) {
-      newMap.put(
-          aggregatorFactory.getName(),
-          aggregatorFactory.combine(
-              arg1.getRaw(aggregatorFactory.getName()),
-              arg2.getRaw(aggregatorFactory.getName())
-          )
-      );
+    // Add aggregations.
+    final int aggregatorStart = query.getResultRowAggregatorStart();
+    final List<AggregatorFactory> aggregatorSpecs = query.getAggregatorSpecs();
+    for (int i = 0; i < aggregatorSpecs.size(); i++) {
+      final AggregatorFactory aggregatorFactory = aggregatorSpecs.get(i);
+      final int rowIndex = aggregatorStart + i;
+      newResult.set(rowIndex, aggregatorFactory.combine(arg1.get(rowIndex), arg2.get(rowIndex)));
     }
 
-    return new MapBasedRow(adjustTimestamp(arg1), newMap);
+    return newResult;
   }
 
-  private DateTime adjustTimestamp(final Row row)
+  private long adjustTimestamp(final ResultRow row)
   {
     if (query.getGranularity() instanceof AllGranularity) {
-      return row.getTimestamp();
+      return row.getLong(0);
     } else {
-      return query.getGranularity().bucketStart(row.getTimestamp());
+      return query.getGranularity().bucketStart(DateTimes.utc(row.getLong(0))).getMillis();
     }
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java
index cccb3e5..aa162db 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java
@@ -34,7 +34,6 @@ import com.google.common.util.concurrent.MoreExecutors;
 import org.apache.druid.collections.BlockingPool;
 import org.apache.druid.collections.ReferenceCountingResourceHolder;
 import org.apache.druid.collections.Releaser;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.StringUtils;
@@ -51,13 +50,12 @@ import org.apache.druid.query.QueryPlus;
 import org.apache.druid.query.QueryRunner;
 import org.apache.druid.query.QueryWatcher;
 import org.apache.druid.query.ResourceLimitExceededException;
-import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.context.ResponseContext;
 import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper.RowBasedKey;
 
-import java.io.Closeable;
 import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -69,13 +67,27 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-public class GroupByMergingQueryRunnerV2 implements QueryRunner<Row>
+/**
+ * Class that knows how to merge a collection of groupBy {@link QueryRunner} objects, called {@code queryables},
+ * using a buffer provided by {@code mergeBufferPool} and a parallel executor provided by {@code exec}. Outputs a
+ * fully aggregated stream of {@link ResultRow} objects. Does not apply post-aggregators.
+ *
+ * The input {@code queryables} are expected to come from a {@link GroupByQueryEngineV2}. This code runs on data
+ * servers, like Historicals.
+ *
+ * This class has some resemblance to {@link GroupByRowProcessor}. See the javadoc of that class for a discussion of
+ * similarities and differences.
+ *
+ * Used by
+ * {@link org.apache.druid.query.groupby.strategy.GroupByStrategyV2#mergeRunners(ListeningExecutorService, Iterable)}.
+ */
+public class GroupByMergingQueryRunnerV2 implements QueryRunner<ResultRow>
 {
   private static final Logger log = new Logger(GroupByMergingQueryRunnerV2.class);
   private static final String CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION = "mergeRunnersUsingChainedExecution";
 
   private final GroupByQueryConfig config;
-  private final Iterable<QueryRunner<Row>> queryables;
+  private final Iterable<QueryRunner<ResultRow>> queryables;
   private final ListeningExecutorService exec;
   private final QueryWatcher queryWatcher;
   private final int concurrencyHint;
@@ -88,7 +100,7 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<Row>
       GroupByQueryConfig config,
       ExecutorService exec,
       QueryWatcher queryWatcher,
-      Iterable<QueryRunner<Row>> queryables,
+      Iterable<QueryRunner<ResultRow>> queryables,
       int concurrencyHint,
       BlockingPool<ByteBuffer> mergeBufferPool,
       int mergeBufferSize,
@@ -108,7 +120,7 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<Row>
   }
 
   @Override
-  public Sequence<Row> run(final QueryPlus<Row> queryPlus, final ResponseContext responseContext)
+  public Sequence<ResultRow> run(final QueryPlus<ResultRow> queryPlus, final ResponseContext responseContext)
   {
     final GroupByQuery query = (GroupByQuery) queryPlus.getQuery();
     final GroupByQueryConfig querySpecificConfig = config.withOverrides(query);
@@ -122,24 +134,19 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<Row>
         CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION,
         false
     );
-    final QueryPlus<Row> queryPlusForRunners = queryPlus
+    final QueryPlus<ResultRow> queryPlusForRunners = queryPlus
         .withQuery(
             query.withOverriddenContext(ImmutableMap.of(CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION, true))
         )
         .withoutThreadUnsafeState();
 
     if (QueryContexts.isBySegment(query) || forceChainedExecution) {
-      ChainedExecutionQueryRunner<Row> runner = new ChainedExecutionQueryRunner<>(exec, queryWatcher, queryables);
+      ChainedExecutionQueryRunner<ResultRow> runner = new ChainedExecutionQueryRunner<>(exec, queryWatcher, queryables);
       return runner.run(queryPlusForRunners, responseContext);
     }
 
     final boolean isSingleThreaded = querySpecificConfig.isSingleThreaded();
 
-    final AggregatorFactory[] combiningAggregatorFactories = new AggregatorFactory[query.getAggregatorSpecs().size()];
-    for (int i = 0; i < query.getAggregatorSpecs().size(); i++) {
-      combiningAggregatorFactories[i] = query.getAggregatorSpecs().get(i).getCombiningFactory();
-    }
-
     final File temporaryStorageDirectory = new File(
         processingTmpDir,
         StringUtils.format("druid-groupBy-%s_%s", UUID.randomUUID(), query.getId())
@@ -154,10 +161,10 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<Row>
     final long timeoutAt = System.currentTimeMillis() + queryTimeout;
 
     return new BaseSequence<>(
-        new BaseSequence.IteratorMaker<Row, CloseableGrouperIterator<RowBasedKey, Row>>()
+        new BaseSequence.IteratorMaker<ResultRow, CloseableGrouperIterator<RowBasedKey, ResultRow>>()
         {
           @Override
-          public CloseableGrouperIterator<RowBasedKey, Row> make()
+          public CloseableGrouperIterator<RowBasedKey, ResultRow> make()
           {
             final List<ReferenceCountingResourceHolder> resources = new ArrayList<>();
 
@@ -185,10 +192,9 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<Row>
                                                                                       mergeBufferHolders.get(1) :
                                                                                       null;
 
-              Pair<Grouper<RowBasedKey>, Accumulator<AggregateResult, Row>> pair =
+              Pair<Grouper<RowBasedKey>, Accumulator<AggregateResult, ResultRow>> pair =
                   RowBasedGrouperHelper.createGrouperAccumulatorPair(
                       query,
-                      false,
                       null,
                       config,
                       Suppliers.ofInstance(mergeBufferHolder.get()),
@@ -196,16 +202,14 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<Row>
                       concurrencyHint,
                       temporaryStorage,
                       spillMapper,
-                      combiningAggregatorFactories,
                       exec,
                       priority,
                       hasTimeout,
                       timeoutAt,
-                      mergeBufferSize,
-                      false
+                      mergeBufferSize
                   );
               final Grouper<RowBasedKey> grouper = pair.lhs;
-              final Accumulator<AggregateResult, Row> accumulator = pair.rhs;
+              final Accumulator<AggregateResult, ResultRow> accumulator = pair.rhs;
               grouper.init();
 
               final ReferenceCountingResourceHolder<Grouper<RowBasedKey>> grouperHolder =
@@ -216,15 +220,13 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<Row>
                   Lists.newArrayList(
                       Iterables.transform(
                           queryables,
-                          new Function<QueryRunner<Row>, ListenableFuture<AggregateResult>>()
+                          new Function<QueryRunner<ResultRow>, ListenableFuture<AggregateResult>>()
                           {
                             @Override
-                            public ListenableFuture<AggregateResult> apply(final QueryRunner<Row> input)
+                            public ListenableFuture<AggregateResult> apply(final QueryRunner<ResultRow> input)
                             {
                               if (input == null) {
-                                throw new ISE(
-                                    "Null queryRunner! Looks to be some segment unmapping action happening"
-                                );
+                                throw new ISE("Null queryRunner! Looks to be some segment unmapping action happening");
                               }
 
                               ListenableFuture<AggregateResult> future = exec.submit(
@@ -240,14 +242,9 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<Row>
                                           @SuppressWarnings("unused")
                                           Releaser grouperReleaser = grouperHolder.increment()
                                       ) {
-                                        final AggregateResult retVal = input.run(queryPlusForRunners, responseContext)
-                                                                            .accumulate(
-                                                                                AggregateResult.ok(),
-                                                                                accumulator
-                                                                            );
-
                                         // Return true if OK, false if resources were exhausted.
-                                        return retVal;
+                                        return input.run(queryPlusForRunners, responseContext)
+                                                    .accumulate(AggregateResult.ok(), accumulator);
                                       }
                                       catch (QueryInterruptedException e) {
                                         throw e;
@@ -283,29 +280,18 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<Row>
               return RowBasedGrouperHelper.makeGrouperIterator(
                   grouper,
                   query,
-                  new Closeable()
-                  {
-                    @Override
-                    public void close()
-                    {
-                      for (Closeable closeable : Lists.reverse(resources)) {
-                        CloseQuietly.close(closeable);
-                      }
-                    }
-                  }
+                  () -> Lists.reverse(resources).forEach(CloseQuietly::close)
               );
             }
             catch (Throwable e) {
               // Exception caught while setting up the iterator; release resources.
-              for (Closeable closeable : Lists.reverse(resources)) {
-                CloseQuietly.close(closeable);
-              }
+              Lists.reverse(resources).forEach(CloseQuietly::close);
               throw e;
             }
           }
 
           @Override
-          public void cleanup(CloseableGrouperIterator<RowBasedKey, Row> iterFromMake)
+          public void cleanup(CloseableGrouperIterator<RowBasedKey, ResultRow> iterFromMake)
           {
             iterFromMake.close();
           }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java
index 836f36f..c26e515 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java
@@ -23,12 +23,9 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
 import org.apache.druid.collections.NonBlockingPool;
 import org.apache.druid.collections.ResourceHolder;
 import org.apache.druid.common.config.NullHandling;
-import org.apache.druid.data.input.MapBasedRow;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.ISE;
@@ -43,6 +40,7 @@ import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.epinephelinae.column.DictionaryBuildingStringGroupByColumnSelectorStrategy;
 import org.apache.druid.query.groupby.epinephelinae.column.DoubleGroupByColumnSelectorStrategy;
 import org.apache.druid.query.groupby.epinephelinae.column.FloatGroupByColumnSelectorStrategy;
@@ -71,20 +69,32 @@ import java.io.Closeable;
 import java.nio.ByteBuffer;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.function.Function;
 
+/**
+ * Class that knows how to process a groupBy query on a single {@link StorageAdapter}. It returns a {@link Sequence}
+ * of {@link ResultRow} objects that are not guaranteed to be in any particular order, and may not even be fully
+ * grouped. It is expected that a downstream {@link GroupByMergingQueryRunnerV2} will finish grouping these results.
+ *
+ * This code runs on data servers, like Historicals.
+ *
+ * Used by
+ * {@link GroupByStrategyV2#process(GroupByQuery, StorageAdapter)}.
+ */
 public class GroupByQueryEngineV2
 {
   private static final GroupByStrategyFactory STRATEGY_FACTORY = new GroupByStrategyFactory();
 
-  private static GroupByColumnSelectorPlus[] createGroupBySelectorPlus(ColumnSelectorPlus<GroupByColumnSelectorStrategy>[] baseSelectorPlus)
+  private static GroupByColumnSelectorPlus[] createGroupBySelectorPlus(
+      ColumnSelectorPlus<GroupByColumnSelectorStrategy>[] baseSelectorPlus,
+      int dimensionStart
+  )
   {
     GroupByColumnSelectorPlus[] retInfo = new GroupByColumnSelectorPlus[baseSelectorPlus.length];
     int curPos = 0;
     for (int i = 0; i < retInfo.length; i++) {
-      retInfo[i] = new GroupByColumnSelectorPlus(baseSelectorPlus[i], curPos);
+      retInfo[i] = new GroupByColumnSelectorPlus(baseSelectorPlus[i], curPos, dimensionStart + i);
       curPos += retInfo[i].getColumnSelectorStrategy().getGroupingKeySize();
     }
     return retInfo;
@@ -95,9 +105,9 @@ public class GroupByQueryEngineV2
     // No instantiation
   }
 
-  public static Sequence<Row> process(
+  public static Sequence<ResultRow> process(
       final GroupByQuery query,
-      final StorageAdapter storageAdapter,
+      @Nullable final StorageAdapter storageAdapter,
       final NonBlockingPool<ByteBuffer> intermediateResultsBufferPool,
       final GroupByQueryConfig querySpecificConfig
   )
@@ -130,7 +140,7 @@ public class GroupByQueryEngineV2
         VectorGroupByEngine.canVectorize(query, storageAdapter, filter)
     );
 
-    final Sequence<Row> result;
+    final Sequence<ResultRow> result;
 
     if (doVectorize) {
       result = VectorGroupByEngine.process(
@@ -157,7 +167,7 @@ public class GroupByQueryEngineV2
     return result.withBaggage(bufferHolder);
   }
 
-  private static Sequence<Row> processNonVectorized(
+  private static Sequence<ResultRow> processNonVectorized(
       final GroupByQuery query,
       final StorageAdapter storageAdapter,
       final ByteBuffer processingBuffer,
@@ -178,7 +188,7 @@ public class GroupByQueryEngineV2
 
     return cursors.flatMap(
         cursor -> new BaseSequence<>(
-            new BaseSequence.IteratorMaker<Row, GroupByEngineIterator<?>>()
+            new BaseSequence.IteratorMaker<ResultRow, GroupByEngineIterator<?>>()
             {
               @Override
               public GroupByEngineIterator make()
@@ -190,8 +200,10 @@ public class GroupByQueryEngineV2
                         query.getDimensions(),
                         columnSelectorFactory
                     );
-
-                final GroupByColumnSelectorPlus[] dims = createGroupBySelectorPlus(selectorPlus);
+                final GroupByColumnSelectorPlus[] dims = createGroupBySelectorPlus(
+                    selectorPlus,
+                    query.getResultRowDimensionStart()
+                );
 
                 final int cardinalityForArrayAggregation = getCardinalityForArrayAggregation(
                     querySpecificConfig,
@@ -353,7 +365,7 @@ public class GroupByQueryEngineV2
     }
   }
 
-  private abstract static class GroupByEngineIterator<KeyType> implements Iterator<Row>, Closeable
+  private abstract static class GroupByEngineIterator<KeyType> implements Iterator<ResultRow>, Closeable
   {
     protected final GroupByQuery query;
     protected final GroupByQueryConfig querySpecificConfig;
@@ -364,7 +376,7 @@ public class GroupByQueryEngineV2
     protected final DateTime timestamp;
 
     @Nullable
-    protected CloseableGrouperIterator<KeyType, Row> delegate = null;
+    protected CloseableGrouperIterator<KeyType, ResultRow> delegate = null;
     protected final boolean allSingleValueDims;
 
     public GroupByEngineIterator(
@@ -389,7 +401,7 @@ public class GroupByQueryEngineV2
       this.allSingleValueDims = allSingleValueDims;
     }
 
-    private CloseableGrouperIterator<KeyType, Row> initNewDelegate()
+    private CloseableGrouperIterator<KeyType, ResultRow> initNewDelegate()
     {
       final Grouper<KeyType> grouper = newGrouper();
       grouper.init();
@@ -400,29 +412,37 @@ public class GroupByQueryEngineV2
         aggregateMultiValueDims(grouper);
       }
 
+      final boolean resultRowHasTimestamp = query.getResultRowHasTimestamp();
+      final int resultRowDimensionStart = query.getResultRowDimensionStart();
+      final int resultRowAggregatorStart = query.getResultRowAggregatorStart();
+
       return new CloseableGrouperIterator<>(
           grouper.iterator(false),
           entry -> {
-            Map<String, Object> theMap = Maps.newLinkedHashMap();
+            final ResultRow resultRow = ResultRow.create(query.getResultRowSizeWithoutPostAggregators());
 
-            // Add dimensions.
-            putToMap(entry.getKey(), theMap);
+            // Add timestamp, if necessary.
+            if (resultRowHasTimestamp) {
+              resultRow.set(0, timestamp.getMillis());
+            }
 
-            convertRowTypesToOutputTypes(query.getDimensions(), theMap);
+            // Add dimensions, and convert their types if necessary.
+            putToRow(entry.getKey(), resultRow);
+            convertRowTypesToOutputTypes(query.getDimensions(), resultRow, resultRowDimensionStart);
 
             // Add aggregations.
             for (int i = 0; i < entry.getValues().length; i++) {
-              theMap.put(query.getAggregatorSpecs().get(i).getName(), entry.getValues()[i]);
+              resultRow.set(resultRowAggregatorStart + i, entry.getValues()[i]);
             }
 
-            return new MapBasedRow(timestamp, theMap);
+            return resultRow;
           },
           grouper
       );
     }
 
     @Override
-    public Row next()
+    public ResultRow next()
     {
       if (delegate == null || !delegate.hasNext()) {
         throw new NoSuchElementException();
@@ -481,10 +501,10 @@ public class GroupByQueryEngineV2
     protected abstract void aggregateMultiValueDims(Grouper<KeyType> grouper);
 
     /**
-     * Add the key to the result map.  Some pre-processing like deserialization might be done for the key before
+     * Add the key to the result row.  Some pre-processing like deserialization might be done for the key before
      * adding to the map.
      */
-    protected abstract void putToMap(KeyType key, Map<String, Object> map);
+    protected abstract void putToRow(KeyType key, ResultRow resultRow);
 
     protected int getSingleValue(IndexedInts indexedInts)
     {
@@ -633,13 +653,13 @@ public class GroupByQueryEngineV2
     }
 
     @Override
-    protected void putToMap(ByteBuffer key, Map<String, Object> map)
+    protected void putToRow(ByteBuffer key, ResultRow resultRow)
     {
       for (GroupByColumnSelectorPlus selectorPlus : dims) {
         selectorPlus.getColumnSelectorStrategy().processValueFromGroupingKey(
             selectorPlus,
             key,
-            map,
+            resultRow,
             selectorPlus.getKeyBufferPosition()
         );
       }
@@ -653,6 +673,7 @@ public class GroupByQueryEngineV2
     @Nullable
     private final GroupByColumnSelectorPlus dim;
 
+    @Nullable
     private IndexedInts multiValues;
     private int nextValIndex;
 
@@ -754,28 +775,32 @@ public class GroupByQueryEngineV2
     }
 
     @Override
-    protected void putToMap(Integer key, Map<String, Object> map)
+    protected void putToRow(Integer key, ResultRow resultRow)
     {
       if (dim != null) {
         if (key != GroupByColumnSelectorStrategy.GROUP_BY_MISSING_VALUE) {
-          map.put(
-              dim.getOutputName(),
-              ((DimensionSelector) dim.getSelector()).lookupName(key)
-          );
+          resultRow.set(dim.getResultRowPosition(), ((DimensionSelector) dim.getSelector()).lookupName(key));
         } else {
-          map.put(dim.getOutputName(), NullHandling.defaultStringValue());
+          resultRow.set(dim.getResultRowPosition(), NullHandling.defaultStringValue());
         }
       }
     }
   }
 
-  public static void convertRowTypesToOutputTypes(List<DimensionSpec> dimensionSpecs, Map<String, Object> rowMap)
+  public static void convertRowTypesToOutputTypes(
+      final List<DimensionSpec> dimensionSpecs,
+      final ResultRow resultRow,
+      final int resultRowDimensionStart
+  )
   {
-    for (DimensionSpec dimSpec : dimensionSpecs) {
+    for (int i = 0; i < dimensionSpecs.size(); i++) {
+      DimensionSpec dimSpec = dimensionSpecs.get(i);
+      final int resultRowIndex = resultRowDimensionStart + i;
       final ValueType outputType = dimSpec.getOutputType();
-      rowMap.compute(
-          dimSpec.getOutputName(),
-          (dimName, baseVal) -> DimensionHandlerUtils.convertObjectToType(baseVal, outputType)
+
+      resultRow.set(
+          resultRowIndex,
+          DimensionHandlerUtils.convertObjectToType(resultRow.get(resultRowIndex), outputType)
       );
     }
   }
@@ -784,7 +809,7 @@ public class GroupByQueryEngineV2
   {
     private final int keySize;
 
-    public GroupByEngineKeySerde(final GroupByColumnSelectorPlus dims[])
+    public GroupByEngineKeySerde(final GroupByColumnSelectorPlus[] dims)
     {
       int keySize = 0;
       for (GroupByColumnSelectorPlus selectorPlus : dims) {
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByRowProcessor.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByRowProcessor.java
index 2fa3b57..c1ac966 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByRowProcessor.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByRowProcessor.java
@@ -21,73 +21,87 @@ package org.apache.druid.query.groupby.epinephelinae;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
 import org.apache.druid.collections.ResourceHolder;
-import org.apache.druid.common.guava.SettableSupplier;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.guava.Accumulator;
 import org.apache.druid.java.util.common.guava.BaseSequence;
-import org.apache.druid.java.util.common.guava.FilteredSequence;
+import org.apache.druid.java.util.common.guava.CloseQuietly;
 import org.apache.druid.java.util.common.guava.Sequence;
-import org.apache.druid.query.Query;
 import org.apache.druid.query.ResourceLimitExceededException;
-import org.apache.druid.query.aggregation.AggregatorFactory;
-import org.apache.druid.query.filter.Filter;
-import org.apache.druid.query.filter.ValueMatcher;
 import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
-import org.apache.druid.query.groupby.RowBasedColumnSelectorFactory;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper.RowBasedKey;
 import org.apache.druid.query.groupby.resource.GroupByQueryResource;
-import org.apache.druid.segment.column.ValueType;
-import org.apache.druid.segment.filter.BooleanValueMatcher;
-import org.apache.druid.segment.filter.Filters;
-import org.joda.time.DateTime;
-import org.joda.time.Interval;
 
+import javax.annotation.Nullable;
 import java.io.Closeable;
 import java.io.File;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 import java.util.UUID;
 
+/**
+ * Utility class that knows how to do higher-level groupBys: i.e. group a {@link Sequence} of {@link ResultRow}
+ * originating from a subquery. It uses a buffer provided by a {@link GroupByQueryResource}. The output rows may not
+ * be perfectly grouped and will not have PostAggregators applied, so they should be fed into
+ * {@link org.apache.druid.query.groupby.strategy.GroupByStrategy#mergeResults}.
+ *
+ * This class has two primary uses: processing nested groupBys, and processing subtotals.
+ *
+ * This class has some similarity to {@link GroupByMergingQueryRunnerV2}, but is different enough that it deserved to
+ * be its own class. Some common code between the two classes is in {@link RowBasedGrouperHelper}.
+ */
 public class GroupByRowProcessor
 {
-  public static Grouper createGrouper(
-      final Query queryParam,
-      final Sequence<Row> rows,
-      final Map<String, ValueType> rowSignature,
+  public interface ResultSupplier extends Closeable
+  {
+    /**
+     * Return a result sequence. Can be called any number of times. When the results are no longer needed,
+     * call {@link #close()} (but make sure any result sequences have been fully consumed first!).
+     *
+     * @param dimensionsToInclude list of dimensions to include, or null to include all dimensions. Used by processing
+     *                            of subtotals. If specified, the results will not necessarily be fully grouped.
+     */
+    Sequence<ResultRow> results(@Nullable List<String> dimensionsToInclude);
+  }
+
+  private GroupByRowProcessor()
+  {
+    // No instantiation
+  }
+
+  /**
+   * Process the input of sequence "rows" (output by "subquery") based on "query" and returns a {@link ResultSupplier}.
+   *
+   * In addition to grouping using dimensions and metrics, it will also apply filters (both DimFilter and interval
+   * filters).
+   *
+   * The input sequence is processed synchronously with the call to this method, and result iteration happens lazy upon
+   * calls to the {@link ResultSupplier}. Make sure to close it when you're done.
+   */
+  public static ResultSupplier process(
+      final GroupByQuery query,
+      final GroupByQuery subquery,
+      final Sequence<ResultRow> rows,
       final GroupByQueryConfig config,
       final GroupByQueryResource resource,
       final ObjectMapper spillMapper,
       final String processingTmpDir,
-      final int mergeBufferSize,
-      final List<Closeable> closeOnExit,
-      final boolean wasQueryPushedDown,
-      final boolean useVirtualizedColumnSelectorFactory
+      final int mergeBufferSize
   )
   {
-    final GroupByQuery query = (GroupByQuery) queryParam;
+    final List<Closeable> closeOnExit = new ArrayList<>();
     final GroupByQueryConfig querySpecificConfig = config.withOverrides(query);
 
-    final AggregatorFactory[] aggregatorFactories = new AggregatorFactory[query.getAggregatorSpecs().size()];
-    for (int i = 0; i < query.getAggregatorSpecs().size(); i++) {
-      aggregatorFactories[i] = query.getAggregatorSpecs().get(i);
-    }
-
     final File temporaryStorageDirectory = new File(
         processingTmpDir,
         StringUtils.format("druid-groupBy-%s_%s", UUID.randomUUID(), query.getId())
     );
 
-    Sequence<Row> sequenceToGroup = rows;
-    // When query is pushed down, rows have already been filtered
-    if (!wasQueryPushedDown) {
-      sequenceToGroup = getFilteredSequence(rows, rowSignature, query);
-    }
-
     final LimitedTemporaryStorage temporaryStorage = new LimitedTemporaryStorage(
         temporaryStorageDirectory,
         querySpecificConfig.getMaxOnDiskStorage()
@@ -95,10 +109,9 @@ public class GroupByRowProcessor
 
     closeOnExit.add(temporaryStorage);
 
-    Pair<Grouper<RowBasedKey>, Accumulator<AggregateResult, Row>> pair = RowBasedGrouperHelper.createGrouperAccumulatorPair(
+    Pair<Grouper<RowBasedKey>, Accumulator<AggregateResult, ResultRow>> pair = RowBasedGrouperHelper.createGrouperAccumulatorPair(
         query,
-        true,
-        rowSignature,
+        subquery,
         querySpecificConfig,
         new Supplier<ByteBuffer>()
         {
@@ -112,82 +125,56 @@ public class GroupByRowProcessor
         },
         temporaryStorage,
         spillMapper,
-        aggregatorFactories,
-        mergeBufferSize,
-        useVirtualizedColumnSelectorFactory
+        mergeBufferSize
     );
     final Grouper<RowBasedKey> grouper = pair.lhs;
-    final Accumulator<AggregateResult, Row> accumulator = pair.rhs;
+    final Accumulator<AggregateResult, ResultRow> accumulator = pair.rhs;
     closeOnExit.add(grouper);
 
-    final AggregateResult retVal = sequenceToGroup.accumulate(AggregateResult.ok(), accumulator);
+    final AggregateResult retVal = rows.accumulate(AggregateResult.ok(), accumulator);
 
     if (!retVal.isOk()) {
       throw new ResourceLimitExceededException(retVal.getReason());
     }
 
-    return grouper;
+    return new ResultSupplier()
+    {
+      @Override
+      public Sequence<ResultRow> results(@Nullable List<String> dimensionsToInclude)
+      {
+        return getRowsFromGrouper(query, grouper, dimensionsToInclude);
+      }
+
+      @Override
+      public void close()
+      {
+        Lists.reverse(closeOnExit).forEach(CloseQuietly::close);
+      }
+    };
   }
 
-  private static Sequence<Row> getFilteredSequence(
-      Sequence<Row> rows,
-      Map<String, ValueType> rowSignature,
-      GroupByQuery query
+  private static Sequence<ResultRow> getRowsFromGrouper(
+      final GroupByQuery query,
+      final Grouper<RowBasedKey> grouper,
+      @Nullable List<String> dimensionsToInclude
   )
   {
-    final List<Interval> queryIntervals = query.getIntervals();
-    final Filter filter = Filters.convertToCNFFromQueryContext(
-        query,
-        Filters.toFilter(query.getDimFilter())
-    );
-
-    final SettableSupplier<Row> rowSupplier = new SettableSupplier<>();
-    final RowBasedColumnSelectorFactory columnSelectorFactory = RowBasedColumnSelectorFactory.create(
-        rowSupplier,
-        rowSignature
-    );
-    final ValueMatcher filterMatcher = filter == null
-                                       ? BooleanValueMatcher.of(true)
-                                       : filter.makeMatcher(columnSelectorFactory);
-
-    return new FilteredSequence<>(
-        rows,
-        input -> {
-          boolean inInterval = false;
-          DateTime rowTime = input.getTimestamp();
-          for (Interval queryInterval : queryIntervals) {
-            if (queryInterval.contains(rowTime)) {
-              inInterval = true;
-              break;
-            }
-          }
-          if (!inInterval) {
-            return false;
-          }
-          rowSupplier.set(input);
-          return filterMatcher.matches();
-        }
-    );
-  }
-
-  public static Sequence<Row> getRowsFromGrouper(GroupByQuery query, List<String> subtotalSpec, Supplier<Grouper> grouper)
-  {
     return new BaseSequence<>(
-        new BaseSequence.IteratorMaker<Row, CloseableGrouperIterator<RowBasedKey, Row>>()
+        new BaseSequence.IteratorMaker<ResultRow, CloseableGrouperIterator<RowBasedKey, ResultRow>>()
         {
           @Override
-          public CloseableGrouperIterator<RowBasedKey, Row> make()
+          public CloseableGrouperIterator<RowBasedKey, ResultRow> make()
           {
             return RowBasedGrouperHelper.makeGrouperIterator(
-                grouper.get(),
+                grouper,
                 query,
-                subtotalSpec,
+                dimensionsToInclude,
                 () -> {}
             );
           }
 
           @Override
-          public void cleanup(CloseableGrouperIterator<RowBasedKey, Row> iterFromMake)
+          public void cleanup(CloseableGrouperIterator<RowBasedKey, ResultRow> iterFromMake)
           {
             iterFromMake.close();
           }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java
index 2c75c35..9315b77 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java
@@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonValue;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
-import com.google.common.collect.Maps;
 import com.google.common.primitives.Ints;
 import com.google.common.primitives.Longs;
 import com.google.common.util.concurrent.ListeningExecutorService;
@@ -33,9 +32,9 @@ import it.unimi.dsi.fastutil.objects.Object2IntMap;
 import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
 import org.apache.druid.collections.ReferenceCountingResourceHolder;
 import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.common.guava.SettableSupplier;
 import org.apache.druid.common.utils.IntArrayUtils;
-import org.apache.druid.data.input.MapBasedRow;
-import org.apache.druid.data.input.Row;
+import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.Pair;
@@ -48,13 +47,16 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.dimension.ColumnSelectorStrategy;
 import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
 import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.filter.Filter;
+import org.apache.druid.query.filter.ValueMatcher;
 import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
+import org.apache.druid.query.groupby.GroupByQueryHelper;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.RowBasedColumnSelectorFactory;
 import org.apache.druid.query.groupby.epinephelinae.Grouper.BufferComparator;
 import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
 import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
-import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
 import org.apache.druid.query.ordering.StringComparator;
 import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.segment.BaseDoubleColumnValueSelector;
@@ -67,23 +69,29 @@ import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.column.ColumnCapabilities;
 import org.apache.druid.segment.column.ValueType;
 import org.apache.druid.segment.data.IndexedInts;
+import org.apache.druid.segment.filter.BooleanValueMatcher;
+import org.apache.druid.segment.filter.Filters;
 import org.joda.time.DateTime;
+import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
 import java.io.Closeable;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.BitSet;
 import java.util.Comparator;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.ToLongFunction;
 import java.util.stream.IntStream;
 
-// this class contains shared code between GroupByMergingQueryRunnerV2 and GroupByRowProcessor
+/**
+ * This class contains shared code between {@link GroupByMergingQueryRunnerV2} and {@link GroupByRowProcessor}.
+ */
 public class RowBasedGrouperHelper
 {
   // Entry in dictionary, node pointer in reverseDictionary, hash + k/v/next pointer in reverseDictionary nodes
@@ -93,89 +101,111 @@ public class RowBasedGrouperHelper
   private static final int UNKNOWN_THREAD_PRIORITY = -1;
   private static final long UNKNOWN_TIMEOUT = -1L;
 
+  private RowBasedGrouperHelper()
+  {
+    // No instantiation.
+  }
+
   /**
    * Create a single-threaded grouper and accumulator.
    */
-  public static Pair<Grouper<RowBasedKey>, Accumulator<AggregateResult, Row>> createGrouperAccumulatorPair(
+  public static Pair<Grouper<RowBasedKey>, Accumulator<AggregateResult, ResultRow>> createGrouperAccumulatorPair(
       final GroupByQuery query,
-      final boolean isInputRaw,
-      final Map<String, ValueType> rawInputRowSignature,
+      @Nullable final GroupByQuery subquery,
       final GroupByQueryConfig config,
       final Supplier<ByteBuffer> bufferSupplier,
       final LimitedTemporaryStorage temporaryStorage,
       final ObjectMapper spillMapper,
-      final AggregatorFactory[] aggregatorFactories,
-      final int mergeBufferSize,
-      final boolean useVirtualizedColumnSelectorFactory
+      final int mergeBufferSize
   )
   {
     return createGrouperAccumulatorPair(
         query,
-        isInputRaw,
-        rawInputRowSignature,
+        subquery,
         config,
         bufferSupplier,
         null,
         SINGLE_THREAD_CONCURRENCY_HINT,
         temporaryStorage,
         spillMapper,
-        aggregatorFactories,
         null,
         UNKNOWN_THREAD_PRIORITY,
         false,
         UNKNOWN_TIMEOUT,
-        mergeBufferSize,
-        useVirtualizedColumnSelectorFactory
+        mergeBufferSize
     );
   }
 
   /**
-   * If isInputRaw is true, transformations such as timestamp truncation and extraction functions have not
-   * been applied to the input rows yet, for example, in a nested query, if an extraction function is being
-   * applied in the outer query to a field of the inner query. This method must apply those transformations.
+   * Create a {@link Grouper} that groups according to the dimensions and aggregators in "query", along with
+   * an {@link Accumulator} that accepts ResultRows and forwards them to the grouper.
+   *
+   * The pair will operate in one of two modes:
+   *
+   * 1) Combining mode (used if "subquery" is null). In this mode, filters from the "query" are ignored, and
+   * its aggregators are converted into combining form. The input ResultRows are assumed to be partially-grouped
+   * results originating from the provided "query".
+   *
+   * 2) Subquery mode (used if "subquery" is nonnull). In this mode, filters from the "query" (both intervals
+   * and dim filters) are respected, and its aggregators are used in standard (not combining) form. The input
+   * ResultRows are assumed to be results originating from the provided "subquery".
+   *
+   * @param query               query that we are grouping for
+   * @param subquery            optional subquery that we are receiving results from (see combining vs. subquery
+   *                            mode above)
+   * @param config              groupBy query config
+   * @param bufferSupplier      supplier of merge buffers
+   * @param combineBufferHolder holder of combine buffers. Unused if concurrencyHint = -1, and may be null in that case
+   * @param concurrencyHint     -1 for single-threaded Grouper, >=1 for concurrent Grouper
+   * @param temporaryStorage    temporary storage used for spilling from the Grouper
+   * @param spillMapper         object mapper used for spilling from the Grouper
+   * @param grouperSorter       executor service used for parallel combining. Unused if concurrencyHint = -1, and may
+   *                            be null in that case
+   * @param priority            query priority
+   * @param hasQueryTimeout     whether or not this query has a timeout
+   * @param queryTimeoutAt      when this query times out, in milliseconds since the epoch
+   * @param mergeBufferSize     size of the merge buffers from "bufferSupplier"
    */
-  public static Pair<Grouper<RowBasedKey>, Accumulator<AggregateResult, Row>> createGrouperAccumulatorPair(
+  public static Pair<Grouper<RowBasedKey>, Accumulator<AggregateResult, ResultRow>> createGrouperAccumulatorPair(
       final GroupByQuery query,
-      final boolean isInputRaw,
-      final Map<String, ValueType> rawInputRowSignature,
+      @Nullable final GroupByQuery subquery,
       final GroupByQueryConfig config,
       final Supplier<ByteBuffer> bufferSupplier,
       @Nullable final ReferenceCountingResourceHolder<ByteBuffer> combineBufferHolder,
       final int concurrencyHint,
       final LimitedTemporaryStorage temporaryStorage,
       final ObjectMapper spillMapper,
-      final AggregatorFactory[] aggregatorFactories,
       @Nullable final ListeningExecutorService grouperSorter,
       final int priority,
       final boolean hasQueryTimeout,
       final long queryTimeoutAt,
-      final int mergeBufferSize,
-      final boolean useVirtualizedColumnSelectorFactory
+      final int mergeBufferSize
   )
   {
     // concurrencyHint >= 1 for concurrent groupers, -1 for single-threaded
     Preconditions.checkArgument(concurrencyHint >= 1 || concurrencyHint == -1, "invalid concurrencyHint");
 
+    if (concurrencyHint >= 1) {
+      Preconditions.checkNotNull(grouperSorter, "grouperSorter executor must be provided");
+    }
+
+    // See method-level javadoc; we go into combining mode if there is no subquery.
+    final boolean combining = subquery == null;
+
     final List<ValueType> valueTypes = DimensionHandlerUtils.getValueTypesFromDimensionSpecs(query.getDimensions());
 
     final GroupByQueryConfig querySpecificConfig = config.withOverrides(query);
-    final boolean includeTimestamp = GroupByStrategyV2.getUniversalTimestamp(query) == null;
+    final boolean includeTimestamp = query.getResultRowHasTimestamp();
 
-    final ThreadLocal<Row> columnSelectorRow = new ThreadLocal<>();
+    final ThreadLocal<ResultRow> columnSelectorRow = new ThreadLocal<>();
 
-    ColumnSelectorFactory columnSelectorFactory = RowBasedColumnSelectorFactory.create(
-        columnSelectorRow,
-        rawInputRowSignature
+    ColumnSelectorFactory columnSelectorFactory = createResultRowBasedColumnSelectorFactory(
+        combining ? query : subquery,
+        columnSelectorRow::get
     );
 
-    // Although queries would work fine if we always wrap the columnSelectorFactory into a
-    // VirtualizedColumnSelectorFactory. However, VirtualizedColumnSelectorFactory is incapable of using
-    // ColumnSelector based variants of makeXXX methods which are more efficient.
-    // this flag is set to true when it is essential to wrap e.g. a nested groupBy query with virtual columns in
-    // the outer query. Without this flag, groupBy query processing would never use more efficient ColumnSelector
-    // based methods in VirtualColumn interface.
-    // For more details, See https://github.com/apache/incubator-druid/issues/7574
-    if (useVirtualizedColumnSelectorFactory) {
+    // Apply virtual columns if we are in subquery (non-combining) mode.
+    if (!combining) {
       columnSelectorFactory = query.getVirtualColumns().wrap(columnSelectorFactory);
     }
 
@@ -189,6 +219,17 @@ public class RowBasedGrouperHelper
       );
     }
 
+    final AggregatorFactory[] aggregatorFactories;
+
+    if (combining) {
+      aggregatorFactories = query.getAggregatorSpecs()
+                                 .stream()
+                                 .map(AggregatorFactory::getCombiningFactory)
+                                 .toArray(AggregatorFactory[]::new);
+    } else {
+      aggregatorFactories = query.getAggregatorSpecs().toArray(new AggregatorFactory[0]);
+    }
+
     final Grouper.KeySerdeFactory<RowBasedKey> keySerdeFactory = new RowBasedKeySerdeFactory(
         includeTimestamp,
         query.getContextSortByDimsFirst(),
@@ -250,13 +291,22 @@ public class RowBasedGrouperHelper
     final int keySize = includeTimestamp ? query.getDimensions().size() + 1 : query.getDimensions().size();
     final ValueExtractFunction valueExtractFn = makeValueExtractFunction(
         query,
-        isInputRaw,
+        combining,
         includeTimestamp,
         columnSelectorFactory,
         valueTypes
     );
 
-    final Accumulator<AggregateResult, Row> accumulator = (priorResult, row) -> {
+    final Predicate<ResultRow> rowPredicate;
+
+    if (combining) {
+      // Filters are not applied in combining mode.
+      rowPredicate = row -> true;
+    } else {
+      rowPredicate = getResultRowPredicate(query, subquery);
+    }
+
+    final Accumulator<AggregateResult, ResultRow> accumulator = (priorResult, row) -> {
       BaseQuery.checkInterrupted();
 
       if (priorResult != null && !priorResult.isOk()) {
@@ -268,6 +318,10 @@ public class RowBasedGrouperHelper
         grouper.init();
       }
 
+      if (!rowPredicate.test(row)) {
+        return AggregateResult.ok();
+      }
+
       columnSelectorRow.set(row);
 
       final Comparable[] key = new Comparable[keySize];
@@ -282,47 +336,140 @@ public class RowBasedGrouperHelper
     return new Pair<>(grouper, accumulator);
   }
 
+  /**
+   * Creates a {@link ColumnSelectorFactory} that can read rows which originate as results of the provided "query".
+   *
+   * @param query    a groupBy query
+   * @param supplier supplier of result rows from the query
+   */
+  public static ColumnSelectorFactory createResultRowBasedColumnSelectorFactory(
+      final GroupByQuery query,
+      final Supplier<ResultRow> supplier
+  )
+  {
+    final RowBasedColumnSelectorFactory.RowAdapter<ResultRow> adapter =
+        new RowBasedColumnSelectorFactory.RowAdapter<ResultRow>()
+        {
+          @Override
+          public ToLongFunction<ResultRow> timestampFunction()
+          {
+            if (query.getResultRowHasTimestamp()) {
+              return row -> row.getLong(0);
+            } else {
+              final long timestamp = query.getUniversalTimestamp().getMillis();
+              return row -> timestamp;
+            }
+          }
+
+          @Override
+          public Function<ResultRow, Object> rawFunction(final String columnName)
+          {
+            final int columnIndex = query.getResultRowPositionLookup().getInt(columnName);
+            if (columnIndex < 0) {
+              return row -> null;
+            } else {
+              return row -> row.get(columnIndex);
+            }
+          }
+        };
+
+    return RowBasedColumnSelectorFactory.create(adapter, supplier::get, GroupByQueryHelper.rowSignatureFor(query));
+  }
+
+  /**
+   * Returns a predicate that filters result rows from a particular "subquery" based on the intervals and dim filters
+   * from "query".
+   *
+   * @param query    outer query
+   * @param subquery inner query
+   */
+  private static Predicate<ResultRow> getResultRowPredicate(final GroupByQuery query, final GroupByQuery subquery)
+  {
+    final List<Interval> queryIntervals = query.getIntervals();
+    final Filter filter = Filters.convertToCNFFromQueryContext(
+        query,
+        Filters.toFilter(query.getDimFilter())
+    );
+
+    final SettableSupplier<ResultRow> rowSupplier = new SettableSupplier<>();
+    final ColumnSelectorFactory columnSelectorFactory =
+        RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory(subquery, rowSupplier);
+
+    final ValueMatcher filterMatcher = filter == null
+                                       ? BooleanValueMatcher.of(true)
+                                       : filter.makeMatcher(columnSelectorFactory);
+
+    if (subquery.getUniversalTimestamp() != null
+        && queryIntervals.stream().noneMatch(itvl -> itvl.contains(subquery.getUniversalTimestamp()))) {
+      // There's a universal timestamp, and it doesn't match our query intervals, so no row should match.
+      // By the way, if there's a universal timestamp that _does_ match the query intervals, we do nothing special here.
+      return row -> false;
+    }
+
+    return row -> {
+      if (subquery.getResultRowHasTimestamp()) {
+        boolean inInterval = false;
+        for (Interval queryInterval : queryIntervals) {
+          if (queryInterval.contains(row.getLong(0))) {
+            inInterval = true;
+            break;
+          }
+        }
+        if (!inInterval) {
+          return false;
+        }
+      }
+      rowSupplier.set(row);
+      return filterMatcher.matches();
+    };
+  }
+
   private interface TimestampExtractFunction
   {
-    long apply(Row row);
+    long apply(ResultRow row);
   }
 
   private static TimestampExtractFunction makeTimestampExtractFunction(
       final GroupByQuery query,
-      final boolean isInputRaw
+      final boolean combining
   )
   {
-    if (isInputRaw) {
-      if (query.getGranularity() instanceof AllGranularity) {
-        return row -> query.getIntervals().get(0).getStartMillis();
+    if (query.getResultRowHasTimestamp()) {
+      if (combining) {
+        return row -> row.getLong(0);
       } else {
-        return row -> query.getGranularity().bucketStart(row.getTimestamp()).getMillis();
+        if (query.getGranularity() instanceof AllGranularity) {
+          return row -> query.getIntervals().get(0).getStartMillis();
+        } else {
+          return row -> query.getGranularity().bucketStart(DateTimes.utc(row.getLong(0))).getMillis();
+        }
       }
     } else {
-      return Row::getTimestampFromEpoch;
+      final long timestamp = query.getUniversalTimestamp().getMillis();
+      return row -> timestamp;
     }
   }
 
   private interface ValueExtractFunction
   {
-    Comparable[] apply(Row row, Comparable[] key);
+    Comparable[] apply(ResultRow row, Comparable[] key);
   }
 
   private static ValueExtractFunction makeValueExtractFunction(
       final GroupByQuery query,
-      final boolean isInputRaw,
+      final boolean combining,
       final boolean includeTimestamp,
       final ColumnSelectorFactory columnSelectorFactory,
       final List<ValueType> valueTypes
   )
   {
     final TimestampExtractFunction timestampExtractFn = includeTimestamp ?
-                                                        makeTimestampExtractFunction(query, isInputRaw) :
+                                                        makeTimestampExtractFunction(query, combining) :
                                                         null;
 
     final Function<Comparable, Comparable>[] valueConvertFns = makeValueConvertFunctions(valueTypes);
 
-    if (isInputRaw) {
+    if (!combining) {
       final Supplier<Comparable>[] inputRawSuppliers = getValueSuppliersForDimensions(
           columnSelectorFactory,
           query.getDimensions()
@@ -347,11 +494,13 @@ public class RowBasedGrouperHelper
         };
       }
     } else {
+      final int dimensionStartPosition = query.getResultRowDimensionStart();
+
       if (includeTimestamp) {
         return (row, key) -> {
           key[0] = timestampExtractFn.apply(row);
           for (int i = 1; i < key.length; i++) {
-            final Comparable val = (Comparable) row.getRaw(query.getDimensions().get(i - 1).getOutputName());
+            final Comparable val = (Comparable) row.get(dimensionStartPosition + i - 1);
             key[i] = valueConvertFns[i - 1].apply(val);
           }
           return key;
@@ -359,7 +508,7 @@ public class RowBasedGrouperHelper
       } else {
         return (row, key) -> {
           for (int i = 0; i < key.length; i++) {
-            final Comparable val = (Comparable) row.getRaw(query.getDimensions().get(i).getOutputName());
+            final Comparable val = (Comparable) row.get(dimensionStartPosition + i);
             key[i] = valueConvertFns[i].apply(val);
           }
           return key;
@@ -368,7 +517,7 @@ public class RowBasedGrouperHelper
     }
   }
 
-  public static CloseableGrouperIterator<RowBasedKey, Row> makeGrouperIterator(
+  public static CloseableGrouperIterator<RowBasedKey, ResultRow> makeGrouperIterator(
       final Grouper<RowBasedKey> grouper,
       final GroupByQuery query,
       final Closeable closeable
@@ -377,68 +526,61 @@ public class RowBasedGrouperHelper
     return makeGrouperIterator(grouper, query, null, closeable);
   }
 
-  public static CloseableGrouperIterator<RowBasedKey, Row> makeGrouperIterator(
+  public static CloseableGrouperIterator<RowBasedKey, ResultRow> makeGrouperIterator(
       final Grouper<RowBasedKey> grouper,
       final GroupByQuery query,
-      final List<String> dimsToInclude,
+      @Nullable final List<String> dimsToInclude,
       final Closeable closeable
   )
   {
-    final boolean includeTimestamp = GroupByStrategyV2.getUniversalTimestamp(query) == null;
+    final boolean includeTimestamp = query.getResultRowHasTimestamp();
+    final BitSet dimsToIncludeBitSet = new BitSet(query.getDimensions().size());
+    final int resultRowDimensionStart = query.getResultRowDimensionStart();
+
+    if (dimsToInclude != null) {
+      for (String dimension : dimsToInclude) {
+        final int dimIndex = query.getResultRowPositionLookup().getInt(dimension);
+        if (dimIndex >= 0) {
+          dimsToIncludeBitSet.set(dimIndex - resultRowDimensionStart);
+        }
+      }
+    }
 
     return new CloseableGrouperIterator<>(
         grouper.iterator(true),
         entry -> {
-          Map<String, Object> theMap = Maps.newLinkedHashMap();
-
-          // Get timestamp, maybe.
-          final DateTime timestamp;
-          final int dimStart;
+          final ResultRow resultRow = ResultRow.create(query.getResultRowSizeWithoutPostAggregators());
 
+          // Add timestamp, maybe.
           if (includeTimestamp) {
-            timestamp = query.getGranularity().toDateTime(((long) (entry.getKey().getKey()[0])));
-            dimStart = 1;
-          } else {
-            timestamp = null;
-            dimStart = 0;
+            final DateTime timestamp = query.getGranularity().toDateTime(((long) (entry.getKey().getKey()[0])));
+            resultRow.set(0, timestamp.getMillis());
           }
 
           // Add dimensions.
-          if (dimsToInclude == null) {
-            for (int i = dimStart; i < entry.getKey().getKey().length; i++) {
-              Object dimVal = entry.getKey().getKey()[i];
-              theMap.put(
-                  query.getDimensions().get(i - dimStart).getOutputName(),
-                  dimVal instanceof String ? NullHandling.emptyToNullIfNeeded((String) dimVal) : dimVal
-              );
-            }
-          } else {
-            Map<String, Object> dimensions = new HashMap<>();
-            for (int i = dimStart; i < entry.getKey().getKey().length; i++) {
-              Object dimVal = entry.getKey().getKey()[i];
-              dimensions.put(
-                  query.getDimensions().get(i - dimStart).getOutputName(),
+          for (int i = resultRowDimensionStart; i < entry.getKey().getKey().length; i++) {
+            if (dimsToInclude == null || dimsToIncludeBitSet.get(i - resultRowDimensionStart)) {
+              final Object dimVal = entry.getKey().getKey()[i];
+              resultRow.set(
+                  i,
                   dimVal instanceof String ? NullHandling.emptyToNullIfNeeded((String) dimVal) : dimVal
               );
             }
-
-            for (String dimToInclude : dimsToInclude) {
-              theMap.put(dimToInclude, dimensions.get(dimToInclude));
-            }
           }
 
           // Add aggregations.
+          final int resultRowAggregatorStart = query.getResultRowAggregatorStart();
           for (int i = 0; i < entry.getValues().length; i++) {
-            theMap.put(query.getAggregatorSpecs().get(i).getName(), entry.getValues()[i]);
+            resultRow.set(resultRowAggregatorStart + i, entry.getValues()[i]);
           }
 
-          return new MapBasedRow(timestamp, theMap);
+          return resultRow;
         },
         closeable
     );
   }
 
-  static class RowBasedKey
+  public static class RowBasedKey
   {
     private final Object[] key;
 
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java
index 6cb9634..e1b6641 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java
@@ -22,6 +22,7 @@ package org.apache.druid.query.groupby.epinephelinae.column;
 import com.google.common.base.Preconditions;
 import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
 import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.data.ArrayBasedIndexedInts;
@@ -30,7 +31,6 @@ import org.apache.druid.segment.data.IndexedInts;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
 /**
  * A String strategy that builds an internal String<->Integer dictionary for
@@ -52,7 +52,7 @@ public class DictionaryBuildingStringGroupByColumnSelectorStrategy extends Strin
   public void processValueFromGroupingKey(
       GroupByColumnSelectorPlus selectorPlus,
       ByteBuffer key,
-      Map<String, Object> resultMap,
+      ResultRow resultRow,
       int keyBufferPosition
   )
   {
@@ -61,12 +61,9 @@ public class DictionaryBuildingStringGroupByColumnSelectorStrategy extends Strin
     // GROUP_BY_MISSING_VALUE is used to indicate empty rows, which are omitted from the result map.
     if (id != GROUP_BY_MISSING_VALUE) {
       final String value = dictionary.get(id);
-      resultMap.put(
-          selectorPlus.getOutputName(),
-          value
-      );
+      resultRow.set(selectorPlus.getResultRowPosition(), value);
     } else {
-      resultMap.put(selectorPlus.getOutputName(), NullHandling.defaultStringValue());
+      resultRow.set(selectorPlus.getResultRowPosition(), NullHandling.defaultStringValue());
     }
   }
 
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DoubleGroupByColumnSelectorStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DoubleGroupByColumnSelectorStrategy.java
index e8961e4..6b3e577 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DoubleGroupByColumnSelectorStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DoubleGroupByColumnSelectorStrategy.java
@@ -20,12 +20,12 @@
 package org.apache.druid.query.groupby.epinephelinae.column;
 
 
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.DimensionHandlerUtils;
 
 import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
-import java.util.Map;
 
 public class DoubleGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy
 {
@@ -39,12 +39,12 @@ public class DoubleGroupByColumnSelectorStrategy implements GroupByColumnSelecto
   public void processValueFromGroupingKey(
       GroupByColumnSelectorPlus selectorPlus,
       ByteBuffer key,
-      Map<String, Object> resultMap,
+      ResultRow resultRow,
       int keyBufferPosition
   )
   {
     final double val = key.getDouble(keyBufferPosition);
-    resultMap.put(selectorPlus.getOutputName(), val);
+    resultRow.set(selectorPlus.getResultRowPosition(), val);
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/FloatGroupByColumnSelectorStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/FloatGroupByColumnSelectorStrategy.java
index 52cd4b2..453c282 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/FloatGroupByColumnSelectorStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/FloatGroupByColumnSelectorStrategy.java
@@ -19,12 +19,12 @@
 
 package org.apache.druid.query.groupby.epinephelinae.column;
 
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.DimensionHandlerUtils;
 
 import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
-import java.util.Map;
 
 public class FloatGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy
 {
@@ -39,12 +39,12 @@ public class FloatGroupByColumnSelectorStrategy implements GroupByColumnSelector
   public void processValueFromGroupingKey(
       GroupByColumnSelectorPlus selectorPlus,
       ByteBuffer key,
-      Map<String, Object> resultMap,
+      ResultRow resultRow,
       int keyBufferPosition
   )
   {
     final float val = key.getFloat(keyBufferPosition);
-    resultMap.put(selectorPlus.getOutputName(), val);
+    resultRow.set(selectorPlus.getResultRowPosition(), val);
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java
index a272373..4eb0e95 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java
@@ -24,11 +24,20 @@ import org.apache.druid.query.ColumnSelectorPlus;
 public class GroupByColumnSelectorPlus extends ColumnSelectorPlus<GroupByColumnSelectorStrategy>
 {
   /**
+   * Indicates the offset of this dimension's value within ResultRows.
+   */
+  private final int resultRowPosition;
+
+  /**
    * Indicates the offset of this dimension's value within the grouping key.
    */
-  private int keyBufferPosition;
+  private final int keyBufferPosition;
 
-  public GroupByColumnSelectorPlus(ColumnSelectorPlus<GroupByColumnSelectorStrategy> baseInfo, int keyBufferPosition)
+  public GroupByColumnSelectorPlus(
+      ColumnSelectorPlus<GroupByColumnSelectorStrategy> baseInfo,
+      int keyBufferPosition,
+      int resultRowPosition
+  )
   {
     super(
         baseInfo.getName(),
@@ -37,10 +46,16 @@ public class GroupByColumnSelectorPlus extends ColumnSelectorPlus<GroupByColumnS
         baseInfo.getSelector()
     );
     this.keyBufferPosition = keyBufferPosition;
+    this.resultRowPosition = resultRowPosition;
   }
 
   public int getKeyBufferPosition()
   {
     return keyBufferPosition;
   }
+
+  public int getResultRowPosition()
+  {
+    return resultRowPosition;
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java
index 10172bc..9665541 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java
@@ -20,10 +20,10 @@
 package org.apache.druid.query.groupby.epinephelinae.column;
 
 import org.apache.druid.query.dimension.ColumnSelectorStrategy;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.ColumnValueSelector;
 
 import java.nio.ByteBuffer;
-import java.util.Map;
 
 /**
  * Contains a collection of query processing methods for type-specific operations used exclusively by
@@ -45,10 +45,10 @@ public interface GroupByColumnSelectorStrategy extends ColumnSelectorStrategy
   int getGroupingKeySize();
 
   /**
-   * Read a value from a grouping key and add it to the group by query result map, using the output name specified
+   * Read a value from a grouping key and add it to the group by query result row, using the output name specified
    * in a DimensionSpec.
    *
-   * An implementation may choose to not add anything to the result map
+   * An implementation may choose to not add anything to the result row
    * (e.g., as the String implementation does for empty rows)
    *
    * selectorPlus provides access to:
@@ -56,24 +56,24 @@ public interface GroupByColumnSelectorStrategy extends ColumnSelectorStrategy
    * - the dimension value selector
    * - the DimensionSpec for this dimension from the query
    *
-   * @param selectorPlus dimension info containing the key offset, value selector, and dimension spec
-   * @param resultMap result map for the group by query being served
-   * @param key grouping key
+   * @param selectorPlus      dimension info containing the key offset, value selector, and dimension spec
+   * @param resultRow         result row for the group by query being served
+   * @param key               grouping key
    * @param keyBufferPosition buffer position for the grouping key, added to support chaining multiple {@link ColumnSelectorStrategy}
    */
   void processValueFromGroupingKey(
       GroupByColumnSelectorPlus selectorPlus,
       ByteBuffer key,
-      Map<String, Object> resultMap,
+      ResultRow resultRow,
       int keyBufferPosition
   );
 
   /**
    * Retrieve a row object from the {@link ColumnValueSelector} and put it in valuess at columnIndex.
    *
-   * @param selector Value selector for a column.
+   * @param selector    Value selector for a column.
    * @param columnIndex Index of the column within the row values array
-   * @param valuess Row values array, one index per column
+   * @param valuess     Row values array, one index per column
    */
   void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess);
 
@@ -85,10 +85,10 @@ public interface GroupByColumnSelectorStrategy extends ColumnSelectorStrategy
    * If the size of the row is > 0, write 1 to stack[] at columnIndex, otherwise write 0.
    *
    * @param keyBufferPosition Starting offset for this column's value within the grouping key.
-   * @param columnIndex Index of the column within the row values array
-   * @param rowObj Row value object for this column
-   * @param keyBuffer grouping key
-   * @param stack array containing the current within-row value index for each column
+   * @param columnIndex       Index of the column within the row values array
+   * @param rowObj            Row value object for this column
+   * @param keyBuffer         grouping key
+   * @param stack             array containing the current within-row value index for each column
    */
   void initGroupingKeyColumnValue(
       int keyBufferPosition,
@@ -106,12 +106,18 @@ public interface GroupByColumnSelectorStrategy extends ColumnSelectorStrategy
    * Otherwise, return false.
    *
    * @param keyBufferPosition Starting offset for this column's value within the grouping key.
-   * @param rowObj Row value object for this column (e.g., IndexedInts)
-   * @param rowValIdx Index of the current value being grouped on within the row
-   * @param keyBuffer grouping key
+   * @param rowObj            Row value object for this column (e.g., IndexedInts)
+   * @param rowValIdx         Index of the current value being grouped on within the row
+   * @param keyBuffer         grouping key
+   *
    * @return true if rowValIdx < size of rowObj, false otherwise
    */
-  boolean checkRowIndexAndAddValueToGroupingKey(int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer);
+  boolean checkRowIndexAndAddValueToGroupingKey(
+      int keyBufferPosition,
+      Object rowObj,
+      int rowValIdx,
+      ByteBuffer keyBuffer
+  );
 
   /**
    * Retrieve a single object using the {@link ColumnValueSelector}.  The reading column must have a single value.
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/LongGroupByColumnSelectorStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/LongGroupByColumnSelectorStrategy.java
index 5afc7aa..3c5f8fe 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/LongGroupByColumnSelectorStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/LongGroupByColumnSelectorStrategy.java
@@ -19,12 +19,12 @@
 
 package org.apache.druid.query.groupby.epinephelinae.column;
 
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.DimensionHandlerUtils;
 
 import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
-import java.util.Map;
 
 public class LongGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy
 {
@@ -39,12 +39,12 @@ public class LongGroupByColumnSelectorStrategy implements GroupByColumnSelectorS
   public void processValueFromGroupingKey(
       GroupByColumnSelectorPlus selectorPlus,
       ByteBuffer key,
-      Map<String, Object> resultMap,
+      ResultRow resultRow,
       int keyBufferPosition
   )
   {
     final long val = key.getLong(keyBufferPosition);
-    resultMap.put(selectorPlus.getOutputName(), val);
+    resultRow.set(selectorPlus.getResultRowPosition(), val);
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/NullableValueGroupByColumnSelectorStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/NullableValueGroupByColumnSelectorStrategy.java
index f92ec94..f0cfaf1 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/NullableValueGroupByColumnSelectorStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/NullableValueGroupByColumnSelectorStrategy.java
@@ -21,11 +21,11 @@ package org.apache.druid.query.groupby.epinephelinae.column;
 
 
 import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.ColumnValueSelector;
 
 import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
-import java.util.Map;
 
 public class NullableValueGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy
 {
@@ -46,14 +46,14 @@ public class NullableValueGroupByColumnSelectorStrategy implements GroupByColumn
   public void processValueFromGroupingKey(
       GroupByColumnSelectorPlus selectorPlus,
       ByteBuffer key,
-      Map<String, Object> resultMap,
+      ResultRow resultRow,
       int keyBufferPosition
   )
   {
     if (key.get(keyBufferPosition) == NullHandling.IS_NULL_BYTE) {
-      resultMap.put(selectorPlus.getOutputName(), null);
+      resultRow.set(selectorPlus.getResultRowPosition(), null);
     } else {
-      delegate.processValueFromGroupingKey(selectorPlus, key, resultMap, keyBufferPosition + Byte.BYTES);
+      delegate.processValueFromGroupingKey(selectorPlus, key, resultRow, keyBufferPosition + Byte.BYTES);
     }
   }
 
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java
index e7a1f8e..65c4841 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java
@@ -21,12 +21,12 @@ package org.apache.druid.query.groupby.epinephelinae.column;
 
 import com.google.common.base.Preconditions;
 import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.data.IndexedInts;
 
 import java.nio.ByteBuffer;
-import java.util.Map;
 
 public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy
 {
@@ -40,7 +40,7 @@ public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelecto
   public void processValueFromGroupingKey(
       GroupByColumnSelectorPlus selectorPlus,
       ByteBuffer key,
-      Map<String, Object> resultMap,
+      ResultRow resultRow,
       int keyBufferPosition
   )
   {
@@ -48,12 +48,12 @@ public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelecto
 
     // GROUP_BY_MISSING_VALUE is used to indicate empty rows, which are omitted from the result map.
     if (id != GROUP_BY_MISSING_VALUE) {
-      resultMap.put(
-          selectorPlus.getOutputName(),
+      resultRow.set(
+          selectorPlus.getResultRowPosition(),
           ((DimensionSelector) selectorPlus.getSelector()).lookupName(id)
       );
     } else {
-      resultMap.put(selectorPlus.getOutputName(), NullHandling.defaultStringValue());
+      resultRow.set(selectorPlus.getResultRowPosition(), NullHandling.defaultStringValue());
     }
   }
 
@@ -81,7 +81,13 @@ public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelecto
   }
 
   @Override
-  public void initGroupingKeyColumnValue(int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack)
+  public void initGroupingKeyColumnValue(
+      int keyBufferPosition,
+      int columnIndex,
+      Object rowObj,
+      ByteBuffer keyBuffer,
+      int[] stack
+  )
   {
     IndexedInts row = (IndexedInts) rowObj;
     int rowSize = row.size();
@@ -91,7 +97,12 @@ public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelecto
   }
 
   @Override
-  public boolean checkRowIndexAndAddValueToGroupingKey(int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer)
+  public boolean checkRowIndexAndAddValueToGroupingKey(
+      int keyBufferPosition,
+      Object rowObj,
+      int rowValIdx,
+      ByteBuffer keyBuffer
+  )
   {
     IndexedInts row = (IndexedInts) rowObj;
     int rowSize = row.size();
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DoubleGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DoubleGroupByVectorColumnSelector.java
index 2802e3a..69f6c86 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DoubleGroupByVectorColumnSelector.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DoubleGroupByVectorColumnSelector.java
@@ -19,10 +19,10 @@
 
 package org.apache.druid.query.groupby.epinephelinae.vector;
 
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.vector.VectorValueSelector;
 
 import java.nio.ByteBuffer;
-import java.util.Map;
 
 public class DoubleGroupByVectorColumnSelector implements GroupByVectorColumnSelector
 {
@@ -59,13 +59,13 @@ public class DoubleGroupByVectorColumnSelector implements GroupByVectorColumnSel
 
   @Override
   public void writeKeyToResultRow(
-      final String outputName,
       final ByteBuffer keyBuffer,
       final int keyOffset,
-      final Map<String, Object> resultMap
+      final ResultRow resultRow,
+      final int resultRowPosition
   )
   {
     final double value = keyBuffer.getDouble(keyOffset * Integer.BYTES);
-    resultMap.put(outputName, value);
+    resultRow.set(resultRowPosition, value);
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/FloatGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/FloatGroupByVectorColumnSelector.java
index 5adbdb1..9b9d53a 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/FloatGroupByVectorColumnSelector.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/FloatGroupByVectorColumnSelector.java
@@ -19,10 +19,10 @@
 
 package org.apache.druid.query.groupby.epinephelinae.vector;
 
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.vector.VectorValueSelector;
 
 import java.nio.ByteBuffer;
-import java.util.Map;
 
 public class FloatGroupByVectorColumnSelector implements GroupByVectorColumnSelector
 {
@@ -57,13 +57,13 @@ public class FloatGroupByVectorColumnSelector implements GroupByVectorColumnSele
 
   @Override
   public void writeKeyToResultRow(
-      final String outputName,
       final ByteBuffer keyBuffer,
       final int keyOffset,
-      final Map<String, Object> resultMap
+      final ResultRow resultRow,
+      final int resultRowPosition
   )
   {
     final float value = Float.intBitsToFloat(keyBuffer.getInt(keyOffset * Integer.BYTES));
-    resultMap.put(outputName, value);
+    resultRow.set(resultRowPosition, value);
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java
index 3cc4153..087c778 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java
@@ -19,8 +19,9 @@
 
 package org.apache.druid.query.groupby.epinephelinae.vector;
 
+import org.apache.druid.query.groupby.ResultRow;
+
 import java.nio.ByteBuffer;
-import java.util.Map;
 
 public interface GroupByVectorColumnSelector
 {
@@ -29,9 +30,9 @@ public interface GroupByVectorColumnSelector
   void writeKeys(int[] keySpace, int keySize, int keyOffset, int startRow, int endRow);
 
   void writeKeyToResultRow(
-      String outputName,
       ByteBuffer keyBuffer,
       int keyOffset,
-      Map<String, Object> resultMap
+      ResultRow resultRow,
+      int resultRowPosition
   );
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/LongGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/LongGroupByVectorColumnSelector.java
index 6ddbd99..afacd22 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/LongGroupByVectorColumnSelector.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/LongGroupByVectorColumnSelector.java
@@ -19,10 +19,10 @@
 
 package org.apache.druid.query.groupby.epinephelinae.vector;
 
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.vector.VectorValueSelector;
 
 import java.nio.ByteBuffer;
-import java.util.Map;
 
 public class LongGroupByVectorColumnSelector implements GroupByVectorColumnSelector
 {
@@ -58,13 +58,13 @@ public class LongGroupByVectorColumnSelector implements GroupByVectorColumnSelec
 
   @Override
   public void writeKeyToResultRow(
-      final String outputName,
       final ByteBuffer keyBuffer,
       final int keyOffset,
-      final Map<String, Object> resultMap
+      final ResultRow resultRow,
+      final int resultRowPosition
   )
   {
     final long value = keyBuffer.getLong(keyOffset * Integer.BYTES);
-    resultMap.put(outputName, value);
+    resultRow.set(resultRowPosition, value);
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java
index 6a9b428..f63f9cc 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java
@@ -19,10 +19,10 @@
 
 package org.apache.druid.query.groupby.epinephelinae.vector;
 
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
 
 import java.nio.ByteBuffer;
-import java.util.Map;
 
 public class SingleValueStringGroupByVectorColumnSelector implements GroupByVectorColumnSelector
 {
@@ -57,13 +57,13 @@ public class SingleValueStringGroupByVectorColumnSelector implements GroupByVect
 
   @Override
   public void writeKeyToResultRow(
-      final String outputName,
       final ByteBuffer keyBuffer,
       final int keyOffset,
-      final Map<String, Object> resultMap
+      final ResultRow resultRow,
+      final int resultRowPosition
   )
   {
     final int id = keyBuffer.getInt(keyOffset * Integer.BYTES);
-    resultMap.put(outputName, selector.lookupName(id));
+    resultRow.set(resultRowPosition, selector.lookupName(id));
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java
index 96b9988..6d76e0c 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java
@@ -20,8 +20,6 @@
 package org.apache.druid.query.groupby.epinephelinae.vector;
 
 import com.google.common.base.Suppliers;
-import org.apache.druid.data.input.MapBasedRow;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.guava.BaseSequence;
 import org.apache.druid.java.util.common.guava.Sequence;
@@ -33,6 +31,7 @@ import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.epinephelinae.AggregateResult;
 import org.apache.druid.query.groupby.epinephelinae.BufferArrayGrouper;
 import org.apache.druid.query.groupby.epinephelinae.BufferHashGrouper;
@@ -55,9 +54,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.Iterator;
-import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.stream.Collectors;
 
@@ -92,7 +89,7 @@ public class VectorGroupByEngine
            && adapter.canVectorize(filter, query.getVirtualColumns(), false);
   }
 
-  public static Sequence<Row> process(
+  public static Sequence<ResultRow> process(
       final GroupByQuery query,
       final StorageAdapter storageAdapter,
       final ByteBuffer processingBuffer,
@@ -107,10 +104,10 @@ public class VectorGroupByEngine
     }
 
     return new BaseSequence<>(
-        new BaseSequence.IteratorMaker<Row, CloseableIterator<Row>>()
+        new BaseSequence.IteratorMaker<ResultRow, CloseableIterator<ResultRow>>()
         {
           @Override
-          public CloseableIterator<Row> make()
+          public CloseableIterator<ResultRow> make()
           {
             final VectorCursor cursor = storageAdapter.makeVectorCursor(
                 Filters.toFilter(query.getDimFilter()),
@@ -123,7 +120,7 @@ public class VectorGroupByEngine
 
             if (cursor == null) {
               // Return empty iterator.
-              return new CloseableIterator<Row>()
+              return new CloseableIterator<ResultRow>()
               {
                 @Override
                 public boolean hasNext()
@@ -132,7 +129,7 @@ public class VectorGroupByEngine
                 }
 
                 @Override
-                public Row next()
+                public ResultRow next()
                 {
                   throw new NoSuchElementException();
                 }
@@ -179,7 +176,7 @@ public class VectorGroupByEngine
           }
 
           @Override
-          public void cleanup(CloseableIterator<Row> iterFromMake)
+          public void cleanup(CloseableIterator<ResultRow> iterFromMake)
           {
             try {
               iterFromMake.close();
@@ -192,7 +189,7 @@ public class VectorGroupByEngine
     );
   }
 
-  private static class VectorGroupByEngineIterator implements CloseableIterator<Row>
+  private static class VectorGroupByEngineIterator implements CloseableIterator<ResultRow>
   {
     private final GroupByQuery query;
     private final GroupByQueryConfig querySpecificConfig;
@@ -218,7 +215,7 @@ public class VectorGroupByEngine
     private int partiallyAggregatedRows = -1;
 
     @Nullable
-    private CloseableGrouperIterator<ByteBuffer, Row> delegate = null;
+    private CloseableGrouperIterator<ByteBuffer, ResultRow> delegate = null;
 
     VectorGroupByEngineIterator(
         final GroupByQuery query,
@@ -254,7 +251,7 @@ public class VectorGroupByEngine
     }
 
     @Override
-    public Row next()
+    public ResultRow next()
     {
       if (delegate == null || !delegate.hasNext()) {
         throw new NoSuchElementException();
@@ -343,7 +340,7 @@ public class VectorGroupByEngine
       return grouper;
     }
 
-    private CloseableGrouperIterator<ByteBuffer, Row> initNewDelegate()
+    private CloseableGrouperIterator<ByteBuffer, ResultRow> initNewDelegate()
     {
       // Method must not be called unless there's a current bucketInterval.
       assert bucketInterval != null;
@@ -399,10 +396,19 @@ public class VectorGroupByEngine
         }
       }
 
+      final boolean resultRowHasTimestamp = query.getResultRowHasTimestamp();
+      final int resultRowDimensionStart = query.getResultRowDimensionStart();
+      final int resultRowAggregatorStart = query.getResultRowAggregatorStart();
+
       return new CloseableGrouperIterator<>(
           vectorGrouper.iterator(),
           entry -> {
-            Map<String, Object> theMap = new LinkedHashMap<>();
+            final ResultRow resultRow = ResultRow.create(query.getResultRowSizeWithoutPostAggregators());
+
+            // Add timestamp, if necessary.
+            if (resultRowHasTimestamp) {
+              resultRow.set(0, timestamp.getMillis());
+            }
 
             // Add dimensions.
             int keyOffset = 0;
@@ -410,24 +416,28 @@ public class VectorGroupByEngine
               final GroupByVectorColumnSelector selector = selectors.get(i);
 
               selector.writeKeyToResultRow(
-                  query.getDimensions().get(i).getOutputName(),
                   entry.getKey(),
                   keyOffset,
-                  theMap
+                  resultRow,
+                  resultRowDimensionStart + i
               );
 
               keyOffset += selector.getGroupingKeySize();
             }
 
             // Convert dimension values to desired output types, possibly.
-            GroupByQueryEngineV2.convertRowTypesToOutputTypes(query.getDimensions(), theMap);
+            GroupByQueryEngineV2.convertRowTypesToOutputTypes(
+                query.getDimensions(),
+                resultRow,
+                resultRowDimensionStart
+            );
 
             // Add aggregations.
             for (int i = 0; i < entry.getValues().length; i++) {
-              theMap.put(query.getAggregatorSpecs().get(i).getName(), entry.getValues()[i]);
+              resultRow.set(resultRowAggregatorStart + i, entry.getValues()[i]);
             }
 
-            return new MapBasedRow(timestamp, theMap);
+            return resultRow;
           },
           vectorGrouper
       );
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/AlwaysHavingSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/having/AlwaysHavingSpec.java
index 8450589..350d6ef 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/having/AlwaysHavingSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/having/AlwaysHavingSpec.java
@@ -19,16 +19,23 @@
 
 package org.apache.druid.query.groupby.having;
 
-import org.apache.druid.data.input.Row;
 import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 
 /**
  * A "having" spec that always evaluates to true
  */
-public class AlwaysHavingSpec extends BaseHavingSpec
+public class AlwaysHavingSpec implements HavingSpec
 {
   @Override
-  public boolean eval(Row row)
+  public void setQuery(GroupByQuery query)
+  {
+    // Do nothing.
+  }
+
+  @Override
+  public boolean eval(ResultRow row)
   {
     return true;
   }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/AndHavingSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/having/AndHavingSpec.java
index f035db3..9c11ab9 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/having/AndHavingSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/having/AndHavingSpec.java
@@ -22,18 +22,16 @@ package org.apache.druid.query.groupby.having;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.collect.ImmutableList;
-import org.apache.druid.data.input.Row;
-import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.cache.CacheKeyBuilder;
-import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 
 import java.util.List;
-import java.util.Map;
 
 /**
  * The logical "and" operator for the "having" clause.
  */
-public class AndHavingSpec extends BaseHavingSpec
+public class AndHavingSpec implements HavingSpec
 {
   private final List<HavingSpec> havingSpecs;
 
@@ -50,23 +48,15 @@ public class AndHavingSpec extends BaseHavingSpec
   }
 
   @Override
-  public void setRowSignature(Map<String, ValueType> rowSignature)
+  public void setQuery(GroupByQuery query)
   {
     for (HavingSpec havingSpec : havingSpecs) {
-      havingSpec.setRowSignature(rowSignature);
+      havingSpec.setQuery(query);
     }
   }
 
   @Override
-  public void setAggregators(Map<String, AggregatorFactory> aggregators)
-  {
-    for (HavingSpec havingSpec : havingSpecs) {
-      havingSpec.setAggregators(aggregators);
-    }
-  }
-
-  @Override
-  public boolean eval(Row row)
+  public boolean eval(ResultRow row)
   {
     for (HavingSpec havingSpec : havingSpecs) {
       if (!havingSpec.eval(row)) {
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/DimFilterHavingSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/having/DimFilterHavingSpec.java
index cf916bf..9039b21 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/having/DimFilterHavingSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/having/DimFilterHavingSpec.java
@@ -22,35 +22,30 @@ package org.apache.druid.query.groupby.having;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
-import org.apache.druid.data.input.InputRow;
-import org.apache.druid.data.input.Row;
+import it.unimi.dsi.fastutil.ints.Int2ObjectArrayMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import org.apache.druid.common.guava.SettableSupplier;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.cache.CacheKeyBuilder;
 import org.apache.druid.query.filter.DimFilter;
-import org.apache.druid.segment.column.ValueType;
-import org.apache.druid.segment.transform.RowFunction;
-import org.apache.druid.segment.transform.Transform;
-import org.apache.druid.segment.transform.TransformSpec;
-import org.apache.druid.segment.transform.Transformer;
-import org.joda.time.DateTime;
+import org.apache.druid.query.filter.ValueMatcher;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
+import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
 import java.util.Objects;
+import java.util.function.Function;
 
-public class DimFilterHavingSpec extends BaseHavingSpec
+public class DimFilterHavingSpec implements HavingSpec
 {
   private static final boolean DEFAULT_FINALIZE = true;
 
   private final DimFilter dimFilter;
   private final boolean finalize;
+  private final SettableSupplier<ResultRow> rowSupplier = new SettableSupplier<>();
 
-  private Map<String, ValueType> rowSignature = new HashMap<>();
-  private Map<String, AggregatorFactory> aggregators = new HashMap<>();
-  private Transformer transformer = null;
+  private Int2ObjectMap<Function<Object, Object>> finalizers = new Int2ObjectArrayMap<>();
+  private ValueMatcher matcher = null;
   private int evalCount;
 
   @JsonCreator
@@ -76,28 +71,43 @@ public class DimFilterHavingSpec extends BaseHavingSpec
   }
 
   @Override
-  public void setRowSignature(Map<String, ValueType> rowSignature)
+  public void setQuery(GroupByQuery query)
   {
-    this.rowSignature = rowSignature;
-  }
+    this.finalizers = new Int2ObjectArrayMap<>(query.getAggregatorSpecs().size());
 
-  @Override
-  public void setAggregators(final Map<String, AggregatorFactory> aggregators)
-  {
-    this.aggregators = aggregators;
+    for (AggregatorFactory factory : query.getAggregatorSpecs()) {
+      final int i = query.getResultRowPositionLookup().getInt(factory.getName());
+      this.finalizers.put(i, factory::finalizeComputation);
+    }
+
+    this.matcher = dimFilter.toFilter().makeMatcher(
+        RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory(
+            query,
+            rowSupplier
+        )
+    );
   }
 
   @Override
-  public boolean eval(final Row row)
+  public boolean eval(final ResultRow row)
   {
     int oldEvalCount = evalCount;
     evalCount++;
 
-    if (transformer == null) {
-      transformer = createTransformer(dimFilter, rowSignature, aggregators, finalize);
+    if (finalize && !finalizers.isEmpty()) {
+      // Create finalized copy.
+      final ResultRow finalizedCopy = row.copy();
+
+      for (Int2ObjectMap.Entry<Function<Object, Object>> entry : finalizers.int2ObjectEntrySet()) {
+        finalizedCopy.set(entry.getIntKey(), entry.getValue().apply(row.get(entry.getIntKey())));
+      }
+
+      rowSupplier.set(finalizedCopy);
+    } else {
+      rowSupplier.set(row);
     }
 
-    final boolean retVal = transformer.transform(new RowAsInputRow(row)) != null;
+    final boolean retVal = matcher.matches();
 
     if (evalCount != oldEvalCount + 1) {
       // Oops, someone was using this from two different threads, bad caller.
@@ -136,41 +146,6 @@ public class DimFilterHavingSpec extends BaseHavingSpec
            '}';
   }
 
-  private static Transformer createTransformer(
-      final DimFilter filter,
-      final Map<String, ValueType> rowSignature,
-      final Map<String, AggregatorFactory> aggregators,
-      final boolean finalize
-  )
-  {
-    final List<Transform> transforms = new ArrayList<>();
-
-    if (finalize) {
-      for (AggregatorFactory aggregator : aggregators.values()) {
-        final String name = aggregator.getName();
-
-        transforms.add(
-            new Transform()
-            {
-              @Override
-              public String getName()
-              {
-                return name;
-              }
-
-              @Override
-              public RowFunction getRowFunction()
-              {
-                return row -> aggregator.finalizeComputation(row.getRaw(name));
-              }
-            }
-        );
-      }
-    }
-
-    return new TransformSpec(filter, transforms).toTransformer(rowSignature);
-  }
-
   @Override
   public byte[] getCacheKey()
   {
@@ -179,83 +154,4 @@ public class DimFilterHavingSpec extends BaseHavingSpec
         .appendByte((byte) (isFinalize() ? 1 : 0))
         .build();
   }
-
-  private static class RowAsInputRow implements InputRow
-  {
-    private final Row row;
-
-    public RowAsInputRow(final Row row)
-    {
-      this.row = row;
-    }
-
-    @Override
-    public List<String> getDimensions()
-    {
-      return Collections.emptyList();
-    }
-
-    @Override
-    public long getTimestampFromEpoch()
-    {
-      return row.getTimestampFromEpoch();
-    }
-
-    @Override
-    public DateTime getTimestamp()
-    {
-      return row.getTimestamp();
-    }
-
-    @Override
-    public List<String> getDimension(final String dimension)
-    {
-      return row.getDimension(dimension);
-    }
-
-    @Override
-    public Object getRaw(final String dimension)
-    {
-      return row.getRaw(dimension);
-    }
-
-    @Override
-    public Number getMetric(final String metric)
-    {
-      return row.getMetric(metric);
-    }
-
-    @Override
-    public int compareTo(final Row o)
-    {
-      return row.compareTo(o);
-    }
-
-    @Override
-    public boolean equals(final Object o)
-    {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      final RowAsInputRow that = (RowAsInputRow) o;
-      return Objects.equals(row, that.row);
-    }
-
-    @Override
-    public int hashCode()
-    {
-      return Objects.hash(row);
-    }
-
-    @Override
-    public String toString()
-    {
-      return "RowAsInputRow{" +
-             "row=" + row +
-             '}';
-    }
-  }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/DimensionSelectorHavingSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/having/DimensionSelectorHavingSpec.java
index 4dfc6e6..80a9669 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/having/DimensionSelectorHavingSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/having/DimensionSelectorHavingSpec.java
@@ -23,20 +23,24 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
-import org.apache.druid.data.input.Row;
+import org.apache.druid.data.input.Rows;
 import org.apache.druid.query.cache.CacheKeyBuilder;
 import org.apache.druid.query.extraction.ExtractionFn;
 import org.apache.druid.query.extraction.IdentityExtractionFn;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 
 import java.util.List;
 import java.util.Objects;
 
-public class DimensionSelectorHavingSpec extends BaseHavingSpec
+public class DimensionSelectorHavingSpec implements HavingSpec
 {
   private final String dimension;
   private final String value;
   private final ExtractionFn extractionFn;
 
+  private volatile int columnNumber;
+
   @JsonCreator
   public DimensionSelectorHavingSpec(
       @JsonProperty("dimension") String dimName,
@@ -68,9 +72,19 @@ public class DimensionSelectorHavingSpec extends BaseHavingSpec
   }
 
   @Override
-  public boolean eval(Row row)
+  public void setQuery(GroupByQuery query)
+  {
+    columnNumber = query.getResultRowPositionLookup().getInt(dimension);
+  }
+
+  @Override
+  public boolean eval(ResultRow row)
   {
-    List<String> dimRowValList = row.getDimension(dimension);
+    if (columnNumber < 0) {
+      return Strings.isNullOrEmpty(value);
+    }
+
+    List<String> dimRowValList = Rows.objectToStrings(row.get(columnNumber));
     if (dimRowValList == null || dimRowValList.isEmpty()) {
       return Strings.isNullOrEmpty(value);
     }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/EqualToHavingSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/having/EqualToHavingSpec.java
index 00c471b..0bd0d45 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/having/EqualToHavingSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/having/EqualToHavingSpec.java
@@ -21,10 +21,11 @@ package org.apache.druid.query.groupby.having;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 
 import java.util.Map;
 
@@ -32,12 +33,13 @@ import java.util.Map;
  * The "=" operator in a "having" clause. This is similar to SQL's "having aggregation = value",
  * except that in SQL an aggregation is an expression instead of an aggregation name as in Druid.
  */
-public class EqualToHavingSpec extends BaseHavingSpec
+public class EqualToHavingSpec implements HavingSpec
 {
   private final String aggregationName;
   private final Number value;
 
   private volatile Map<String, AggregatorFactory> aggregators;
+  private volatile int columnNumber;
 
   @JsonCreator
   public EqualToHavingSpec(
@@ -62,15 +64,20 @@ public class EqualToHavingSpec extends BaseHavingSpec
   }
 
   @Override
-  public void setAggregators(Map<String, AggregatorFactory> aggregators)
+  public void setQuery(GroupByQuery query)
   {
-    this.aggregators = aggregators;
+    columnNumber = query.getResultRowPositionLookup().getInt(aggregationName);
+    aggregators = HavingSpecUtil.computeAggregatorsMap(query.getAggregatorSpecs());
   }
 
   @Override
-  public boolean eval(Row row)
+  public boolean eval(ResultRow row)
   {
-    Object metricVal = row.getRaw(aggregationName);
+    if (columnNumber < 0) {
+      return value == null;
+    }
+
+    Object metricVal = row.get(columnNumber);
     if (metricVal == null || value == null) {
       return metricVal == null && value == null;
     }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/GreaterThanHavingSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/having/GreaterThanHavingSpec.java
index aa276a9..be141e9 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/having/GreaterThanHavingSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/having/GreaterThanHavingSpec.java
@@ -21,10 +21,11 @@ package org.apache.druid.query.groupby.having;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 
 import java.util.Map;
 
@@ -32,12 +33,13 @@ import java.util.Map;
  * The "&gt;" operator in a "having" clause. This is similar to SQL's "having aggregation &gt; value",
  * except that an aggregation in SQL is an expression instead of an aggregation name as in Druid.
  */
-public class GreaterThanHavingSpec extends BaseHavingSpec
+public class GreaterThanHavingSpec implements HavingSpec
 {
   private final String aggregationName;
   private final Number value;
 
   private volatile Map<String, AggregatorFactory> aggregators;
+  private volatile int columnNumber;
 
   @JsonCreator
   public GreaterThanHavingSpec(
@@ -62,15 +64,20 @@ public class GreaterThanHavingSpec extends BaseHavingSpec
   }
 
   @Override
-  public void setAggregators(Map<String, AggregatorFactory> aggregators)
+  public void setQuery(GroupByQuery query)
   {
-    this.aggregators = aggregators;
+    columnNumber = query.getResultRowPositionLookup().getInt(aggregationName);
+    aggregators = HavingSpecUtil.computeAggregatorsMap(query.getAggregatorSpecs());
   }
 
   @Override
-  public boolean eval(Row row)
+  public boolean eval(ResultRow row)
   {
-    Object metricVal = row.getRaw(aggregationName);
+    if (columnNumber < 0) {
+      return false;
+    }
+
+    Object metricVal = row.get(columnNumber);
     if (metricVal == null || value == null) {
       return false;
     }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/HavingSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/having/HavingSpec.java
index e75641f..c179eb6 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/having/HavingSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/having/HavingSpec.java
@@ -21,12 +21,9 @@ package org.apache.druid.query.groupby.having;
 
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.Cacheable;
-import org.apache.druid.query.aggregation.AggregatorFactory;
-import org.apache.druid.segment.column.ValueType;
-
-import java.util.Map;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 
 /**
  * A "having" clause that filters aggregated/dimension value. This is similar to SQL's "having"
@@ -47,20 +44,10 @@ import java.util.Map;
 })
 public interface HavingSpec extends Cacheable
 {
-  // Atoms for easy combination, but for now they are mostly useful
-  // for testing.
-  HavingSpec NEVER = new NeverHavingSpec();
-  HavingSpec ALWAYS = new AlwaysHavingSpec();
-
   /**
-   * Informs this HavingSpec that rows passed to "eval" will have a certain signature. Will be called
-   * before "eval".
-   *
-   * @param rowSignature signature of the rows
+   * Informs this HavingSpec that rows passed to "eval" will originate from a particular groupBy query.
    */
-  void setRowSignature(Map<String, ValueType> rowSignature);
-
-  void setAggregators(Map<String, AggregatorFactory> aggregators);
+  void setQuery(GroupByQuery query);
 
   /**
    * Evaluates if a given row satisfies the having spec.
@@ -69,5 +56,5 @@ public interface HavingSpec extends Cacheable
    *
    * @return true if the given row satisfies the having spec. False otherwise.
    */
-  boolean eval(Row row);
+  boolean eval(ResultRow row);
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/HavingSpecUtil.java b/processing/src/main/java/org/apache/druid/query/groupby/having/HavingSpecUtil.java
index e1227e7..57a4a6d 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/having/HavingSpecUtil.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/having/HavingSpecUtil.java
@@ -19,6 +19,12 @@
 
 package org.apache.druid.query.groupby.having;
 
+import org.apache.druid.query.aggregation.AggregatorFactory;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 public class HavingSpecUtil
 {
   static final byte CACHE_TYPE_ID_ALWAYS = 0x0;
@@ -32,4 +38,11 @@ public class HavingSpecUtil
   static final byte CACHE_TYPE_ID_NOT = 0x8;
   static final byte CACHE_TYPE_ID_OR = 0x9;
   static final byte CACHE_TYPE_ID_COUNTING = 0xA;
+
+  public static Map<String, AggregatorFactory> computeAggregatorsMap(List<AggregatorFactory> aggregatorSpecs)
+  {
+    Map<String, AggregatorFactory> map = new HashMap<>(aggregatorSpecs.size());
+    aggregatorSpecs.forEach(v -> map.put(v.getName(), v));
+    return map;
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/LessThanHavingSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/having/LessThanHavingSpec.java
index 3c937cf..8bc3f97 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/having/LessThanHavingSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/having/LessThanHavingSpec.java
@@ -20,10 +20,11 @@
 package org.apache.druid.query.groupby.having;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 
 import java.util.Map;
 
@@ -31,12 +32,13 @@ import java.util.Map;
  * The "&lt;" operator in a "having" clause. This is similar to SQL's "having aggregation &lt; value",
  * except that an aggregation in SQL is an expression instead of an aggregation name as in Druid.
  */
-public class LessThanHavingSpec extends BaseHavingSpec
+public class LessThanHavingSpec implements HavingSpec
 {
   private final String aggregationName;
   private final Number value;
 
   private volatile Map<String, AggregatorFactory> aggregators;
+  private volatile int columnNumber;
 
   public LessThanHavingSpec(
       @JsonProperty("aggregation") String aggName,
@@ -60,15 +62,20 @@ public class LessThanHavingSpec extends BaseHavingSpec
   }
 
   @Override
-  public void setAggregators(Map<String, AggregatorFactory> aggregators)
+  public void setQuery(GroupByQuery query)
   {
-    this.aggregators = aggregators;
+    columnNumber = query.getResultRowPositionLookup().getInt(aggregationName);
+    aggregators = HavingSpecUtil.computeAggregatorsMap(query.getAggregatorSpecs());
   }
 
   @Override
-  public boolean eval(Row row)
+  public boolean eval(ResultRow row)
   {
-    Object metricVal = row.getRaw(aggregationName);
+    if (columnNumber < 0) {
+      return false;
+    }
+
+    Object metricVal = row.get(columnNumber);
     if (metricVal == null || value == null) {
       return false;
     }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/NeverHavingSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/having/NeverHavingSpec.java
index fa2c15d..a05ebca 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/having/NeverHavingSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/having/NeverHavingSpec.java
@@ -19,16 +19,23 @@
 
 package org.apache.druid.query.groupby.having;
 
-import org.apache.druid.data.input.Row;
 import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 
 /**
  * A "having" spec that always evaluates to false
  */
-public class NeverHavingSpec extends BaseHavingSpec
+public class NeverHavingSpec implements HavingSpec
 {
   @Override
-  public boolean eval(Row row)
+  public void setQuery(GroupByQuery query)
+  {
+    // Do nothing.
+  }
+
+  @Override
+  public boolean eval(ResultRow row)
   {
     return false;
   }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/NotHavingSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/having/NotHavingSpec.java
index 81d7a63..cd8f4c6 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/having/NotHavingSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/having/NotHavingSpec.java
@@ -21,17 +21,14 @@ package org.apache.druid.query.groupby.having;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.druid.data.input.Row;
-import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.cache.CacheKeyBuilder;
-import org.apache.druid.segment.column.ValueType;
-
-import java.util.Map;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 
 /**
  * The logical "not" operator for the "having" clause.
  */
-public class NotHavingSpec extends BaseHavingSpec
+public class NotHavingSpec implements HavingSpec
 {
   private final HavingSpec havingSpec;
 
@@ -48,19 +45,13 @@ public class NotHavingSpec extends BaseHavingSpec
   }
 
   @Override
-  public void setRowSignature(Map<String, ValueType> rowSignature)
-  {
-    havingSpec.setRowSignature(rowSignature);
-  }
-
-  @Override
-  public void setAggregators(Map<String, AggregatorFactory> aggregators)
+  public void setQuery(GroupByQuery query)
   {
-    havingSpec.setAggregators(aggregators);
+    havingSpec.setQuery(query);
   }
 
   @Override
-  public boolean eval(Row row)
+  public boolean eval(ResultRow row)
   {
     return !havingSpec.eval(row);
   }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/OrHavingSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/having/OrHavingSpec.java
index e648349..097619e 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/having/OrHavingSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/having/OrHavingSpec.java
@@ -22,18 +22,16 @@ package org.apache.druid.query.groupby.having;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.collect.ImmutableList;
-import org.apache.druid.data.input.Row;
-import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.cache.CacheKeyBuilder;
-import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 
 import java.util.List;
-import java.util.Map;
 
 /**
  * The logical "or" operator for the "having" clause.
  */
-public class OrHavingSpec extends BaseHavingSpec
+public class OrHavingSpec implements HavingSpec
 {
   private final List<HavingSpec> havingSpecs;
 
@@ -50,23 +48,15 @@ public class OrHavingSpec extends BaseHavingSpec
   }
 
   @Override
-  public void setRowSignature(Map<String, ValueType> rowSignature)
+  public void setQuery(GroupByQuery query)
   {
     for (HavingSpec havingSpec : havingSpecs) {
-      havingSpec.setRowSignature(rowSignature);
+      havingSpec.setQuery(query);
     }
   }
 
   @Override
-  public void setAggregators(Map<String, AggregatorFactory> aggregators)
-  {
-    for (HavingSpec havingSpec : havingSpecs) {
-      havingSpec.setAggregators(aggregators);
-    }
-  }
-
-  @Override
-  public boolean eval(Row row)
+  public boolean eval(ResultRow row)
   {
     for (HavingSpec havingSpec : havingSpecs) {
       if (havingSpec.eval(row)) {
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java
index 7a4c5f3..49c49ed 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java
@@ -25,19 +25,22 @@ import com.google.common.base.Function;
 import com.google.common.base.Functions;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Ordering;
 import com.google.common.primitives.Ints;
 import com.google.common.primitives.Longs;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
 import org.apache.druid.common.config.NullHandling;
-import org.apache.druid.data.input.Row;
+import org.apache.druid.data.input.Rows;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.granularity.Granularities;
-import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.PostAggregator;
 import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.ordering.StringComparator;
 import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.segment.column.ValueType;
@@ -52,6 +55,7 @@ import java.util.Map;
 import java.util.Set;
 
 /**
+ *
  */
 public class DefaultLimitSpec implements LimitSpec
 {
@@ -64,8 +68,9 @@ public class DefaultLimitSpec implements LimitSpec
    * Check if a limitSpec has columns in the sorting order that are not part of the grouping fields represented
    * by `dimensions`.
    *
-   * @param limitSpec LimitSpec, assumed to be non-null
+   * @param limitSpec  LimitSpec, assumed to be non-null
    * @param dimensions Grouping fields for a groupBy query
+   *
    * @return True if limitSpec has sorting columns not contained in dimensions
    */
   public static boolean sortingOrderHasNonGroupingFields(DefaultLimitSpec limitSpec, List<DimensionSpec> dimensions)
@@ -119,23 +124,18 @@ public class DefaultLimitSpec implements LimitSpec
   }
 
   @Override
-  public Function<Sequence<Row>, Sequence<Row>> build(
-      List<DimensionSpec> dimensions,
-      List<AggregatorFactory> aggs,
-      List<PostAggregator> postAggs,
-      Granularity granularity,
-      boolean sortByDimsFirst
-  )
+  public Function<Sequence<ResultRow>, Sequence<ResultRow>> build(final GroupByQuery query)
   {
-    // Can avoid re-sorting if the natural ordering is good enough.
+    final List<DimensionSpec> dimensions = query.getDimensions();
 
+    // Can avoid re-sorting if the natural ordering is good enough.
     boolean sortingNeeded = dimensions.size() < columns.size();
 
     final Set<String> aggAndPostAggNames = new HashSet<>();
-    for (AggregatorFactory agg : aggs) {
+    for (AggregatorFactory agg : query.getAggregatorSpecs()) {
       aggAndPostAggNames.add(agg.getName());
     }
-    for (PostAggregator postAgg : postAggs) {
+    for (PostAggregator postAgg : query.getPostAggregatorSpecs()) {
       aggAndPostAggNames.add(postAgg.getName());
     }
 
@@ -170,7 +170,7 @@ public class DefaultLimitSpec implements LimitSpec
 
     if (!sortingNeeded) {
       // If granularity is ALL, sortByDimsFirst doesn't change the sorting order.
-      sortingNeeded = !granularity.equals(Granularities.ALL) && sortByDimsFirst;
+      sortingNeeded = !query.getGranularity().equals(Granularities.ALL) && query.getContextSortByDimsFirst();
     }
 
     if (!sortingNeeded) {
@@ -178,7 +178,14 @@ public class DefaultLimitSpec implements LimitSpec
     }
 
     // Materialize the Comparator first for fast-fail error checking.
-    final Ordering<Row> ordering = makeComparator(dimensions, aggs, postAggs, sortByDimsFirst);
+    final Ordering<ResultRow> ordering = makeComparator(
+        query.getResultRowPositionLookup(),
+        query.getResultRowHasTimestamp(),
+        query.getDimensions(),
+        query.getAggregatorSpecs(),
+        query.getPostAggregatorSpecs(),
+        query.getContextSortByDimsFirst()
+    );
 
     if (isLimited()) {
       return new TopNFunction(ordering, limit);
@@ -204,21 +211,29 @@ public class DefaultLimitSpec implements LimitSpec
     throw new ISE("Unknown column in order clause[%s]", columnSpec);
   }
 
-  private Ordering<Row> makeComparator(
+  private Ordering<ResultRow> makeComparator(
+      Object2IntMap<String> rowOrderLookup,
+      boolean hasTimestamp,
       List<DimensionSpec> dimensions,
       List<AggregatorFactory> aggs,
       List<PostAggregator> postAggs,
       boolean sortByDimsFirst
   )
   {
-    Ordering<Row> timeOrdering = new Ordering<Row>()
-    {
-      @Override
-      public int compare(Row left, Row right)
+    final Ordering<ResultRow> timeOrdering;
+
+    if (hasTimestamp) {
+      timeOrdering = new Ordering<ResultRow>()
       {
-        return Longs.compare(left.getTimestampFromEpoch(), right.getTimestampFromEpoch());
-      }
-    };
+        @Override
+        public int compare(ResultRow left, ResultRow right)
+        {
+          return Longs.compare(left.getLong(0), right.getLong(0));
+        }
+      };
+    } else {
+      timeOrdering = null;
+    }
 
     Map<String, DimensionSpec> dimensionsMap = new HashMap<>();
     for (DimensionSpec spec : dimensions) {
@@ -235,17 +250,23 @@ public class DefaultLimitSpec implements LimitSpec
       postAggregatorsMap.put(postAgg.getName(), postAgg);
     }
 
-    Ordering<Row> ordering = null;
+    Ordering<ResultRow> ordering = null;
     for (OrderByColumnSpec columnSpec : columns) {
       String columnName = columnSpec.getDimension();
-      Ordering<Row> nextOrdering = null;
-
-      if (postAggregatorsMap.containsKey(columnName)) {
-        nextOrdering = metricOrdering(columnName, postAggregatorsMap.get(columnName).getComparator());
-      } else if (aggregatorsMap.containsKey(columnName)) {
-        nextOrdering = metricOrdering(columnName, aggregatorsMap.get(columnName).getComparator());
-      } else if (dimensionsMap.containsKey(columnName)) {
-        nextOrdering = dimensionOrdering(columnName, columnSpec.getDimensionComparator());
+      Ordering<ResultRow> nextOrdering = null;
+
+      final int columnIndex = rowOrderLookup.applyAsInt(columnName);
+
+      if (columnIndex >= 0) {
+        if (postAggregatorsMap.containsKey(columnName)) {
+          //noinspection unchecked
+          nextOrdering = metricOrdering(columnIndex, postAggregatorsMap.get(columnName).getComparator());
+        } else if (aggregatorsMap.containsKey(columnName)) {
+          //noinspection unchecked
+          nextOrdering = metricOrdering(columnIndex, aggregatorsMap.get(columnName).getComparator());
+        } else if (dimensionsMap.containsKey(columnName)) {
+          nextOrdering = dimensionOrdering(columnIndex, columnSpec.getDimensionComparator());
+        }
       }
 
       if (nextOrdering == null) {
@@ -259,37 +280,40 @@ public class DefaultLimitSpec implements LimitSpec
       ordering = ordering == null ? nextOrdering : ordering.compound(nextOrdering);
     }
 
-    if (ordering != null) {
-      ordering = sortByDimsFirst ? ordering.compound(timeOrdering) : timeOrdering.compound(ordering);
-    } else {
+    if (ordering == null) {
       ordering = timeOrdering;
+    } else if (timeOrdering != null) {
+      ordering = sortByDimsFirst ? ordering.compound(timeOrdering) : timeOrdering.compound(ordering);
     }
 
-    return ordering;
+    //noinspection unchecked
+    return ordering != null ? ordering : (Ordering) Ordering.allEqual();
   }
 
-  private Ordering<Row> metricOrdering(final String column, final Comparator comparator)
+  private <T> Ordering<ResultRow> metricOrdering(final int column, final Comparator<T> comparator)
   {
     // As per SQL standard we need to have same ordering for metrics as dimensions i.e nulls first
     // If SQL compatibility is not enabled we use nullsLast ordering for null metrics for backwards compatibility.
-    if (NullHandling.sqlCompatible()) {
-      return Ordering.from(Comparator.comparing((Row row) -> row.getRaw(column), Comparator.nullsFirst(comparator)));
-    } else {
-      return Ordering.from(Comparator.comparing((Row row) -> row.getRaw(column), Comparator.nullsLast(comparator)));
-    }
+    final Comparator<T> nullFriendlyComparator = NullHandling.sqlCompatible()
+                                                 ? Comparator.nullsFirst(comparator)
+                                                 : Comparator.nullsLast(comparator);
+
+    //noinspection unchecked
+    return Ordering.from(Comparator.comparing(row -> (T) row.get(column), nullFriendlyComparator));
   }
 
-  private Ordering<Row> dimensionOrdering(final String dimension, final StringComparator comparator)
+  private Ordering<ResultRow> dimensionOrdering(final int column, final StringComparator comparator)
   {
     return Ordering.from(
-        Comparator.comparing((Row row) -> getDimensionValue(row, dimension), Comparator.nullsFirst(comparator))
+        Comparator.comparing((ResultRow row) -> getDimensionValue(row, column), Comparator.nullsFirst(comparator))
     );
   }
 
-  private static String getDimensionValue(Row row, String column)
+  @Nullable
+  private static String getDimensionValue(ResultRow row, int column)
   {
-    List<String> values = row.getDimension(column);
-    return values.isEmpty() ? null : values.get(0);
+    final List<String> values = Rows.objectToStrings(row.get(column));
+    return values.isEmpty() ? null : Iterables.getOnlyElement(values);
   }
 
   @Override
@@ -301,9 +325,9 @@ public class DefaultLimitSpec implements LimitSpec
            '}';
   }
 
-  private static class LimitingFn implements Function<Sequence<Row>, Sequence<Row>>
+  private static class LimitingFn implements Function<Sequence<ResultRow>, Sequence<ResultRow>>
   {
-    private int limit;
+    private final int limit;
 
     public LimitingFn(int limit)
     {
@@ -311,41 +335,41 @@ public class DefaultLimitSpec implements LimitSpec
     }
 
     @Override
-    public Sequence<Row> apply(Sequence<Row> input)
+    public Sequence<ResultRow> apply(Sequence<ResultRow> input)
     {
       return input.limit(limit);
     }
   }
 
-  private static class SortingFn implements Function<Sequence<Row>, Sequence<Row>>
+  private static class SortingFn implements Function<Sequence<ResultRow>, Sequence<ResultRow>>
   {
-    private final Ordering<Row> ordering;
+    private final Ordering<ResultRow> ordering;
 
-    public SortingFn(Ordering<Row> ordering)
+    public SortingFn(Ordering<ResultRow> ordering)
     {
       this.ordering = ordering;
     }
 
     @Override
-    public Sequence<Row> apply(@Nullable Sequence<Row> input)
+    public Sequence<ResultRow> apply(@Nullable Sequence<ResultRow> input)
     {
       return Sequences.sort(input, ordering);
     }
   }
 
-  private static class TopNFunction implements Function<Sequence<Row>, Sequence<Row>>
+  private static class TopNFunction implements Function<Sequence<ResultRow>, Sequence<ResultRow>>
   {
-    private final Ordering<Row> ordering;
+    private final Ordering<ResultRow> ordering;
     private final int limit;
 
-    public TopNFunction(Ordering<Row> ordering, int limit)
+    public TopNFunction(Ordering<ResultRow> ordering, int limit)
     {
       this.ordering = ordering;
       this.limit = limit;
     }
 
     @Override
-    public Sequence<Row> apply(final Sequence<Row> input)
+    public Sequence<ResultRow> apply(final Sequence<ResultRow> input)
     {
       return new TopNSequence<>(input, ordering, limit);
     }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/orderby/LimitSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/orderby/LimitSpec.java
index d724335..8f0c9b5 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/orderby/LimitSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/orderby/LimitSpec.java
@@ -22,18 +22,15 @@ package org.apache.druid.query.groupby.orderby;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import com.google.common.base.Function;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.Cacheable;
-import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.guava.Sequence;
-import org.apache.druid.query.aggregation.AggregatorFactory;
-import org.apache.druid.query.aggregation.PostAggregator;
-import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 
 import javax.annotation.Nullable;
-import java.util.List;
 
 /**
+ *
  */
 @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoopLimitSpec.class)
 @JsonSubTypes(value = {
@@ -49,21 +46,11 @@ public interface LimitSpec extends Cacheable
   /**
    * Returns a function that applies a limit to an input sequence that is assumed to be sorted on dimensions.
    *
-   * @param dimensions      query dimensions
-   * @param aggs            query aggregators
-   * @param postAggs        query postAggregators
-   * @param granularity     query granularity
-   * @param sortByDimsFirst 'sortByDimsFirst' value in queryContext
+   * @param query the query that this limit spec belongs to
    *
    * @return limit function
    */
-  Function<Sequence<Row>, Sequence<Row>> build(
-      List<DimensionSpec> dimensions,
-      List<AggregatorFactory> aggs,
-      List<PostAggregator> postAggs,
-      Granularity granularity,
-      boolean sortByDimsFirst
-  );
+  Function<Sequence<ResultRow>, Sequence<ResultRow>> build(GroupByQuery query);
 
   LimitSpec merge(LimitSpec other);
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/orderby/NoopLimitSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/orderby/NoopLimitSpec.java
index fcfc291..9bec0cf 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/orderby/NoopLimitSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/orderby/NoopLimitSpec.java
@@ -22,16 +22,12 @@ package org.apache.druid.query.groupby.orderby;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.google.common.base.Function;
 import com.google.common.base.Functions;
-import org.apache.druid.data.input.Row;
-import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.guava.Sequence;
-import org.apache.druid.query.aggregation.AggregatorFactory;
-import org.apache.druid.query.aggregation.PostAggregator;
-import org.apache.druid.query.dimension.DimensionSpec;
-
-import java.util.List;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
 
 /**
+ *
  */
 public final class NoopLimitSpec implements LimitSpec
 {
@@ -50,13 +46,7 @@ public final class NoopLimitSpec implements LimitSpec
   }
 
   @Override
-  public Function<Sequence<Row>, Sequence<Row>> build(
-      List<DimensionSpec> dimensions,
-      List<AggregatorFactory> aggs,
-      List<PostAggregator> postAggs,
-      Granularity granularity,
-      boolean sortByDimsFirst
-  )
+  public Function<Sequence<ResultRow>, Sequence<ResultRow>> build(GroupByQuery query)
   {
     return Functions.identity();
   }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java
index b8dace9..6125dd0 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java
@@ -20,7 +20,6 @@
 package org.apache.druid.query.groupby.strategy;
 
 import com.google.common.util.concurrent.ListeningExecutorService;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.java.util.common.UOE;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator;
@@ -30,6 +29,7 @@ import org.apache.druid.query.QueryRunnerFactory;
 import org.apache.druid.query.context.ResponseContext;
 import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.resource.GroupByQueryResource;
 import org.apache.druid.segment.StorageAdapter;
 
@@ -41,45 +41,77 @@ import java.util.function.BinaryOperator;
 public interface GroupByStrategy
 {
   /**
-   * Initializes resources required for a broker to process the given query.
+   * Initializes resources required to run {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} for a
+   * particular query. That method is also the primary caller of this method.
+   *
+   * Used by {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}.
    *
    * @param query a groupBy query to be processed
+   *
    * @return broker resource
    */
-  GroupByQueryResource prepareResource(GroupByQuery query, boolean willMergeRunners);
+  GroupByQueryResource prepareResource(GroupByQuery query);
 
   /**
-   * Indicates this strategy is cacheable or not.
-   * The {@code willMergeRunners} parameter can be used for distinguishing the caller is a broker or a data node.
+   * Indicates if results from this query are cacheable or not.
+   *
+   * Used by {@link GroupByQueryQueryToolChest#getCacheStrategy(GroupByQuery)}.
    *
    * @param willMergeRunners indicates that {@link QueryRunnerFactory#mergeRunners(ExecutorService, Iterable)} will be
-   *                         called on the cached by-segment results
+   *                         called on the cached by-segment results. Can be used to distinguish if we are running on
+   *                         a broker or data node.
+   *
    * @return true if this strategy is cacheable, otherwise false.
    */
   boolean isCacheable(boolean willMergeRunners);
 
   /**
-   * Indicates if this query should undergo "mergeResults" or not.
+   * Indicates if this query should undergo "mergeResults" or not. Checked by
+   * {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}.
    */
   boolean doMergeResults(GroupByQuery query);
 
   /**
    * Decorate a runner with an interval chunking decorator.
    */
-  QueryRunner<Row> createIntervalChunkingRunner(
+  QueryRunner<ResultRow> createIntervalChunkingRunner(
       IntervalChunkingQueryRunnerDecorator decorator,
-      QueryRunner<Row> runner,
+      QueryRunner<ResultRow> runner,
       GroupByQueryQueryToolChest toolChest
   );
 
-  Sequence<Row> mergeResults(QueryRunner<Row> baseRunner, GroupByQuery query, ResponseContext responseContext);
+  /**
+   * Runs a provided {@link QueryRunner} on a provided {@link GroupByQuery}, which is assumed to return rows that are
+   * properly sorted (by timestamp and dimensions) but not necessarily fully merged (that is, there may be adjacent
+   * rows with the same timestamp and dimensions) and without PostAggregators computed. This method will fully merge
+   * the rows, apply PostAggregators, and return the resulting {@link Sequence}.
+   *
+   * The query will be modified before passing it down to the base runner. For example, "having" clauses will be
+   * removed and various context parameters will be adjusted.
+   *
+   * Despite the similar name, this method is much reduced in scope compared to
+   * {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}. That method does delegate to this one at some points,
+   * but has a truckload of other responsibility, including computing outer query results (if there are subqueries),
+   * computing subtotals (like GROUPING SETS), and computing the havingSpec and limitSpec.
+   *
+   * @param baseRunner      base query runner
+   * @param query           the groupBy query to run inside the base query runner
+   * @param responseContext the response context to pass to the base query runner
+   *
+   * @return merged result sequence
+   */
+  Sequence<ResultRow> mergeResults(
+      QueryRunner<ResultRow> baseRunner,
+      GroupByQuery query,
+      ResponseContext responseContext
+  );
 
   /**
    * See {@link org.apache.druid.query.QueryToolChest#createMergeFn(Query)} for details, allows
    * {@link GroupByQueryQueryToolChest} to delegate implementation to the strategy
    */
   @Nullable
-  default BinaryOperator<Row> createMergeFn(Query<Row> query)
+  default BinaryOperator<ResultRow> createMergeFn(Query<ResultRow> query)
   {
     throw new UOE("%s doesn't provide a merge function", this.getClass().getName());
   }
@@ -89,26 +121,91 @@ public interface GroupByStrategy
    * {@link GroupByQueryQueryToolChest} to delegate implementation to the strategy
    */
   @Nullable
-  default Comparator<Row> createResultComparator(Query<Row> queryParam)
+  default Comparator<ResultRow> createResultComparator(Query<ResultRow> queryParam)
   {
     throw new UOE("%s doesn't provide a result comparator", this.getClass().getName());
   }
 
-  Sequence<Row> applyPostProcessing(Sequence<Row> results, GroupByQuery query);
+  /**
+   * Apply the {@link GroupByQuery} "postProcessingFn", which is responsible for HavingSpec and LimitSpec.
+   *
+   * @param results sequence of results
+   * @param query   the groupBy query
+   *
+   * @return post-processed results, with HavingSpec and LimitSpec applied
+   */
+  Sequence<ResultRow> applyPostProcessing(Sequence<ResultRow> results, GroupByQuery query);
 
-  Sequence<Row> processSubqueryResult(
+  /**
+   * Called by {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} when it needs to process a subquery.
+   *
+   * @param subquery           inner query
+   * @param query              outer query
+   * @param resource           resources returned by {@link #prepareResource(GroupByQuery)}
+   * @param subqueryResult     result rows from the subquery
+   * @param wasQueryPushedDown true if the outer query was pushed down (so we only need to merge the outer query's
+   *                           results, not run it from scratch like a normal outer query)
+   *
+   * @return results of the outer query
+   */
+  Sequence<ResultRow> processSubqueryResult(
       GroupByQuery subquery,
       GroupByQuery query,
       GroupByQueryResource resource,
-      Sequence<Row> subqueryResult,
+      Sequence<ResultRow> subqueryResult,
       boolean wasQueryPushedDown
   );
 
-  Sequence<Row> processSubtotalsSpec(GroupByQuery query, GroupByQueryResource resource, Sequence<Row> queryResult);
+  /**
+   * Called by {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} when it needs to generate subtotals.
+   *
+   * @param query       query that has a "subtotalsSpec"
+   * @param resource    resources returned by {@link #prepareResource(GroupByQuery)}
+   * @param queryResult result rows from the main query
+   *
+   * @return results for each list of subtotals in the query, concatenated together
+   */
+  Sequence<ResultRow> processSubtotalsSpec(
+      GroupByQuery query,
+      GroupByQueryResource resource,
+      Sequence<ResultRow> queryResult
+  );
 
-  QueryRunner<Row> mergeRunners(ListeningExecutorService exec, Iterable<QueryRunner<Row>> queryRunners);
+  /**
+   * Merge a variety of single-segment query runners into a combined runner. Used by
+   * {@link org.apache.druid.query.groupby.GroupByQueryRunnerFactory#mergeRunners(ExecutorService, Iterable)}. In
+   * that sense, it is intended to go along with {@link #process(GroupByQuery, StorageAdapter)} (the runners created
+   * by that method will be fed into this method).
+   *
+   * This method is only called on data servers, like Historicals (not the Broker).
+   *
+   * @param exec         executor service used for parallel execution of the query runners
+   * @param queryRunners collection of query runners to merge
+   *
+   * @return merged query runner
+   */
+  QueryRunner<ResultRow> mergeRunners(ListeningExecutorService exec, Iterable<QueryRunner<ResultRow>> queryRunners);
 
-  Sequence<Row> process(GroupByQuery query, StorageAdapter storageAdapter);
+  /**
+   * Process a groupBy query on a single {@link StorageAdapter}. This is used by
+   * {@link org.apache.druid.query.groupby.GroupByQueryRunnerFactory#createRunner} to create per-segment
+   * QueryRunners.
+   *
+   * This method is only called on data servers, like Historicals (not the Broker).
+   *
+   * @param query          the groupBy query
+   * @param storageAdapter storage adatper for the segment in question
+   *
+   * @return result sequence for the storage adapter
+   */
+  Sequence<ResultRow> process(GroupByQuery query, StorageAdapter storageAdapter);
 
+  /**
+   * Returns whether this strategy supports pushing down outer queries. This is used by
+   * {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} when it decides whether or not to push down an
+   * outer query from the Broker to data servers, like Historicals.
+   *
+   * Can be removed when the "v1" groupBy strategy is removed. ("v1" returns false, and "v2" returns true.)
+   */
   boolean supportsNestedQueryPushDown();
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java
index 69f8790..be9ba0a 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java
@@ -28,7 +28,6 @@ import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.inject.Inject;
 import org.apache.druid.collections.NonBlockingPool;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.guice.annotations.Global;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.guava.Sequence;
@@ -46,6 +45,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryEngine;
 import org.apache.druid.query.groupby.GroupByQueryHelper;
 import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.orderby.NoopLimitSpec;
 import org.apache.druid.query.groupby.resource.GroupByQueryResource;
 import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
@@ -81,7 +81,7 @@ public class GroupByStrategyV1 implements GroupByStrategy
   }
 
   @Override
-  public GroupByQueryResource prepareResource(GroupByQuery query, boolean willMergeRunners)
+  public GroupByQueryResource prepareResource(GroupByQuery query)
   {
     return new GroupByQueryResource();
   }
@@ -93,9 +93,9 @@ public class GroupByStrategyV1 implements GroupByStrategy
   }
 
   @Override
-  public QueryRunner<Row> createIntervalChunkingRunner(
+  public QueryRunner<ResultRow> createIntervalChunkingRunner(
       final IntervalChunkingQueryRunnerDecorator decorator,
-      final QueryRunner<Row> runner,
+      final QueryRunner<ResultRow> runner,
       final GroupByQueryQueryToolChest toolChest
   )
   {
@@ -109,14 +109,15 @@ public class GroupByStrategyV1 implements GroupByStrategy
   }
 
   @Override
-  public Sequence<Row> mergeResults(
-      final QueryRunner<Row> baseRunner,
+  public Sequence<ResultRow> mergeResults(
+      final QueryRunner<ResultRow> baseRunner,
       final GroupByQuery query,
       final ResponseContext responseContext
   )
   {
     final IncrementalIndex index = GroupByQueryHelper.makeIncrementalIndex(
         query,
+        null,
         configSupplier.get(),
         bufferPool,
         baseRunner.run(
@@ -128,39 +129,43 @@ public class GroupByStrategyV1 implements GroupByStrategy
                     .setHavingSpec(null)
                     .setLimitSpec(NoopLimitSpec.instance())
                     .overrideContext(
-                        ImmutableMap.of(
-                            "finalize", false,
-                            //set sort to false avoids unnecessary sorting while merging results. we only need to sort
-                            //in the end when returning results to user. (note this is only respected by groupBy v1)
-                            GroupByQueryHelper.CTX_KEY_SORT_RESULTS, false,
-                            //no merging needed at historicals because GroupByQueryRunnerFactory.mergeRunners(..) would
-                            //return merged results. (note this is only respected by groupBy v1)
-                            GroupByQueryQueryToolChest.GROUP_BY_MERGE_KEY, false,
-                            GroupByQueryConfig.CTX_KEY_STRATEGY, GroupByStrategySelector.STRATEGY_V1
-                        )
+                        ImmutableMap.<String, Object>builder()
+                            .put(GroupByQueryConfig.CTX_KEY_STRATEGY, GroupByStrategySelector.STRATEGY_V1)
+                            .put("finalize", false)
+
+                            // Always request array result rows when passing the query down.
+                            .put(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true)
+
+                            // Set sort to false avoids unnecessary sorting while merging results. we only need to sort
+                            // in the end when returning results to user. (note this is only respected by groupBy v1)
+                            .put(GroupByQueryHelper.CTX_KEY_SORT_RESULTS, false)
+
+                            // No merging needed at historicals because GroupByQueryRunnerFactory.mergeRunners(..) would
+                            // return merged results. (note this is only respected by groupBy v1)
+                            .put(GroupByQueryQueryToolChest.GROUP_BY_MERGE_KEY, false)
+                            .build()
                     )
                     .build()
             ),
             responseContext
-        ),
-        true
+        )
     );
 
     return Sequences.withBaggage(GroupByQueryHelper.postAggregate(query, index), index);
   }
 
   @Override
-  public Sequence<Row> applyPostProcessing(Sequence<Row> results, GroupByQuery query)
+  public Sequence<ResultRow> applyPostProcessing(Sequence<ResultRow> results, GroupByQuery query)
   {
     return query.postProcess(results);
   }
 
   @Override
-  public Sequence<Row> processSubqueryResult(
+  public Sequence<ResultRow> processSubqueryResult(
       GroupByQuery subquery,
       GroupByQuery query,
       GroupByQueryResource resource,
-      Sequence<Row> subqueryResult,
+      Sequence<ResultRow> subqueryResult,
       boolean wasQueryPushedDown
   )
   {
@@ -221,10 +226,10 @@ public class GroupByStrategyV1 implements GroupByStrategy
                 GroupByQueryHelper.CTX_KEY_SORT_RESULTS, true
             )
         ),
+        subquery,
         configSupplier.get(),
         bufferPool,
-        subqueryResult,
-        false
+        subqueryResult
     );
 
     //Outer query might have multiple intervals, but they are expected to be non-overlapping and sorted which
@@ -233,15 +238,16 @@ public class GroupByStrategyV1 implements GroupByStrategy
     //and concatenate the results.
     final IncrementalIndex outerQueryResultIndex = GroupByQueryHelper.makeIncrementalIndex(
         outerQuery,
+        null,
         configSupplier.get(),
         bufferPool,
         Sequences.concat(
             Sequences.map(
                 Sequences.simple(outerQuery.getIntervals()),
-                new Function<Interval, Sequence<Row>>()
+                new Function<Interval, Sequence<ResultRow>>()
                 {
                   @Override
-                  public Sequence<Row> apply(Interval interval)
+                  public Sequence<ResultRow> apply(Interval interval)
                   {
                     return process(
                         outerQuery.withQuerySegmentSpec(
@@ -252,8 +258,7 @@ public class GroupByStrategyV1 implements GroupByStrategy
                   }
                 }
             )
-        ),
-        true
+        )
     );
 
     innerQueryResultIndex.close();
@@ -265,28 +270,31 @@ public class GroupByStrategyV1 implements GroupByStrategy
   }
 
   @Override
-  public Sequence<Row> processSubtotalsSpec(
+  public Sequence<ResultRow> processSubtotalsSpec(
       GroupByQuery query,
       GroupByQueryResource resource,
-      Sequence<Row> queryResult
+      Sequence<ResultRow> queryResult
   )
   {
     throw new UnsupportedOperationException("subtotalsSpec is not supported for v1 groupBy strategy.");
   }
 
   @Override
-  public QueryRunner<Row> mergeRunners(
+  public QueryRunner<ResultRow> mergeRunners(
       final ListeningExecutorService exec,
-      final Iterable<QueryRunner<Row>> queryRunners
+      final Iterable<QueryRunner<ResultRow>> queryRunners
   )
   {
     return new GroupByMergedQueryRunner<>(exec, configSupplier, queryWatcher, bufferPool, queryRunners);
   }
 
   @Override
-  public Sequence<Row> process(final GroupByQuery query, final StorageAdapter storageAdapter)
+  public Sequence<ResultRow> process(final GroupByQuery query, final StorageAdapter storageAdapter)
   {
-    return engine.process(query, storageAdapter);
+    return Sequences.map(
+        engine.process(query, storageAdapter),
+        row -> GroupByQueryHelper.toResultRow(query, row)
+    );
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java
index 72801a0..4c417b0 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java
@@ -20,25 +20,19 @@
 package org.apache.druid.query.groupby.strategy;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Function;
 import com.google.common.base.Supplier;
-import com.google.common.base.Suppliers;
 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 com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.inject.Inject;
 import org.apache.druid.collections.BlockingPool;
 import org.apache.druid.collections.NonBlockingPool;
 import org.apache.druid.collections.ReferenceCountingResourceHolder;
-import org.apache.druid.data.input.MapBasedRow;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.guice.annotations.Global;
 import org.apache.druid.guice.annotations.Merging;
 import org.apache.druid.guice.annotations.Smile;
-import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.granularity.Granularities;
-import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.guava.CloseQuietly;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
@@ -54,33 +48,29 @@ import org.apache.druid.query.QueryRunner;
 import org.apache.druid.query.QueryWatcher;
 import org.apache.druid.query.ResourceLimitExceededException;
 import org.apache.druid.query.ResultMergeQueryRunner;
+import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.PostAggregator;
 import org.apache.druid.query.context.ResponseContext;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
-import org.apache.druid.query.groupby.GroupByQueryHelper;
 import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.epinephelinae.GroupByBinaryFnV2;
 import org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunnerV2;
 import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2;
 import org.apache.druid.query.groupby.epinephelinae.GroupByRowProcessor;
-import org.apache.druid.query.groupby.epinephelinae.Grouper;
 import org.apache.druid.query.groupby.resource.GroupByQueryResource;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
 import org.apache.druid.segment.StorageAdapter;
-import org.joda.time.DateTime;
-import org.joda.time.Interval;
 
-import java.io.Closeable;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Comparator;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.function.BinaryOperator;
-import java.util.stream.Collectors;
 
 public class GroupByStrategyV2 implements GroupByStrategy
 {
@@ -115,58 +105,31 @@ public class GroupByStrategyV2 implements GroupByStrategy
     this.queryWatcher = queryWatcher;
   }
 
-  /**
-   * If "query" has a single universal timestamp, return it. Otherwise return null. This is useful
-   * for keeping timestamps in sync across partial queries that may have different intervals.
-   *
-   * @param query the query
-   *
-   * @return universal timestamp, or null
-   */
-  public static DateTime getUniversalTimestamp(final GroupByQuery query)
-  {
-    final Granularity gran = query.getGranularity();
-    final String timestampStringFromContext = query.getContextValue(CTX_KEY_FUDGE_TIMESTAMP, "");
-
-    if (!timestampStringFromContext.isEmpty()) {
-      return DateTimes.utc(Long.parseLong(timestampStringFromContext));
-    } else if (Granularities.ALL.equals(gran)) {
-      final DateTime timeStart = query.getIntervals().get(0).getStart();
-      return gran.getIterable(new Interval(timeStart, timeStart.plus(1))).iterator().next().getStart();
-    } else {
-      return null;
-    }
-  }
-
   @Override
-  public GroupByQueryResource prepareResource(GroupByQuery query, boolean willMergeRunners)
+  public GroupByQueryResource prepareResource(GroupByQuery query)
   {
-    if (!willMergeRunners) {
-      final int requiredMergeBufferNum = countRequiredMergeBufferNum(query, 1) +
-                                         (query.getSubtotalsSpec() != null ? 1 : 0);
-
-      if (requiredMergeBufferNum > mergeBufferPool.maxSize()) {
-        throw new ResourceLimitExceededException(
-            "Query needs " + requiredMergeBufferNum + " merge buffers, but only "
-            + mergeBufferPool.maxSize() + " merge buffers were configured"
-        );
-      } else if (requiredMergeBufferNum == 0) {
-        return new GroupByQueryResource();
+    final int requiredMergeBufferNum = countRequiredMergeBufferNum(query, 1) +
+                                       (query.getSubtotalsSpec() != null ? 1 : 0);
+
+    if (requiredMergeBufferNum > mergeBufferPool.maxSize()) {
+      throw new ResourceLimitExceededException(
+          "Query needs " + requiredMergeBufferNum + " merge buffers, but only "
+          + mergeBufferPool.maxSize() + " merge buffers were configured"
+      );
+    } else if (requiredMergeBufferNum == 0) {
+      return new GroupByQueryResource();
+    } else {
+      final List<ReferenceCountingResourceHolder<ByteBuffer>> mergeBufferHolders;
+      if (QueryContexts.hasTimeout(query)) {
+        mergeBufferHolders = mergeBufferPool.takeBatch(requiredMergeBufferNum, QueryContexts.getTimeout(query));
       } else {
-        final List<ReferenceCountingResourceHolder<ByteBuffer>> mergeBufferHolders;
-        if (QueryContexts.hasTimeout(query)) {
-          mergeBufferHolders = mergeBufferPool.takeBatch(requiredMergeBufferNum, QueryContexts.getTimeout(query));
-        } else {
-          mergeBufferHolders = mergeBufferPool.takeBatch(requiredMergeBufferNum);
-        }
-        if (mergeBufferHolders.isEmpty()) {
-          throw new InsufficientResourcesException("Cannot acquire enough merge buffers");
-        } else {
-          return new GroupByQueryResource(mergeBufferHolders);
-        }
+        mergeBufferHolders = mergeBufferPool.takeBatch(requiredMergeBufferNum);
+      }
+      if (mergeBufferHolders.isEmpty()) {
+        throw new InsufficientResourcesException("Cannot acquire enough merge buffers");
+      } else {
+        return new GroupByQueryResource(mergeBufferHolders);
       }
-    } else {
-      return new GroupByQueryResource();
     }
   }
 
@@ -200,9 +163,9 @@ public class GroupByStrategyV2 implements GroupByStrategy
   }
 
   @Override
-  public QueryRunner<Row> createIntervalChunkingRunner(
+  public QueryRunner<ResultRow> createIntervalChunkingRunner(
       final IntervalChunkingQueryRunnerDecorator decorator,
-      final QueryRunner<Row> runner,
+      final QueryRunner<ResultRow> runner,
       final GroupByQueryQueryToolChest toolChest
   )
   {
@@ -215,44 +178,47 @@ public class GroupByStrategyV2 implements GroupByStrategy
   }
 
   @Override
-  public Comparator<Row> createResultComparator(Query<Row> queryParam)
+  public Comparator<ResultRow> createResultComparator(Query<ResultRow> queryParam)
   {
     return ((GroupByQuery) queryParam).getRowOrdering(true);
   }
 
   @Override
-  public BinaryOperator<Row> createMergeFn(Query<Row> queryParam)
+  public BinaryOperator<ResultRow> createMergeFn(Query<ResultRow> queryParam)
   {
     return new GroupByBinaryFnV2((GroupByQuery) queryParam);
   }
 
   @Override
-  public Sequence<Row> mergeResults(
-      final QueryRunner<Row> baseRunner,
+  public Sequence<ResultRow> mergeResults(
+      final QueryRunner<ResultRow> baseRunner,
       final GroupByQuery query,
       final ResponseContext responseContext
   )
   {
     // Merge streams using ResultMergeQueryRunner, then apply postaggregators, then apply limit (which may
     // involve materialization)
-    final ResultMergeQueryRunner<Row> mergingQueryRunner = new ResultMergeQueryRunner<>(
+    final ResultMergeQueryRunner<ResultRow> mergingQueryRunner = new ResultMergeQueryRunner<>(
         baseRunner,
         this::createResultComparator,
         this::createMergeFn
     );
 
-    // Fudge timestamp, maybe.
-    final DateTime fudgeTimestamp = getUniversalTimestamp(query);
-    ImmutableMap.Builder<String, Object> context = ImmutableMap.builder();
+    // Set up downstream context.
+    final ImmutableMap.Builder<String, Object> context = ImmutableMap.builder();
     context.put("finalize", false);
     context.put(GroupByQueryConfig.CTX_KEY_STRATEGY, GroupByStrategySelector.STRATEGY_V2);
-    if (fudgeTimestamp != null) {
-      context.put(CTX_KEY_FUDGE_TIMESTAMP, String.valueOf(fudgeTimestamp.getMillis()));
-    }
     context.put(CTX_KEY_OUTERMOST, false);
-    // the having spec shouldn't be passed down, so we need to convey the existing limit push down status
+    if (query.getUniversalTimestamp() != null) {
+      context.put(CTX_KEY_FUDGE_TIMESTAMP, String.valueOf(query.getUniversalTimestamp().getMillis()));
+    }
+
+    // The having spec shouldn't be passed down, so we need to convey the existing limit push down status
     context.put(GroupByQueryConfig.CTX_KEY_APPLY_LIMIT_PUSH_DOWN, query.isApplyLimitPushDown());
 
+    // Always request array result rows when passing the query downstream.
+    context.put(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true);
+
     final GroupByQuery newQuery = new GroupByQuery(
         query.getDataSource(),
         query.getQuerySegmentSpec(),
@@ -271,56 +237,48 @@ public class GroupByStrategyV2 implements GroupByStrategy
         context.build()
     );
 
-    return Sequences.map(
-        mergingQueryRunner.run(
-            QueryPlus.wrap(newQuery),
-            responseContext
-        ),
-        new Function<Row, Row>()
-        {
-          @Override
-          public Row apply(final Row row)
-          {
-            if (query.getContextBoolean(GroupByQueryConfig.CTX_KEY_EXECUTING_NESTED_QUERY, false)) {
-              // When executing nested queries, we need to make sure that we are
-              // extracting out the event from the row.  Post aggregators are not invoked since
-              // they should only be used after merging all the nested query responses. Timestamp
-              // if it needs to be fudged, it is ok to do here.
-              return new MapBasedRow(
-                  fudgeTimestamp != null ? fudgeTimestamp : row.getTimestamp(),
-                  ((MapBasedRow) row).getEvent()
-              );
-            }
-            // Apply postAggregators and fudgeTimestamp if present and if this is the outermost mergeResults.
+    final Sequence<ResultRow> mergedResults = mergingQueryRunner.run(QueryPlus.wrap(newQuery), responseContext);
 
-            if (!query.getContextBoolean(CTX_KEY_OUTERMOST, true)) {
-              return row;
-            }
+    // Apply postaggregators if this is the outermost mergeResults (CTX_KEY_OUTERMOST) and we are not executing a
+    // pushed-down subquery (CTX_KEY_EXECUTING_NESTED_QUERY).
+
+    if (!query.getContextBoolean(CTX_KEY_OUTERMOST, true)
+        || query.getPostAggregatorSpecs().isEmpty()
+        || query.getContextBoolean(GroupByQueryConfig.CTX_KEY_EXECUTING_NESTED_QUERY, false)) {
+      return mergedResults;
+    } else {
+      return Sequences.map(
+          mergedResults,
+          row -> {
+            // This function's purpose is to apply PostAggregators.
+
+            final ResultRow rowWithPostAggregations = ResultRow.create(query.getResultRowSizeWithPostAggregators());
 
-            if (query.getPostAggregatorSpecs().isEmpty() && fudgeTimestamp == null) {
-              return row;
+            // Copy everything that comes before the postaggregations.
+            for (int i = 0; i < query.getResultRowPostAggregatorStart(); i++) {
+              rowWithPostAggregations.set(i, row.get(i));
             }
 
-            final Map<String, Object> newMap;
+            // Compute postaggregations. We need to do this with a result-row map because PostAggregator.compute
+            // expects a map. Some further design adjustment may eliminate the need for it, and speed up this function.
+            final Map<String, Object> mapForPostAggregationComputation = rowWithPostAggregations.toMap(query);
 
-            if (query.getPostAggregatorSpecs().isEmpty()) {
-              newMap = ((MapBasedRow) row).getEvent();
-            } else {
-              newMap = Maps.newLinkedHashMap(((MapBasedRow) row).getEvent());
+            for (int i = 0; i < query.getPostAggregatorSpecs().size(); i++) {
+              final PostAggregator postAggregator = query.getPostAggregatorSpecs().get(i);
+              final Object value = postAggregator.compute(mapForPostAggregationComputation);
 
-              for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) {
-                newMap.put(postAggregator.getName(), postAggregator.compute(newMap));
-              }
+              rowWithPostAggregations.set(query.getResultRowPostAggregatorStart() + i, value);
+              mapForPostAggregationComputation.put(postAggregator.getName(), value);
             }
 
-            return new MapBasedRow(fudgeTimestamp != null ? fudgeTimestamp : row.getTimestamp(), newMap);
+            return rowWithPostAggregations;
           }
-        }
-    );
+      );
+    }
   }
 
   @Override
-  public Sequence<Row> applyPostProcessing(Sequence<Row> results, GroupByQuery query)
+  public Sequence<ResultRow> applyPostProcessing(Sequence<ResultRow> results, GroupByQuery query)
   {
     // Don't apply limit here for inner results, that will be pushed down to the BufferHashGrouper
     if (query.getContextBoolean(CTX_KEY_OUTERMOST, true)) {
@@ -331,143 +289,159 @@ public class GroupByStrategyV2 implements GroupByStrategy
   }
 
   @Override
-  public Sequence<Row> processSubqueryResult(
+  public Sequence<ResultRow> processSubqueryResult(
       GroupByQuery subquery,
       GroupByQuery query,
       GroupByQueryResource resource,
-      Sequence<Row> subqueryResult,
+      Sequence<ResultRow> subqueryResult,
       boolean wasQueryPushedDown
   )
   {
-    // This contains all closeable objects which are closed when the returned iterator iterates all the elements,
-    // or an exceptions is thrown. The objects are closed in their reverse order.
-    final List<Closeable> closeOnExit = new ArrayList<>();
+    // Keep a reference to resultSupplier outside the "try" so we can close it if something goes wrong
+    // while creating the sequence.
+    GroupByRowProcessor.ResultSupplier resultSupplier = null;
 
     try {
-      Supplier<Grouper> grouperSupplier = Suppliers.memoize(
-          () -> GroupByRowProcessor.createGrouper(
-              query,
-              subqueryResult,
-              GroupByQueryHelper.rowSignatureFor(wasQueryPushedDown ? query : subquery),
-              configSupplier.get(),
-              resource,
-              spillMapper,
-              processingConfig.getTmpDir(),
-              processingConfig.intermediateComputeSizeBytes(),
-              closeOnExit,
-              wasQueryPushedDown,
-              true
-          )
+      final GroupByQuery queryToRun;
+
+      if (wasQueryPushedDown) {
+        // If the query was pushed down, filters would have been applied downstream, so skip it here.
+        queryToRun = query.withDimFilter(null)
+                          .withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY));
+      } else {
+        queryToRun = query;
+      }
+
+      resultSupplier = GroupByRowProcessor.process(
+          queryToRun,
+          wasQueryPushedDown ? queryToRun : subquery,
+          subqueryResult,
+          configSupplier.get(),
+          resource,
+          spillMapper,
+          processingConfig.getTmpDir(),
+          processingConfig.intermediateComputeSizeBytes()
       );
 
+      final GroupByRowProcessor.ResultSupplier finalResultSupplier = resultSupplier;
       return Sequences.withBaggage(
-          mergeResults(new QueryRunner<Row>()
-          {
-            @Override
-            public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
-            {
-              return GroupByRowProcessor.getRowsFromGrouper(
-                  query,
-                  null,
-                  grouperSupplier
-              );
-            }
-          }, query, null),
-          () -> Lists.reverse(closeOnExit).forEach(closeable -> CloseQuietly.close(closeable))
+          mergeResults(
+              (queryPlus, responseContext) -> finalResultSupplier.results(null),
+              query,
+              null
+          ),
+          finalResultSupplier
       );
     }
     catch (Exception ex) {
-      Lists.reverse(closeOnExit).forEach(closeable -> CloseQuietly.close(closeable));
+      CloseQuietly.close(resultSupplier);
       throw ex;
     }
   }
 
   @Override
-  public Sequence<Row> processSubtotalsSpec(
+  public Sequence<ResultRow> processSubtotalsSpec(
       GroupByQuery query,
       GroupByQueryResource resource,
-      Sequence<Row> queryResult
+      Sequence<ResultRow> queryResult
   )
   {
-    // This contains all closeable objects which are closed when the returned iterator iterates all the elements,
-    // or an exceptions is thrown. The objects are closed in their reverse order.
-    final List<Closeable> closeOnExit = new ArrayList<>();
+    // Note: the approach used here is not always correct; see https://github.com/apache/incubator-druid/issues/8091.
+
+    // Keep a reference to resultSupplier outside the "try" so we can close it if something goes wrong
+    // while creating the sequence.
+    GroupByRowProcessor.ResultSupplier resultSupplier = null;
 
     try {
       GroupByQuery queryWithoutSubtotalsSpec = query.withSubtotalsSpec(null).withDimFilter(null);
       List<List<String>> subtotals = query.getSubtotalsSpec();
 
-      Supplier<Grouper> grouperSupplier = Suppliers.memoize(
-          () -> GroupByRowProcessor.createGrouper(
-              queryWithoutSubtotalsSpec.withAggregatorSpecs(
-                  Lists.transform(queryWithoutSubtotalsSpec.getAggregatorSpecs(), (agg) -> agg.getCombiningFactory())
-              ).withDimensionSpecs(
+      resultSupplier = GroupByRowProcessor.process(
+          queryWithoutSubtotalsSpec
+              .withAggregatorSpecs(
+                  Lists.transform(
+                      queryWithoutSubtotalsSpec.getAggregatorSpecs(),
+                      AggregatorFactory::getCombiningFactory
+                  )
+              )
+              .withDimensionSpecs(
                   Lists.transform(
                       queryWithoutSubtotalsSpec.getDimensions(),
-                      (dimSpec) -> new DefaultDimensionSpec(
-                          dimSpec.getOutputName(),
-                          dimSpec.getOutputName(),
-                          dimSpec.getOutputType()
-                      )
+                      dimSpec ->
+                          new DefaultDimensionSpec(
+                              dimSpec.getOutputName(),
+                              dimSpec.getOutputName(),
+                              dimSpec.getOutputType()
+                          )
                   )
               ),
-              queryResult,
-              GroupByQueryHelper.rowSignatureFor(queryWithoutSubtotalsSpec),
-              configSupplier.get(),
-              resource,
-              spillMapper,
-              processingConfig.getTmpDir(),
-              processingConfig.intermediateComputeSizeBytes(),
-              closeOnExit,
-              false,
-              false
-          )
+          queryWithoutSubtotalsSpec,
+          queryResult,
+          configSupplier.get(),
+          resource,
+          spillMapper,
+          processingConfig.getTmpDir(),
+          processingConfig.intermediateComputeSizeBytes()
       );
-      List<Sequence<Row>> subtotalsResults = new ArrayList<>(subtotals.size());
-
-      Map<String, DimensionSpec> queryDimensionSpecs = new HashMap(queryWithoutSubtotalsSpec.getDimensions().size());
-      for (DimensionSpec dimSpec : queryWithoutSubtotalsSpec.getDimensions()) {
-        queryDimensionSpecs.put(dimSpec.getOutputName(), dimSpec);
-      }
+      List<Sequence<ResultRow>> subtotalsResults = new ArrayList<>(subtotals.size());
 
       for (List<String> subtotalSpec : subtotals) {
-        GroupByQuery subtotalQuery = queryWithoutSubtotalsSpec.withDimensionSpecs(
-            subtotalSpec.stream()
-                        .map(s -> new DefaultDimensionSpec(s, s, queryDimensionSpecs.get(s).getOutputType()))
-                        .collect(Collectors.toList())
-        );
+        final ImmutableSet<String> dimsInSubtotalSpec = ImmutableSet.copyOf(subtotalSpec);
+        final List<DimensionSpec> dimensions = query.getDimensions();
+        final List<DimensionSpec> newDimensions = new ArrayList<>();
+
+        for (int i = 0; i < dimensions.size(); i++) {
+          DimensionSpec dimensionSpec = dimensions.get(i);
+          if (dimsInSubtotalSpec.contains(dimensionSpec.getOutputName())) {
+            newDimensions.add(
+                new DefaultDimensionSpec(
+                    dimensionSpec.getOutputName(),
+                    dimensionSpec.getOutputName(),
+                    dimensionSpec.getOutputType()
+                )
+            );
+          } else {
+            // Insert dummy dimension so all subtotals queries have ResultRows with the same shape.
+            // Use a field name that does not appear in the main query result, to assure the result will be null.
+            String dimName = "_" + i;
+            while (query.getResultRowPositionLookup().getInt(dimName) >= 0) {
+              dimName = "_" + dimName;
+            }
+            newDimensions.add(DefaultDimensionSpec.of(dimName));
+          }
+        }
 
-        subtotalsResults.add(applyPostProcessing(
-            mergeResults(new QueryRunner<Row>()
-            {
-              @Override
-              public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
-              {
-                return GroupByRowProcessor.getRowsFromGrouper(
-                    queryWithoutSubtotalsSpec,
-                    subtotalSpec,
-                    grouperSupplier
-                );
-              }
-            }, subtotalQuery, null),
-            subtotalQuery
-                             )
+        GroupByQuery subtotalQuery = queryWithoutSubtotalsSpec.withDimensionSpecs(newDimensions);
+
+        final GroupByRowProcessor.ResultSupplier finalResultSupplier = resultSupplier;
+        subtotalsResults.add(
+            applyPostProcessing(
+                mergeResults(
+                    (queryPlus, responseContext) -> finalResultSupplier.results(subtotalSpec),
+                    subtotalQuery,
+                    null
+                ),
+                subtotalQuery
+            )
         );
       }
 
       return Sequences.withBaggage(
           Sequences.concat(subtotalsResults),
-          () -> Lists.reverse(closeOnExit).forEach(closeable -> CloseQuietly.close(closeable))
+          resultSupplier
       );
     }
     catch (Exception ex) {
-      Lists.reverse(closeOnExit).forEach(closeable -> CloseQuietly.close(closeable));
+      CloseQuietly.close(resultSupplier);
       throw ex;
     }
   }
 
   @Override
-  public QueryRunner<Row> mergeRunners(ListeningExecutorService exec, Iterable<QueryRunner<Row>> queryRunners)
+  public QueryRunner<ResultRow> mergeRunners(
+      final ListeningExecutorService exec,
+      final Iterable<QueryRunner<ResultRow>> queryRunners
+  )
   {
     return new GroupByMergingQueryRunnerV2(
         configSupplier.get(),
@@ -483,7 +457,7 @@ public class GroupByStrategyV2 implements GroupByStrategy
   }
 
   @Override
-  public Sequence<Row> process(GroupByQuery query, StorageAdapter storageAdapter)
+  public Sequence<ResultRow> process(GroupByQuery query, StorageAdapter storageAdapter)
   {
     return GroupByQueryEngineV2.process(query, storageAdapter, bufferPool, configSupplier.get().withOverrides(query));
   }
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
index 416fb03..88428c3 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
@@ -61,6 +61,7 @@ import java.util.Map;
 import java.util.function.BinaryOperator;
 
 /**
+ *
  */
 public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<TimeseriesResultValue>, TimeseriesQuery>
 {
@@ -333,7 +334,6 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
             List<Object> results = (List<Object>) input;
             final Map<String, Object> retVal = Maps.newLinkedHashMap();
 
-            Iterator<AggregatorFactory> aggsIter = aggs.iterator();
             Iterator<Object> resultIter = results.iterator();
 
             final Number timestampNumber = (Number) resultIter.next();
@@ -345,12 +345,11 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
             }
 
             CacheStrategy.fetchAggregatorsFromCache(
-                aggsIter,
+                aggs,
                 resultIter,
                 isResultLevelCache,
-                (aggName, aggValueObject) -> {
+                (aggName, aggPosition, aggValueObject) -> {
                   retVal.put(aggName, aggValueObject);
-                  return null;
                 }
             );
 
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java
index 06aa427..3f0b5c7 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java
@@ -58,6 +58,7 @@ import java.util.Map;
 import java.util.function.BinaryOperator;
 
 /**
+ *
  */
 public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultValue>, TopNQuery>
 {
@@ -388,7 +389,6 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
               List<Object> result = (List<Object>) inputIter.next();
               final Map<String, Object> vals = Maps.newLinkedHashMap();
 
-              Iterator<AggregatorFactory> aggIter = aggs.iterator();
               Iterator<Object> resultIter = result.iterator();
 
               // Must convert generic Jackson-deserialized type into the proper type.
@@ -398,12 +398,11 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
               );
 
               CacheStrategy.fetchAggregatorsFromCache(
-                  aggIter,
+                  aggs,
                   resultIter,
                   isResultLevelCache,
-                  (aggName, aggValueObject) -> {
+                  (aggName, aggPos, aggValueObject) -> {
                     vals.put(aggName, aggValueObject);
-                    return null;
                   }
               );
 
diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java
index 8e21296..dc09a56 100644
--- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java
+++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java
@@ -130,7 +130,7 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
       final boolean deserializeComplexMetrics
   )
   {
-    final RowBasedColumnSelectorFactory baseSelectorFactory = RowBasedColumnSelectorFactory.create(in, null);
+    final RowBasedColumnSelectorFactory baseSelectorFactory = RowBasedColumnSelectorFactory.create(in::get, null);
 
     class IncrementalIndexInputRowColumnSelectorFactory implements ColumnSelectorFactory
     {
diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java
index 048299e..1591877 100644
--- a/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java
+++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java
@@ -26,12 +26,10 @@ import org.apache.druid.data.input.impl.InputRowParser;
 import org.apache.druid.data.input.impl.StringInputRowParser;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.query.filter.DimFilter;
-import org.apache.druid.segment.column.ValueType;
 
 import javax.annotation.Nullable;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 
@@ -122,15 +120,6 @@ public class TransformSpec
     return new Transformer(this, null);
   }
 
-  /**
-   * Create a {@link Transformer} from this TransformSpec, taking advantage of the known signature of the rows
-   * to be transformed.
-   */
-  public Transformer toTransformer(@Nullable final Map<String, ValueType> rowSignature)
-  {
-    return new Transformer(this, rowSignature);
-  }
-
   @Override
   public boolean equals(final Object o)
   {
diff --git a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java
index a80eae8..cb1a8b6 100644
--- a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java
+++ b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java
@@ -54,7 +54,7 @@ public class Transformer
       valueMatcher = transformSpec.getFilter().toFilter()
                                   .makeMatcher(
                                       RowBasedColumnSelectorFactory.create(
-                                          rowSupplierForValueMatcher,
+                                          rowSupplierForValueMatcher::get,
                                           rowSignature
                                       )
                                   );
diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
index 2f75700..5689258 100644
--- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
+++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
@@ -26,7 +26,6 @@ import com.google.common.io.Files;
 import org.apache.commons.io.FileUtils;
 import org.apache.druid.collections.CloseableStupidPool;
 import org.apache.druid.common.config.NullHandling;
-import org.apache.druid.data.input.Row;
 import org.apache.druid.data.input.impl.CSVParseSpec;
 import org.apache.druid.data.input.impl.DimensionsSpec;
 import org.apache.druid.data.input.impl.JSONParseSpec;
@@ -47,6 +46,7 @@ import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTestHelper;
+import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
 import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
 import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
@@ -227,7 +227,7 @@ public class MultiValuedDimensionTest
         .setAggregatorSpecs(new CountAggregatorFactory("count"))
         .build();
 
-    Sequence<Row> result = helper.runQueryOnSegmentsObjs(
+    Sequence<ResultRow> result = helper.runQueryOnSegmentsObjs(
         ImmutableList.of(
             new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
             new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
@@ -235,21 +235,22 @@ public class MultiValuedDimensionTest
         query
     );
 
-    List<Row> expectedResults = Arrays.asList(
+    List<ResultRow> expectedResults = Arrays.asList(
         GroupByQueryRunnerTestHelper.createExpectedRow(
-            "1970-01-01T00:00:00.000Z",
+            query,
+            "1970",
             "tags",
             NullHandling.replaceWithDefault() ? null : "",
             "count",
             2L
         ),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t1", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t2", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t3", "count", 4L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t4", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t5", "count", 4L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t6", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t7", "count", 2L)
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tags", "t1", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tags", "t2", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tags", "t3", "count", 4L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tags", "t4", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tags", "t5", "count", 4L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tags", "t6", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tags", "t7", "count", 2L)
     );
 
     TestHelper.assertExpectedObjects(expectedResults, result.toList(), "noFilter");
@@ -269,7 +270,7 @@ public class MultiValuedDimensionTest
         .setContext(context)
         .build();
 
-    Sequence<Row> result = helper.runQueryOnSegmentsObjs(
+    Sequence<ResultRow> result = helper.runQueryOnSegmentsObjs(
         ImmutableList.of(
             new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
             new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
@@ -277,12 +278,12 @@ public class MultiValuedDimensionTest
         query
     );
 
-    List<Row> expectedResults = Arrays.asList(
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t1", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t2", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t3", "count", 4L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t4", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t5", "count", 2L)
+    List<ResultRow> expectedResults = Arrays.asList(
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970-01-01T00:00:00.000Z", "tags", "t1", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970-01-01T00:00:00.000Z", "tags", "t2", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970-01-01T00:00:00.000Z", "tags", "t3", "count", 4L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970-01-01T00:00:00.000Z", "tags", "t4", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970-01-01T00:00:00.000Z", "tags", "t5", "count", 2L)
     );
 
     TestHelper.assertExpectedObjects(expectedResults, result.toList(), "dimFilter");
@@ -302,7 +303,7 @@ public class MultiValuedDimensionTest
         .setContext(context)
         .build();
 
-    Sequence<Row> result = helper.runQueryOnSegmentsObjs(
+    Sequence<ResultRow> result = helper.runQueryOnSegmentsObjs(
         ImmutableList.of(
             new QueryableIndexSegment(queryableIndexNullSampler, SegmentId.dummy("sid1")),
             new IncrementalIndexSegment(incrementalIndexNullSampler, SegmentId.dummy("sid2"))
@@ -310,8 +311,8 @@ public class MultiValuedDimensionTest
         query
     );
 
-    List<Row> expectedResults = Collections.singletonList(
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", null, "count", 2L)
+    List<ResultRow> expectedResults = Collections.singletonList(
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970-01-01T00:00:00.000Z", "tags", null, "count", 2L)
     );
 
     TestHelper.assertExpectedObjects(expectedResults, result.toList(), "filter-empty");
@@ -333,7 +334,7 @@ public class MultiValuedDimensionTest
         .setContext(context)
         .build();
 
-    Sequence<Row> result = helper.runQueryOnSegmentsObjs(
+    Sequence<ResultRow> result = helper.runQueryOnSegmentsObjs(
         ImmutableList.of(
             new QueryableIndexSegment(queryableIndexNullSampler, SegmentId.dummy("sid1")),
             new IncrementalIndexSegment(incrementalIndexNullSampler, SegmentId.dummy("sid2"))
@@ -341,20 +342,20 @@ public class MultiValuedDimensionTest
         query
     );
 
-    List<Row> expectedResults;
+    List<ResultRow> expectedResults;
     // an empty row e.g. [], or group by 'missing' value, is grouped with the default string value, "" or null
     // grouping input is filtered to [], null, [""]
     if (NullHandling.replaceWithDefault()) {
       // when sql compatible null handling is disabled, the inputs are effectively [], null, [null] and
       // are all grouped as null
       expectedResults = Collections.singletonList(
-          GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", null, "count", 6L)
+          GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970-01-01T00:00:00.000Z", "tags", null, "count", 6L)
       );
     } else {
       // with sql compatible null handling, null and [] = null, but [""] = ""
       expectedResults = ImmutableList.of(
-          GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", null, "count", 4L),
-          GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "", "count", 2L)
+          GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970-01-01T00:00:00.000Z", "tags", null, "count", 4L),
+          GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970-01-01T00:00:00.000Z", "tags", "", "count", 2L)
       );
     }
 
@@ -375,7 +376,7 @@ public class MultiValuedDimensionTest
         .setContext(context)
         .build();
 
-    Sequence<Row> result = helper.runQueryOnSegmentsObjs(
+    Sequence<ResultRow> result = helper.runQueryOnSegmentsObjs(
         ImmutableList.of(
             new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
             new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
@@ -383,8 +384,8 @@ public class MultiValuedDimensionTest
         query
     );
 
-    List<Row> expectedResults = Collections.singletonList(
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t3", "count", 4L)
+    List<ResultRow> expectedResults = Collections.singletonList(
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970-01-01T00:00:00.000Z", "tags", "t3", "count", 4L)
     );
 
     TestHelper.assertExpectedObjects(expectedResults, result.toList(), "filteredDim");
@@ -415,7 +416,7 @@ public class MultiValuedDimensionTest
         .setContext(context)
         .build();
 
-    Sequence<Row> result = helper.runQueryOnSegmentsObjs(
+    Sequence<ResultRow> result = helper.runQueryOnSegmentsObjs(
         ImmutableList.of(
             new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
             new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
@@ -423,15 +424,15 @@ public class MultiValuedDimensionTest
         query
     );
 
-    List<Row> expectedResults = Arrays.asList(
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "foo", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t1foo", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t2foo", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t3foo", "count", 4L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t4foo", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t5foo", "count", 4L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t6foo", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t7foo", "count", 2L)
+    List<ResultRow> expectedResults = Arrays.asList(
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "foo", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t1foo", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t2foo", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t3foo", "count", 4L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t4foo", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t5foo", "count", 4L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t6foo", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t7foo", "count", 2L)
     );
 
     TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr");
@@ -463,7 +464,7 @@ public class MultiValuedDimensionTest
         .setContext(context)
         .build();
 
-    Sequence<Row> result = helper.runQueryOnSegmentsObjs(
+    Sequence<ResultRow> result = helper.runQueryOnSegmentsObjs(
         ImmutableList.of(
             new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
             new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
@@ -471,12 +472,12 @@ public class MultiValuedDimensionTest
         query
     );
 
-    List<Row> expectedResults = Arrays.asList(
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t1u1", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t1u2", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t2u1", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t2u2", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t3u1", "count", 2L)
+    List<ResultRow> expectedResults = Arrays.asList(
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t1u1", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t1u2", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t2u1", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t2u2", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t3u1", "count", 2L)
     );
 
     TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-multi-multi");
@@ -508,7 +509,7 @@ public class MultiValuedDimensionTest
         .setContext(context)
         .build();
 
-    Sequence<Row> result = helper.runQueryOnSegmentsObjs(
+    Sequence<ResultRow> result = helper.runQueryOnSegmentsObjs(
         ImmutableList.of(
             new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
             new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
@@ -516,12 +517,12 @@ public class MultiValuedDimensionTest
         query
     );
 
-    List<Row> expectedResults = Arrays.asList(
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t1u1", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t1u2", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t2u1", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t2u2", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t3u1", "count", 2L)
+    List<ResultRow> expectedResults = Arrays.asList(
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t1u1", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t1u2", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t2u1", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t2u2", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t3u1", "count", 2L)
     );
 
     TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-multi-multi-auto");
@@ -553,7 +554,7 @@ public class MultiValuedDimensionTest
         .setContext(context)
         .build();
 
-    Sequence<Row> result = helper.runQueryOnSegmentsObjs(
+    Sequence<ResultRow> result = helper.runQueryOnSegmentsObjs(
         ImmutableList.of(
             new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
             new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
@@ -561,12 +562,12 @@ public class MultiValuedDimensionTest
         query
     );
 
-    List<Row> expectedResults = Arrays.asList(
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t1u1", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t1u2", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t2u1", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t2u2", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t3u1", "count", 2L)
+    List<ResultRow> expectedResults = Arrays.asList(
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t1u1", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t1u2", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t2u1", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t2u2", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t3u1", "count", 2L)
     );
 
     TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-multi-multi-auto-auto");
@@ -598,7 +599,7 @@ public class MultiValuedDimensionTest
         .setContext(context)
         .build();
 
-    Sequence<Row> result = helper.runQueryOnSegmentsObjs(
+    Sequence<ResultRow> result = helper.runQueryOnSegmentsObjs(
         ImmutableList.of(
             new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
             new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
@@ -606,12 +607,12 @@ public class MultiValuedDimensionTest
         query
     );
 
-    List<Row> expectedResults = Arrays.asList(
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t3t3", "count", 4L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t5t5", "count", 4L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t2t1", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t1t2", "count", 2L),
-        GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "texpr", "t7t7", "count", 2L)
+    List<ResultRow> expectedResults = Arrays.asList(
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t3t3", "count", 4L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t5t5", "count", 4L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t2t1", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t1t2", "count", 2L),
+        GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "texpr", "t7t7", "count", 2L)
     );
 
     System.out.println(result.toList());
@@ -645,7 +646,7 @@ public class MultiValuedDimensionTest
         .setContext(context)
         .build();
 
-    Sequence<Row> result = helper.runQueryOnSegmentsObjs(
+    Sequence<ResultRow> result = helper.runQueryOnSegmentsObjs(
         ImmutableList.of(
             new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
             new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
@@ -653,12 +654,12 @@ public class MultiValuedDimensionTest
         query
     );
 
... 11044 lines suppressed ...


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