From 77297f4e6f2e9d617c96cd46852bb5a772961e85 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 31 Jul 2019 16:15:12 -0700 Subject: [PATCH] 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. --- .../GroupByTypeInterfaceBenchmark.java | 116 +- .../benchmark/query/GroupByBenchmark.java | 88 +- .../benchmark/query/SqlVsNativeBenchmark.java | 6 +- .../DistinctCountGroupByQueryTest.java | 8 +- .../MomentsSketchAggregatorTest.java | 24 +- .../movingaverage/MovingAverageQuery.java | 74 +- .../MovingAverageQueryRunner.java | 10 +- .../movingaverage/MovingAverageQueryTest.java | 17 +- .../queryTests/basicGroupByMovingAverage.yaml | 17 +- .../basicGroupByMovingAverage2.yaml | 14 +- .../queryTests/missingGroupByValues.yaml | 24 +- .../queryTests/sortingAveragersAsc.yaml | 29 +- .../queryTests/sortingAveragersDesc.yaml | 29 +- ...ortingWithNonMovingAndMovingAvgMetric.yaml | 24 +- .../sortingWithNonMovingAvgMetric.yaml | 24 +- .../TDigestSketchAggregatorTest.java | 26 +- .../TimestampGroupByAggregationTest.java | 13 +- .../segment/MapVirtualColumnGroupByTest.java | 11 +- .../hll/HllSketchAggregatorTest.java | 50 +- .../DoublesSketchAggregatorTest.java | 60 +- .../theta/SketchAggregationTest.java | 81 +- .../SketchAggregationWithSimpleDataTest.java | 30 +- .../oldapi/OldApiSketchAggregationTest.java | 70 +- .../ArrayOfDoublesSketchAggregationTest.java | 108 +- .../bloom/BloomFilterGroupByQueryTest.java | 6 +- .../ApproximateHistogramAggregationTest.java | 7 +- .../ApproximateHistogramGroupByQueryTest.java | 13 +- .../FixedBucketsHistogramAggregationTest.java | 7 +- ...FixedBucketsHistogramGroupByQueryTest.java | 13 +- .../variance/VarianceGroupByQueryTest.java | 29 +- .../variance/VarianceTestHelper.java | 11 +- .../common/task/RealtimeIndexTask.java | 16 +- .../org/apache/druid/query/BaseQuery.java | 11 +- .../druid/query/BySegmentQueryRunner.java | 8 + .../org/apache/druid/query/CacheStrategy.java | 25 +- .../query/FinalizeResultsQueryRunner.java | 26 +- .../druid/query/GroupByMergedQueryRunner.java | 4 +- .../java/org/apache/druid/query/Query.java | 13 + .../apache/druid/query/QueryToolChest.java | 14 + .../druid/query/groupby/GroupByQuery.java | 358 +- .../query/groupby/GroupByQueryConfig.java | 1 + .../query/groupby/GroupByQueryHelper.java | 94 +- .../groupby/GroupByQueryQueryToolChest.java | 385 +- .../groupby/GroupByQueryRunnerFactory.java | 30 +- .../apache/druid/query/groupby/ResultRow.java | 218 + .../RowBasedColumnSelectorFactory.java | 109 +- .../epinephelinae/GroupByBinaryFnV2.java | 63 +- .../GroupByMergingQueryRunnerV2.java | 82 +- .../epinephelinae/GroupByQueryEngineV2.java | 103 +- .../epinephelinae/GroupByRowProcessor.java | 165 +- .../epinephelinae/RowBasedGrouperHelper.java | 322 +- ...ngStringGroupByColumnSelectorStrategy.java | 11 +- .../DoubleGroupByColumnSelectorStrategy.java | 6 +- .../FloatGroupByColumnSelectorStrategy.java | 6 +- .../column/GroupByColumnSelectorPlus.java | 19 +- .../column/GroupByColumnSelectorStrategy.java | 40 +- .../LongGroupByColumnSelectorStrategy.java | 6 +- ...bleValueGroupByColumnSelectorStrategy.java | 8 +- .../StringGroupByColumnSelectorStrategy.java | 25 +- .../DoubleGroupByVectorColumnSelector.java | 8 +- .../FloatGroupByVectorColumnSelector.java | 8 +- .../vector/GroupByVectorColumnSelector.java | 7 +- .../LongGroupByVectorColumnSelector.java | 8 +- ...alueStringGroupByVectorColumnSelector.java | 8 +- .../vector/VectorGroupByEngine.java | 50 +- .../groupby/having/AlwaysHavingSpec.java | 13 +- .../query/groupby/having/AndHavingSpec.java | 22 +- .../groupby/having/DimFilterHavingSpec.java | 182 +- .../having/DimensionSelectorHavingSpec.java | 22 +- .../groupby/having/EqualToHavingSpec.java | 19 +- .../groupby/having/GreaterThanHavingSpec.java | 19 +- .../query/groupby/having/HavingSpec.java | 23 +- .../query/groupby/having/HavingSpecUtil.java | 13 + .../groupby/having/LessThanHavingSpec.java | 19 +- .../query/groupby/having/NeverHavingSpec.java | 13 +- .../query/groupby/having/NotHavingSpec.java | 21 +- .../query/groupby/having/OrHavingSpec.java | 22 +- .../groupby/orderby/DefaultLimitSpec.java | 138 +- .../query/groupby/orderby/LimitSpec.java | 23 +- .../query/groupby/orderby/NoopLimitSpec.java | 18 +- .../groupby/strategy/GroupByStrategy.java | 133 +- .../groupby/strategy/GroupByStrategyV1.java | 74 +- .../groupby/strategy/GroupByStrategyV2.java | 388 +- .../TimeseriesQueryQueryToolChest.java | 7 +- .../query/topn/TopNQueryQueryToolChest.java | 7 +- .../segment/incremental/IncrementalIndex.java | 2 +- .../segment/transform/TransformSpec.java | 11 - .../druid/segment/transform/Transformer.java | 2 +- .../druid/query/MultiValuedDimensionTest.java | 224 +- .../aggregation/AggregationTestHelper.java | 44 +- .../HyperUniquesAggregationTest.java | 18 +- ...nalizingFieldAccessPostAggregatorTest.java | 12 +- ...ByLimitPushDownInsufficientBufferTest.java | 71 +- ...roupByLimitPushDownMultiNodeMergeTest.java | 73 +- .../groupby/GroupByMultiSegmentTest.java | 26 +- .../groupby/GroupByQueryMergeBufferTest.java | 7 +- .../GroupByQueryQueryToolChestTest.java | 188 +- .../GroupByQueryRunnerFactoryTest.java | 12 +- .../GroupByQueryRunnerFailureTest.java | 7 +- .../query/groupby/GroupByQueryRunnerTest.java | 3864 +++++++++-------- .../groupby/GroupByQueryRunnerTestHelper.java | 53 +- .../druid/query/groupby/GroupByQueryTest.java | 9 +- .../GroupByTimeseriesQueryRunnerTest.java | 14 +- .../groupby/NestedQueryPushDownTest.java | 87 +- .../druid/query/groupby/ResultRowTest.java} | 27 +- .../having/DimFilterHavingSpecTest.java | 47 +- .../DimensionSelectorHavingSpecTest.java | 7 +- .../query/groupby/having/HavingSpecTest.java | 29 +- .../groupby/orderby/DefaultLimitSpecTest.java | 197 +- .../org/apache/druid/segment/TestHelper.java | 70 +- .../druid/segment/filter/BaseFilterTest.java | 2 +- .../virtual/DummyStringVirtualColumnTest.java | 32 +- .../virtual/ExpressionVirtualColumnTest.java | 8 +- .../druid/client/DirectDruidClient.java | 10 +- .../appenderator/SinkQuerySegmentWalker.java | 8 +- .../apache/druid/server/QueryLifecycle.java | 9 + .../apache/druid/server/QueryResource.java | 66 +- .../client/CachingClusteredClientTest.java | 81 +- .../druid/sql/calcite/rel/QueryMaker.java | 33 +- .../druid/sql/calcite/util/CalciteTests.java | 73 +- 120 files changed, 5481 insertions(+), 4414 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/groupby/ResultRow.java rename processing/src/{main/java/org/apache/druid/query/groupby/having/BaseHavingSpec.java => test/java/org/apache/druid/query/groupby/ResultRowTest.java} (59%) 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 e4cf5ac030d..ccd219d1c09 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 queryableIndexes; - private QueryRunnerFactory factory; + private QueryRunnerFactory 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + List 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, Q_INDEX_SEGMENT_ID, new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID) ); - List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + List 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 9076ff966c0..fe1e3e2466d 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 queryableIndexes; - private QueryRunnerFactory factory; + private QueryRunnerFactory 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 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 queryAggs = new ArrayList<>(); @@ -562,14 +593,14 @@ public class GroupByBenchmark @OutputTimeUnit(TimeUnit.MICROSECONDS) public void querySingleIncrementalIndex(Blackhole blackhole) { - QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, SegmentId.dummy("incIndex"), new IncrementalIndexSegment(anIncrementalIndex, SegmentId.dummy("incIndex")) ); - final Sequence results = GroupByBenchmark.runQuery(factory, runner, query); - final Row lastRow = results.accumulate( + final Sequence 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 runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( factory, SegmentId.dummy("qIndex"), new QueryableIndexSegment(queryableIndexes.get(0), SegmentId.dummy("qIndex")) ); - final Sequence results = GroupByBenchmark.runQuery(factory, runner, query); - final Row lastRow = results.accumulate( + final Sequence 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 toolChest = factory.getToolchest(); - QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + QueryToolChest toolChest = factory.getToolchest(); + QueryRunner theRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( factory.mergeRunners(executorService, makeMultiRunners()) ), (QueryToolChest) toolChest ); - Sequence queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); - List results = queryResult.toList(); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + List results = queryResult.toList(); blackhole.consume(results); } @@ -620,8 +651,8 @@ public class GroupByBenchmark @OutputTimeUnit(TimeUnit.MICROSECONDS) public void queryMultiQueryableIndexWithSpilling(Blackhole blackhole) { - QueryToolChest toolChest = factory.getToolchest(); - QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + QueryToolChest toolChest = factory.getToolchest(); + QueryRunner 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 queryResult = theRunner.run(QueryPlus.wrap(spillingQuery), ResponseContext.createEmpty()); - List results = queryResult.toList(); + Sequence queryResult = theRunner.run(QueryPlus.wrap(spillingQuery), ResponseContext.createEmpty()); + List results = queryResult.toList(); blackhole.consume(results); } @@ -641,12 +672,13 @@ public class GroupByBenchmark @OutputTimeUnit(TimeUnit.MICROSECONDS) public void queryMultiQueryableIndexWithSerde(Blackhole blackhole) { - QueryToolChest toolChest = factory.getToolchest(); - QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + QueryToolChest toolChest = factory.getToolchest(); + //noinspection unchecked + QueryRunner 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 queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); - List results = queryResult.toList(); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + List results = queryResult.toList(); blackhole.consume(results); } - private List> makeMultiRunners() + private List> makeMultiRunners() { - List> runners = new ArrayList<>(); + List> runners = new ArrayList<>(); for (int i = 0; i < numSegments; i++) { String segmentName = "qIndex" + i; - QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + QueryRunner 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 47ae6caf1c8..8021a9909e2 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 resultSequence = QueryPlus.wrap(groupByQuery).run(walker, ResponseContext.createEmpty()); - final Row lastRow = resultSequence.accumulate(null, (accumulated, in) -> in); + final Sequence 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 dc4f1daca48..96e7148d551 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 results = GroupByQueryRunnerTestHelper.runQuery( + Iterable results = GroupByQueryRunnerTestHelper.runQuery( factory, factory.createRunner(incrementalIndexSegment), query ); - List expectedResults = Arrays.asList( + List 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 922075ffac6..0299a0dd42c 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 38fc1ebcc12..46689d0900a 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 private final LimitSpec limitSpec; private final HavingSpec havingSpec; private final DimFilter dimFilter; - private final Function, Sequence> limitFn; private final Granularity granularity; private final List dimensions; private final List aggregatorSpecs; @@ -75,6 +77,16 @@ public class MovingAverageQuery extends BaseQuery private final List> averagerSpecs; private final List 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> limitFn; + @JsonCreator public MovingAverageQuery( @JsonProperty("dataSource") DataSource dataSource, @@ -94,7 +106,10 @@ public class MovingAverageQuery extends BaseQuery 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 combinedAggregatorSpecs.add(new AveragerFactoryWrapper(avg, "")); } - Function, Sequence> 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> postProcFn = this.limitSpec.build(groupByQueryForLimitSpec); if (havingSpec != null) { postProcFn = Functions.compose( postProcFn, - new Function, Sequence>() - { - @Override - public Sequence apply(Sequence input) - { - return Sequences.filter( - input, - new Predicate() - { - @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 List postAveragerSpecs, HavingSpec havingSpec, LimitSpec orderBySpec, - Function, Sequence> limitFn, + GroupByQuery groupByQueryForLimitSpec, + Function, Sequence> limitFn, Map context ) { @@ -215,6 +219,7 @@ public class MovingAverageQuery extends BaseQuery this.postAveragerSpecs = postAveragerSpecs; this.havingSpec = havingSpec; this.limitSpec = orderBySpec; + this.groupByQueryForLimitSpec = groupByQueryForLimitSpec; this.limitFn = limitFn; } @@ -307,6 +312,7 @@ public class MovingAverageQuery extends BaseQuery postAveragerSpecs, havingSpec, limitSpec, + groupByQueryForLimitSpec, limitFn, computeOverridenContext(contextOverride) ); @@ -327,6 +333,7 @@ public class MovingAverageQuery extends BaseQuery postAveragerSpecs, havingSpec, limitSpec, + groupByQueryForLimitSpec, limitFn, getContext() ); @@ -347,6 +354,7 @@ public class MovingAverageQuery extends BaseQuery postAveragerSpecs, havingSpec, limitSpec, + groupByQueryForLimitSpec, limitFn, getContext() ); @@ -366,6 +374,7 @@ public class MovingAverageQuery extends BaseQuery postAveragerSpecs, havingSpec, limitSpec, + groupByQueryForLimitSpec, limitFn, getContext() ); @@ -373,6 +382,7 @@ public class MovingAverageQuery extends BaseQuery public Sequence applyLimit(Sequence 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 53c4422575a..645a3b100d7 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 gbqResponseContext.put(QUERY_FAIL_TIME, System.currentTimeMillis() + QueryContexts.getTimeout(gbq)); gbqResponseContext.put(QUERY_TOTAL_BYTES_GATHERED, new AtomicLong()); - Sequence results = gbq.getRunner(walker).run(QueryPlus.wrap(gbq), gbqResponseContext); + Sequence 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 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 maq.getAveragerSpecs(), maq.getPostAggregatorSpecs(), maq.getAggregatorSpecs() - )); + ) + ); // Apply any postAveragers Sequence movingAvgResultsWithPostAveragers = @@ -216,7 +218,7 @@ public class MovingAverageQueryRunner implements QueryRunner ); // 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 625e01f93cf..762f975b36a 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>() - { - })); + groupByResults.addAll(jsonMapper.readValue(getGroupByResultJson(), new TypeReference>() {})); } if (getTimeseriesResultJson() != null) { - timeseriesResults.addAll(jsonMapper.readValue( - getTimeseriesResultJson(), - new TypeReference>>() - { - } - )); + timeseriesResults.addAll( + jsonMapper.readValue( + getTimeseriesResultJson(), + new TypeReference>>() {} + ) + ); } } 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 a3d2d164d2d..fba40e85a64 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 a3d2d164d2d..ac422089925 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 c4ab5a4fa8a..c4bb0c5ef25 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 ba685ff8411..161861c3816 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 59f75bc0480..08bf53cfee2 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 c7d7ddc0511..4b438c44132 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 89ae94155e9..0e8c635103b 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 88adde075e1..67773b165fd 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 1a1165399ef..5a406411354 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence seq = helper.createIndexAndRunQueryOnSegment( zip.getInputStream(zip.getEntry("druid.sample.tsv")), recordParser, aggregator, @@ -185,8 +185,11 @@ public class TimestampGroupByAggregationTest groupBy ); - List results = seq.toList(); + int groupByFieldNumber = ((GroupByQuery) helper.readQuery(groupBy)).getResultRowPositionLookup() + .getInt(groupByField); + + List 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 320f7d3a115..01631e3639c 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 runner; + private QueryRunner runner; @Before public void setup() throws IOException @@ -161,14 +162,14 @@ public class MapVirtualColumnGroupByTest null ); - final List result = runner.run(QueryPlus.wrap(query)).toList(); - final List expected = ImmutableList.of( + final List result = runner.run(QueryPlus.wrap(query)).toList(); + final List 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 1dc64cc4fb8..e2c55be5355 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 dimensions, List 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 a29f6493dab..2a2cc85ccad 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = timeSeriesHelper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 a2a4bad60a7..980a0932edc 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 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 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 results = seq.toList(); + List results = seq.toList(); Assert.assertEquals(1, results.size()); Assert.assertEquals( - new MapBasedRow( - DateTimes.of("2014-10-19T00:00:00.000Z"), - ImmutableMap - .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 + .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 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 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 results = seq.toList(); + List 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 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 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 results = seq.toList(); + List 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 0ee0e49ab1c..8581eb1776e 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 seq = gpByQueryAggregationTestHelper.runQueryOnSegments( ImmutableList.of(s1, s2), - readFileFromClasspathAsString("simple_test_data_group_by_query.json") + groupByQueryString ); - List results = seq.toList(); + List 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 result = (Result) 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 result = (Result) 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 91550615a1c..66e0b88501c 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 - .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 + .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 - .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 + .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 bb1e70c40ed..cc599d62ba0 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 seq = helper.createIndexAndRunQueryOnSegment( + Sequence 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 results = seq.toList(); + List 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 5661e7b9e47..003dc27ca58 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 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 36103b94ceb..01cef7d1f7a 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 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 f5323e0f88e..245fde24a27 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 runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { final String testName = StringUtils.format( "config=%s, runner=%s", config.toString(), @@ -187,8 +188,9 @@ public class ApproximateHistogramGroupByQueryTest ) .build(); - List expectedResults = Collections.singletonList( + List expectedResults = Collections.singletonList( GroupByQueryRunnerTestHelper.createExpectedRow( + query, "1970-01-01T00:00:00.000Z", "marketalias", "upfront", "rows", 186L, @@ -210,7 +212,7 @@ public class ApproximateHistogramGroupByQueryTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "approx-histo"); } @@ -247,8 +249,9 @@ public class ApproximateHistogramGroupByQueryTest ) .build(); - List expectedResults = Collections.singletonList( + List expectedResults = Collections.singletonList( GroupByQueryRunnerTestHelper.createExpectedRow( + query, "1970-01-01T00:00:00.000Z", "marketalias", "upfront", "rows", 186L, @@ -256,7 +259,7 @@ public class ApproximateHistogramGroupByQueryTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable 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 1d817375131..5fbb694e393 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 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 d7761afd64e..929a3469fc0 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 runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { final String testName = StringUtils.format( "config=%s, runner=%s", config.toString(), @@ -188,8 +189,9 @@ public class FixedBucketsHistogramGroupByQueryTest ) .build(); - List expectedResults = Collections.singletonList( + List expectedResults = Collections.singletonList( GroupByQueryRunnerTestHelper.createExpectedRow( + query, "1970-01-01T00:00:00.000Z", "marketalias", "upfront", "rows", 186L, @@ -211,7 +213,7 @@ public class FixedBucketsHistogramGroupByQueryTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "fixed-histo"); } @@ -248,8 +250,9 @@ public class FixedBucketsHistogramGroupByQueryTest ) .build(); - List expectedResults = Collections.singletonList( + List expectedResults = Collections.singletonList( GroupByQueryRunnerTestHelper.createExpectedRow( + query, "1970-01-01T00:00:00.000Z", "marketalias", "upfront", "rows", 186L, @@ -257,7 +260,7 @@ public class FixedBucketsHistogramGroupByQueryTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable 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 b302aef03f3..d769eff75a4 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 expectedResults = builder + List 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 results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable 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 expectedResults = builder + List 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 results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable 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 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 results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + List 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 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 a46b6c3c9fd..e6e9f007519 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 build() + public List 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 c0c7c139948..0dc3ee53633 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 QueryRunner getQueryRunner(Query query) { - if (plumber != null) { - QueryRunnerFactory> factory = queryRunnerFactoryConglomerate.findFactory(query); - QueryToolChest> toolChest = factory.getToolchest(); - - return new FinalizeResultsQueryRunner(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 1f4993ca937..cee136275c3 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> implements Query +public abstract class BaseQuery implements Query { public static void checkInterrupted() { @@ -216,10 +217,16 @@ public abstract class BaseQuery> implements Query 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 getResultOrdering() { - Ordering 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 eced1f35663..cd386c7a6ba 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>}, 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 implements QueryRunner { 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 f93a3955aa6..5d7d94fd2c4 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> @@ -113,19 +115,28 @@ public interface CacheStrategy> * for dimension values (e.g., a Float would become Double). */ static void fetchAggregatorsFromCache( - Iterator aggIter, + List aggregators, Iterator resultIter, boolean isResultLevelCache, - BiFunction 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 ebb18f2c2f8..d63dcdd88de 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>}, 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 implements QueryRunner { @@ -53,19 +60,19 @@ public class FinalizeResultsQueryRunner implements QueryRunner final boolean shouldFinalize = QueryContexts.isFinalize(query, true); final Query queryToRun; - final Function finalizerFn; + final Function 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() + finalizerFn = new Function>>() { final Function baseFinalizer = toolChest.makePostComputeManipulatorFn( query, @@ -73,9 +80,9 @@ public class FinalizeResultsQueryRunner implements QueryRunner ); @Override - @SuppressWarnings("unchecked") - public T apply(T input) + public Result> apply(T input) { + //noinspection unchecked (input is not actually a T; see class-level javadoc) Result> result = (Result>) input; if (input == null) { @@ -84,9 +91,9 @@ public class FinalizeResultsQueryRunner implements QueryRunner BySegmentResultValue 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 implements QueryRunner finalizerFn = toolChest.makePostComputeManipulatorFn(query, metricManipulationFn); } - - return Sequences.map( + //noinspection unchecked (Technically unsound, but see class-level javadoc for rationale) + return (Sequence) 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 9c46d601a1c..90e9f6f3ff5 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 implements QueryRunner final boolean isSingleThreaded = querySpecificConfig.isSingleThreaded(); final Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( query, + null, querySpecificConfig, - bufferPool, - true + bufferPool ); final Pair> 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 36f88fea6e5..91303097fcd 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 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} + * 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 getResultOrdering(); Query withOverriddenContext(Map 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 a8a20a67033..4c760b02cdf 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; @@ -75,6 +76,19 @@ public abstract class QueryToolChest +public class GroupByQuery extends BaseQuery { public static final String CTX_KEY_SORT_BY_DIMS_FIRST = "sortByDimsFirst"; + private static final String CTX_KEY_FUDGE_TIMESTAMP = "fudgeTimestamp"; - private static final Comparator NON_GRANULAR_TIME_COMP = (Row lhs, Row rhs) -> Longs.compare( - lhs.getTimestampFromEpoch(), - rhs.getTimestampFromEpoch() - ); + private static final Comparator 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 private final List> subtotalsSpec; private final boolean applyLimitPushDown; - private final Function, Sequence> postProcessingFn; + private final Function, Sequence> postProcessingFn; + private final List resultRowOrder; + private final Object2IntMap 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 dimensions, @JsonProperty("aggregations") List aggregatorSpecs, @@ -141,22 +154,15 @@ public class GroupByQuery extends BaseQuery ); } - private Function, Sequence> makePostProcessingFn() + private Function, Sequence> makePostProcessingFn() { - Function, Sequence> postProcessingFn = limitSpec.build( - dimensions, - aggregatorSpecs, - postAggregatorSpecs, - getGranularity(), - getContextSortByDimsFirst() - ); + Function, Sequence> postProcessingFn = limitSpec.build(this); if (havingSpec != null) { postProcessingFn = Functions.compose( postProcessingFn, - (Sequence input) -> { - havingSpec.setRowSignature(GroupByQueryHelper.rowSignatureFor(GroupByQuery.this)); - havingSpec.setAggregators(getAggregatorsMap(aggregatorSpecs)); + (Sequence input) -> { + havingSpec.setQuery(this); return Sequences.filter(input, havingSpec::eval); } ); @@ -179,7 +185,7 @@ public class GroupByQuery extends BaseQuery final @Nullable HavingSpec havingSpec, final LimitSpec limitSpec, final @Nullable List> subtotalsSpec, - final @Nullable Function, Sequence> postProcessingFn, + final @Nullable Function, Sequence> postProcessingFn, final Map context ) { @@ -198,9 +204,12 @@ public class GroupByQuery extends BaseQuery 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 @JsonInclude(JsonInclude.Include.NON_NULL) @JsonProperty("subtotalsSpec") + @Nullable public List> 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 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 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 } @Override + @Nullable public DimFilter getFilter() { return dimFilter; @@ -341,14 +439,14 @@ public class GroupByQuery extends BaseQuery @Override public Ordering getResultOrdering() { - final Ordering rowOrdering = getRowOrdering(false); + final Ordering 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 return forcePushDown; } - public boolean determineApplyLimitPushDown() + private Object2IntMap computeResultRowOrderLookup() + { + final Object2IntMap indexes = new Object2IntOpenHashMap<>(); + indexes.defaultReturnValue(-1); + + int index = 0; + for (String columnName : resultRowOrder) { + indexes.put(columnName, index++); + } + + return indexes; + } + + private List computeResultRowOrder() + { + final List 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 * 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 getRowOrderingForPushDown( + private Ordering getRowOrderingForPushDown( final boolean granular, final DefaultLimitSpec limitSpec ) { final boolean sortByDimsFirst = getContextSortByDimsFirst(); - final List orderedFieldNames = new ArrayList<>(); + final IntList orderedFieldNumbers = new IntArrayList(); final Set dimsInOrderBy = new HashSet<>(); final List needsReverseList = new ArrayList<>(); final List dimensionTypes = new ArrayList<>(); @@ -444,7 +570,7 @@ public class GroupByQuery extends BaseQuery 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 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 } } - final Comparator timeComparator = getTimeComparator(granular); + final Comparator timeComparator = getTimeComparator(granular); if (timeComparator == null) { return Ordering.from( - new Comparator() - { - @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() - { - @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() - { - @Override - public int compare(Row lhs, Row rhs) - { - final int timeCompare = timeComparator.compare(lhs, rhs); + (lhs, rhs) -> { + final int timeCompare = timeComparator.compare(lhs, rhs); - if (timeCompare != 0) { - return timeCompare; - } - - return compareDimsForLimitPushDown( - orderedFieldNames, - needsReverseList, - dimensionTypes, - comparators, - lhs, - rhs - ); + if (timeCompare != 0) { + return timeCompare; } + + return compareDimsForLimitPushDown( + orderedFieldNumbers, + needsReverseList, + dimensionTypes, + comparators, + lhs, + rhs + ); } ); } } - public Ordering getRowOrdering(final boolean granular) + public Ordering getRowOrdering(final boolean granular) { if (applyLimitPushDown) { if (!DefaultLimitSpec.sortingOrderHasNonGroupingFields((DefaultLimitSpec) limitSpec, dimensions)) { @@ -542,7 +651,7 @@ public class GroupByQuery extends BaseQuery } final boolean sortByDimsFirst = getContextSortByDimsFirst(); - final Comparator timeComparator = getTimeComparator(granular); + final Comparator timeComparator = getTimeComparator(granular); if (timeComparator == null) { return Ordering.from((lhs, rhs) -> compareDims(dimensions, lhs, rhs)); @@ -573,27 +682,36 @@ public class GroupByQuery extends BaseQuery } @Nullable - private Comparator getTimeComparator(boolean granular) + private Comparator 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 dimensions, Row lhs, Row rhs) + private int compareDims(List 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 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 fields, + final IntList fields, final List needsReverseList, final List dimensionTypes, final List 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 * * @return sequence of rows after applying havingSpec and limitSpec */ - public Sequence postProcess(Sequence results) + public Sequence postProcess(Sequence results) { return postProcessingFn.apply(results); } @@ -671,7 +808,7 @@ public class GroupByQuery extends BaseQuery } @Override - public Query withDataSource(DataSource dataSource) + public Query withDataSource(DataSource dataSource) { return new Builder(this).setDataSource(dataSource).build(); } @@ -734,13 +871,6 @@ public class GroupByQuery extends BaseQuery } } - private static Map getAggregatorsMap(List aggregatorSpecs) - { - Map 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 @Nullable private LimitSpec limitSpec = null; @Nullable - private Function, Sequence> postProcessingFn; + private Function, Sequence> postProcessingFn; private List orderByColumnSpecs = new ArrayList<>(); private int limit = Integer.MAX_VALUE; @@ -1020,7 +1150,7 @@ public class GroupByQuery extends BaseQuery 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 13107fba68e..b272ab16581 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 10f582c5508..28147fb1579 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 Pair> createIndexAccumulatorPair( final GroupByQuery query, + @Nullable final GroupByQuery subquery, final GroupByQueryConfig config, - NonBlockingPool bufferPool, - final boolean combine + NonBlockingPool 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 bufferPool, - Sequence rows, - boolean combine + Sequence rows ) { - Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( - query, - config, - bufferPool, - combine - ); + final Pair> indexAccumulatorPair = + GroupByQueryHelper.createIndexAccumulatorPair(query, subquery, config, bufferPool); return rows.accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); } // Used by GroupByStrategyV1 - public static Sequence postAggregate(final GroupByQuery query, IncrementalIndex index) + public static Sequence postAggregate(final GroupByQuery query, IncrementalIndex index) { return Sequences.map( Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs(), query.isDescending())), - new Function() - { - @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 1b58b207ee9..b942660fe4c 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 +public class GroupByQueryQueryToolChest extends QueryToolChest { private static final byte GROUPBY_QUERY = 0x14; private static final TypeReference OBJECT_TYPE_REFERENCE = new TypeReference() { }; - private static final TypeReference TYPE_REFERENCE = new TypeReference() + private static final TypeReference TYPE_REFERENCE = new TypeReference() { }; public static final String GROUP_BY_MERGE_KEY = "groupByMerge"; @@ -116,7 +117,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeResults(final QueryRunner runner) + public QueryRunner mergeResults(final QueryRunner runner) { return (queryPlus, responseContext) -> { if (QueryContexts.isBySegment(queryPlus.getQuery())) { @@ -132,34 +133,48 @@ public class GroupByQueryQueryToolChest extends QueryToolChest createMergeFn(Query query) + public BinaryOperator createMergeFn(Query query) { return strategySelector.strategize((GroupByQuery) query).createMergeFn(query); } @Override - public Comparator createResultComparator(Query query) + public Comparator createResultComparator(Query query) { return strategySelector.strategize((GroupByQuery) query).createResultComparator(query); } - private Sequence initAndMergeGroupByResults( + private Sequence initAndMergeGroupByResults( final GroupByQuery query, - QueryRunner runner, + QueryRunner 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 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 mergeGroupByResults( + private Sequence mergeGroupByResults( GroupByStrategy groupByStrategy, final GroupByQuery query, GroupByQueryResource resource, - QueryRunner runner, + QueryRunner runner, ResponseContext context ) { @@ -169,11 +184,11 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeGroupByResultsWithoutPushDown( + private Sequence mergeGroupByResultsWithoutPushDown( GroupByStrategy groupByStrategy, GroupByQuery query, GroupByQueryResource resource, - QueryRunner runner, + QueryRunner runner, ResponseContext context ) { @@ -204,7 +219,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest subqueryResult = mergeGroupByResults( + final Sequence subqueryResult = mergeGroupByResults( groupByStrategy, subquery.withOverriddenContext( ImmutableMap.of( @@ -219,7 +234,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest finalizingResults = finalizeSubqueryResults(subqueryResult, subquery); + final Sequence finalizingResults = finalizeSubqueryResults(subqueryResult, subquery); if (query.getSubtotalsSpec() != null) { return groupByStrategy.processSubtotalsSpec( @@ -250,16 +265,16 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeResultsWithNestedQueryPushDown( + private Sequence mergeResultsWithNestedQueryPushDown( GroupByStrategy groupByStrategy, GroupByQuery query, GroupByQueryResource resource, - QueryRunner runner, + QueryRunner runner, ResponseContext context ) { - Sequence pushDownQueryResults = groupByStrategy.mergeResults(runner, query, context); - final Sequence finalizedResults = finalizeSubqueryResults(pushDownQueryResults, query); + Sequence pushDownQueryResults = groupByStrategy.mergeResults(runner, query, context); + final Sequence finalizedResults = finalizeSubqueryResults(pushDownQueryResults, query); GroupByQuery rewrittenQuery = rewriteNestedQueryForPushDown(query); return groupByStrategy.applyPostProcessing(groupByStrategy.processSubqueryResult( query, @@ -288,9 +303,9 @@ public class GroupByQueryQueryToolChest extends QueryToolChest finalizeSubqueryResults(Sequence subqueryResult, GroupByQuery subquery) + private Sequence finalizeSubqueryResults(Sequence subqueryResult, GroupByQuery subquery) { - final Sequence finalizingResults; + final Sequence finalizingResults; if (QueryContexts.isFinalize(subquery, false)) { finalizingResults = new MappedSequence<>( subqueryResult, @@ -322,7 +337,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest makePreComputeManipulatorFn( + public Function makePreComputeManipulatorFn( final GroupByQuery query, final MetricManipulationFn fn ) @@ -331,116 +346,149 @@ public class GroupByQueryQueryToolChest extends QueryToolChest() - { - @Override - public Row apply(Row input) - { - if (input instanceof MapBasedRow) { - final MapBasedRow inputRow = (MapBasedRow) input; - final Map 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 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 makePostComputeManipulatorFn( + public Function makePostComputeManipulatorFn( final GroupByQuery query, final MetricManipulationFn fn ) { - final Set optimizedDims = ImmutableSet.copyOf( - Iterables.transform( - extractionsToRewrite(query), - new Function() - { - @Override - public String apply(DimensionSpec input) - { - return input.getOutputName(); - } - } - ) - ); - final Function preCompute = makePreComputeManipulatorFn(query, fn); + final BitSet optimizedDims = extractionsToRewrite(query); + final Function 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 extractionFnMap = new HashMap<>(); - for (DimensionSpec dimensionSpec : query.getDimensions()) { - final String dimension = dimensionSpec.getOutputName(); - if (optimizedDims.contains(dimension)) { - extractionFnMap.put(dimension, dimensionSpec.getExtractionFn()); + final List dimensions = query.getDimensions(); + final List 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() - { - @Nullable - @Override - public Row apply(Row input) - { - Row preRow = preCompute.apply(input); - if (preRow instanceof MapBasedRow) { - MapBasedRow preMapRow = (MapBasedRow) preRow; - Map 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); - } else { - return preRow; - } + 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 getResultTypeReference() + public TypeReference getResultTypeReference() { return TYPE_REFERENCE; } @Override - public QueryRunner preMergeQueryDecoration(final QueryRunner runner) + 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 serializer = new JsonSerializer() + { + @Override + public void serialize( + final ResultRow resultRow, + final JsonGenerator jg, + final SerializerProvider serializers + ) throws IOException + { + if (resultAsArray) { + jg.writeObject(resultRow.getArray()); + } else { + jg.writeObject(resultRow.toMapBasedRow(query)); + } + } + }; + + // Deserializer that can deserialize either array- or map-based rows. + final JsonDeserializer deserializer = new JsonDeserializer() + { + @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 preMergeQueryDecoration(final QueryRunner runner) { return new SubqueryQueryRunner<>( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { GroupByQuery groupByQuery = (GroupByQuery) queryPlus.getQuery(); if (groupByQuery.getDimFilter() != null) { groupByQuery = groupByQuery.withDimFilter(groupByQuery.getDimFilter().optimize()); } final GroupByQuery delegateGroupByQuery = groupByQuery; - ArrayList dimensionSpecs = new ArrayList<>(); - Set optimizedDimensions = ImmutableSet.copyOf( - Iterables.transform( - extractionsToRewrite(delegateGroupByQuery), - new Function() - { - @Override - public String apply(DimensionSpec input) - { - return input.getDimension(); - } - } - ) - ); - for (DimensionSpec dimensionSpec : delegateGroupByQuery.getDimensions()) { - if (optimizedDimensions.contains(dimensionSpec.getDimension())) { + final List dimensionSpecs = new ArrayList<>(); + final BitSet optimizedDimensions = extractionsToRewrite(delegateGroupByQuery); + final List 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 getCacheStrategy(final GroupByQuery query) + public CacheStrategy getCacheStrategy(final GroupByQuery query) { - return new CacheStrategy() + return new CacheStrategy() { private static final byte CACHE_STRATEGY_VERSION = 0x1; private final List aggs = query.getAggregatorSpecs(); @@ -521,93 +569,106 @@ public class GroupByQueryQueryToolChest extends QueryToolChest prepareForCache(boolean isResultLevelCache) + public Function prepareForCache(boolean isResultLevelCache) { - return new Function() + final boolean resultRowHasTimestamp = query.getResultRowHasTimestamp(); + + return new Function() { @Override - public Object apply(Row input) + public Object apply(ResultRow resultRow) { - if (input instanceof MapBasedRow) { - final MapBasedRow row = (MapBasedRow) input; - final List retVal = Lists.newArrayListWithCapacity(1 + dims.size() + aggs.size()); - retVal.add(row.getTimestamp().getMillis()); - Map 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 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 pullFromCache(boolean isResultLevelCache) + public Function pullFromCache(boolean isResultLevelCache) { - return new Function() + final boolean resultRowHasTimestamp = query.getResultRowHasTimestamp(); + final int dimensionStart = query.getResultRowDimensionStart(); + final int aggregatorStart = query.getResultRowAggregatorStart(); + final int postAggregatorStart = query.getResultRowPostAggregatorStart(); + + return new Function() { private final Granularity granularity = query.getGranularity(); @Override - public Row apply(Object input) + public ResultRow apply(Object input) { Iterator results = ((List) input).iterator(); DateTime timestamp = granularity.toDateTime(((Number) results.next()).longValue()); - final Map event = Maps.newLinkedHashMap(); - Iterator 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 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 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 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 extractionsToRewrite(GroupByQuery query) + private static BitSet extractionsToRewrite(GroupByQuery query) { - return Collections2.filter( - query.getDimensions(), new Predicate() - { - @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 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 6d7c5efbc4c..3fbeb25aa7d 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 +public class GroupByQueryRunnerFactory implements QueryRunnerFactory { private final GroupByStrategySelector strategySelector; private final GroupByQueryQueryToolChest toolChest; @@ -56,38 +56,40 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory createRunner(final Segment segment) + public QueryRunner createRunner(final Segment segment) { return new GroupByQueryRunner(segment, strategySelector); } @Override - public QueryRunner mergeRunners(final ExecutorService exec, final Iterable> queryRunners) + public QueryRunner mergeRunners( + final ExecutorService exec, + final Iterable> queryRunners + ) { // mergeRunners should take ListeningExecutorService at some point final ListeningExecutorService queryExecutor = MoreExecutors.listeningDecorator(exec); - return new QueryRunner() + return new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { - QueryRunner rowQueryRunner = strategySelector.strategize((GroupByQuery) queryPlus.getQuery()).mergeRunners( - queryExecutor, - queryRunners - ); + QueryRunner rowQueryRunner = strategySelector + .strategize((GroupByQuery) queryPlus.getQuery()) + .mergeRunners(queryExecutor, queryRunners); return rowQueryRunner.run(queryPlus, responseContext); } }; } @Override - public QueryToolChest getToolchest() + public QueryToolChest getToolchest() { return toolChest; } - private static class GroupByQueryRunner implements QueryRunner + private static class GroupByQueryRunner implements QueryRunner { private final StorageAdapter adapter; private final GroupByStrategySelector strategySelector; @@ -99,9 +101,9 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { - Query query = queryPlus.getQuery(); + Query 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 00000000000..b31228c2db6 --- /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 toMap(final GroupByQuery query) + { + final List resultRowOrder = query.getResultRowOrder(); + final Map 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 34e5596b1e8..b98dcde1167 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 implements ColumnSelectorFactory { - private final Supplier row; + public interface RowAdapter + { + ToLongFunction timestampFunction(); + + Function rawFunction(String columnName); + } + + private final Supplier supplier; + private final RowAdapter adapter; private final Map rowSignature; private RowBasedColumnSelectorFactory( - final Supplier row, + final Supplier supplier, + final RowAdapter adapter, @Nullable final Map rowSignature ) { - this.row = row; + this.supplier = supplier; + this.adapter = adapter; this.rowSignature = rowSignature != null ? rowSignature : ImmutableMap.of(); } - public static RowBasedColumnSelectorFactory create( - final Supplier row, - @Nullable final Map rowSignature + public static RowBasedColumnSelectorFactory create( + final Supplier supplier, + @Nullable final Map signature ) { - return new RowBasedColumnSelectorFactory(row, rowSignature); + final RowAdapter adapter = new RowAdapter() + { + @Override + public ToLongFunction timestampFunction() + { + return Row::getTimestampFromEpoch; + } + + @Override + public Function rawFunction(String columnName) + { + return r -> r.getRaw(columnName); + } + }; + + return new RowBasedColumnSelectorFactory<>(supplier, adapter, signature); } - public static RowBasedColumnSelectorFactory create( - final ThreadLocal row, - @Nullable final Map rowSignature + public static RowBasedColumnSelectorFactory create( + final RowAdapter adapter, + final Supplier supplier, + @Nullable final Map 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 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 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 dimensionValues = row.get().getDimension(dimension); + final List 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 dimensionValues = row.get().getDimension(dimension); + final List 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 dimensionValues = row.get().getDimension(dimension); + final List 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 dimensionValues = row.get().getDimension(dimension); + final List 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 dimensionValues = row.get().getDimension(dimension); + final List 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 dimensionValues = row.get().getDimension(dimension); + List 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 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 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 6071adce628..71342aa7b76 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 +/** + * 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 { private final GroupByQuery query; @@ -41,7 +47,8 @@ public class GroupByBinaryFnV2 implements BinaryOperator } @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 return arg1; } - final Map newMap = Maps.newHashMapWithExpectedSize( - query.getDimensions().size() + query.getAggregatorSpecs().size() - ); + final ResultRow newResult = ResultRow.create(query.getResultRowSizeWithoutPostAggregators()); - // Add dimensions - for (DimensionSpec dimension : query.getDimensions()) { - newMap.put(dimension.getOutputName(), arg1.getRaw(dimension.getOutputName())); + // Add timestamp. + if (query.getResultRowHasTimestamp()) { + newResult.set(0, adjustTimestamp(arg1)); } - // Add aggregations - for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { - newMap.put( - aggregatorFactory.getName(), - aggregatorFactory.combine( - arg1.getRaw(aggregatorFactory.getName()), - arg2.getRaw(aggregatorFactory.getName()) - ) - ); + // Add dimensions. + final int dimensionStart = query.getResultRowDimensionStart(); + final List dimensions = query.getDimensions(); + for (int i = 0; i < dimensions.size(); i++) { + final int rowIndex = dimensionStart + i; + newResult.set(rowIndex, arg1.get(rowIndex)); } - return new MapBasedRow(adjustTimestamp(arg1), newMap); + // Add aggregations. + final int aggregatorStart = query.getResultRowAggregatorStart(); + final List 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 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 cccb3e57dae..aa162db24f6 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 +/** + * 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 { 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> queryables; + private final Iterable> queryables; private final ListeningExecutorService exec; private final QueryWatcher queryWatcher; private final int concurrencyHint; @@ -88,7 +100,7 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner GroupByQueryConfig config, ExecutorService exec, QueryWatcher queryWatcher, - Iterable> queryables, + Iterable> queryables, int concurrencyHint, BlockingPool mergeBufferPool, int mergeBufferSize, @@ -108,7 +120,7 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner } @Override - public Sequence run(final QueryPlus queryPlus, final ResponseContext responseContext) + public Sequence run(final QueryPlus 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 CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION, false ); - final QueryPlus queryPlusForRunners = queryPlus + final QueryPlus queryPlusForRunners = queryPlus .withQuery( query.withOverriddenContext(ImmutableMap.of(CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION, true)) ) .withoutThreadUnsafeState(); if (QueryContexts.isBySegment(query) || forceChainedExecution) { - ChainedExecutionQueryRunner runner = new ChainedExecutionQueryRunner<>(exec, queryWatcher, queryables); + ChainedExecutionQueryRunner 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 final long timeoutAt = System.currentTimeMillis() + queryTimeout; return new BaseSequence<>( - new BaseSequence.IteratorMaker>() + new BaseSequence.IteratorMaker>() { @Override - public CloseableGrouperIterator make() + public CloseableGrouperIterator make() { final List resources = new ArrayList<>(); @@ -185,10 +192,9 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner mergeBufferHolders.get(1) : null; - Pair, Accumulator> pair = + Pair, Accumulator> pair = RowBasedGrouperHelper.createGrouperAccumulatorPair( query, - false, null, config, Suppliers.ofInstance(mergeBufferHolder.get()), @@ -196,16 +202,14 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner concurrencyHint, temporaryStorage, spillMapper, - combiningAggregatorFactories, exec, priority, hasTimeout, timeoutAt, - mergeBufferSize, - false + mergeBufferSize ); final Grouper grouper = pair.lhs; - final Accumulator accumulator = pair.rhs; + final Accumulator accumulator = pair.rhs; grouper.init(); final ReferenceCountingResourceHolder> grouperHolder = @@ -216,15 +220,13 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner Lists.newArrayList( Iterables.transform( queryables, - new Function, ListenableFuture>() + new Function, ListenableFuture>() { @Override - public ListenableFuture apply(final QueryRunner input) + public ListenableFuture apply(final QueryRunner 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 future = exec.submit( @@ -240,14 +242,9 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner @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 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 iterFromMake) + public void cleanup(CloseableGrouperIterator 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 836f36fc76a..c26e515d0fe 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[] baseSelectorPlus) + private static GroupByColumnSelectorPlus[] createGroupBySelectorPlus( + ColumnSelectorPlus[] 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 process( + public static Sequence process( final GroupByQuery query, - final StorageAdapter storageAdapter, + @Nullable final StorageAdapter storageAdapter, final NonBlockingPool intermediateResultsBufferPool, final GroupByQueryConfig querySpecificConfig ) @@ -130,7 +140,7 @@ public class GroupByQueryEngineV2 VectorGroupByEngine.canVectorize(query, storageAdapter, filter) ); - final Sequence result; + final Sequence result; if (doVectorize) { result = VectorGroupByEngine.process( @@ -157,7 +167,7 @@ public class GroupByQueryEngineV2 return result.withBaggage(bufferHolder); } - private static Sequence processNonVectorized( + private static Sequence 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>() + new BaseSequence.IteratorMaker>() { @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 implements Iterator, Closeable + private abstract static class GroupByEngineIterator implements Iterator, Closeable { protected final GroupByQuery query; protected final GroupByQueryConfig querySpecificConfig; @@ -364,7 +376,7 @@ public class GroupByQueryEngineV2 protected final DateTime timestamp; @Nullable - protected CloseableGrouperIterator delegate = null; + protected CloseableGrouperIterator delegate = null; protected final boolean allSingleValueDims; public GroupByEngineIterator( @@ -389,7 +401,7 @@ public class GroupByQueryEngineV2 this.allSingleValueDims = allSingleValueDims; } - private CloseableGrouperIterator initNewDelegate() + private CloseableGrouperIterator initNewDelegate() { final Grouper 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 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 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 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 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 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 dimensionSpecs, Map rowMap) + public static void convertRowTypesToOutputTypes( + final List 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 2fa3b57bf5a..c1ac96622fc 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 rows, - final Map 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 results(@Nullable List 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 rows, final GroupByQueryConfig config, final GroupByQueryResource resource, final ObjectMapper spillMapper, final String processingTmpDir, - final int mergeBufferSize, - final List closeOnExit, - final boolean wasQueryPushedDown, - final boolean useVirtualizedColumnSelectorFactory + final int mergeBufferSize ) { - final GroupByQuery query = (GroupByQuery) queryParam; + final List 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 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, Accumulator> pair = RowBasedGrouperHelper.createGrouperAccumulatorPair( + Pair, Accumulator> pair = RowBasedGrouperHelper.createGrouperAccumulatorPair( query, - true, - rowSignature, + subquery, querySpecificConfig, new Supplier() { @@ -112,82 +125,56 @@ public class GroupByRowProcessor }, temporaryStorage, spillMapper, - aggregatorFactories, - mergeBufferSize, - useVirtualizedColumnSelectorFactory + mergeBufferSize ); final Grouper grouper = pair.lhs; - final Accumulator accumulator = pair.rhs; + final Accumulator 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 results(@Nullable List dimensionsToInclude) + { + return getRowsFromGrouper(query, grouper, dimensionsToInclude); + } + + @Override + public void close() + { + Lists.reverse(closeOnExit).forEach(CloseQuietly::close); + } + }; } - private static Sequence getFilteredSequence( - Sequence rows, - Map rowSignature, - GroupByQuery query + private static Sequence getRowsFromGrouper( + final GroupByQuery query, + final Grouper grouper, + @Nullable List dimensionsToInclude ) - { - final List queryIntervals = query.getIntervals(); - final Filter filter = Filters.convertToCNFFromQueryContext( - query, - Filters.toFilter(query.getDimFilter()) - ); - - final SettableSupplier 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 getRowsFromGrouper(GroupByQuery query, List subtotalSpec, Supplier grouper) { return new BaseSequence<>( - new BaseSequence.IteratorMaker>() + new BaseSequence.IteratorMaker>() { @Override - public CloseableGrouperIterator make() + public CloseableGrouperIterator make() { return RowBasedGrouperHelper.makeGrouperIterator( - grouper.get(), + grouper, query, - subtotalSpec, + dimensionsToInclude, () -> {} ); } @Override - public void cleanup(CloseableGrouperIterator iterFromMake) + public void cleanup(CloseableGrouperIterator 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 2c75c354e31..9315b779aa5 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, Accumulator> createGrouperAccumulatorPair( + public static Pair, Accumulator> createGrouperAccumulatorPair( final GroupByQuery query, - final boolean isInputRaw, - final Map rawInputRowSignature, + @Nullable final GroupByQuery subquery, final GroupByQueryConfig config, final Supplier 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, Accumulator> createGrouperAccumulatorPair( + public static Pair, Accumulator> createGrouperAccumulatorPair( final GroupByQuery query, - final boolean isInputRaw, - final Map rawInputRowSignature, + @Nullable final GroupByQuery subquery, final GroupByQueryConfig config, final Supplier bufferSupplier, @Nullable final ReferenceCountingResourceHolder 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 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 columnSelectorRow = new ThreadLocal<>(); + final ThreadLocal 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 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 accumulator = (priorResult, row) -> { + final Predicate rowPredicate; + + if (combining) { + // Filters are not applied in combining mode. + rowPredicate = row -> true; + } else { + rowPredicate = getResultRowPredicate(query, subquery); + } + + final Accumulator 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 supplier + ) + { + final RowBasedColumnSelectorFactory.RowAdapter adapter = + new RowBasedColumnSelectorFactory.RowAdapter() + { + @Override + public ToLongFunction timestampFunction() + { + if (query.getResultRowHasTimestamp()) { + return row -> row.getLong(0); + } else { + final long timestamp = query.getUniversalTimestamp().getMillis(); + return row -> timestamp; + } + } + + @Override + public Function 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 getResultRowPredicate(final GroupByQuery query, final GroupByQuery subquery) + { + final List queryIntervals = query.getIntervals(); + final Filter filter = Filters.convertToCNFFromQueryContext( + query, + Filters.toFilter(query.getDimFilter()) + ); + + final SettableSupplier 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 valueTypes ) { final TimestampExtractFunction timestampExtractFn = includeTimestamp ? - makeTimestampExtractFunction(query, isInputRaw) : + makeTimestampExtractFunction(query, combining) : null; final Function[] valueConvertFns = makeValueConvertFunctions(valueTypes); - if (isInputRaw) { + if (!combining) { final Supplier[] 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 makeGrouperIterator( + public static CloseableGrouperIterator makeGrouperIterator( final Grouper grouper, final GroupByQuery query, final Closeable closeable @@ -377,68 +526,61 @@ public class RowBasedGrouperHelper return makeGrouperIterator(grouper, query, null, closeable); } - public static CloseableGrouperIterator makeGrouperIterator( + public static CloseableGrouperIterator makeGrouperIterator( final Grouper grouper, final GroupByQuery query, - final List dimsToInclude, + @Nullable final List 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 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(), + 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 ); } - } else { - Map 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(), - 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 6cb96346ac0..e1b664196c3 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 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 e8961e4f113..6b3e577e218 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 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 52cd4b2aff2..453c282b841 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 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 a272373e073..4eb0e9588a8 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 @@ -23,12 +23,21 @@ import org.apache.druid.query.ColumnSelectorPlus; public class GroupByColumnSelectorPlus extends ColumnSelectorPlus { + /** + * 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 baseInfo, int keyBufferPosition) + public GroupByColumnSelectorPlus( + ColumnSelectorPlus baseInfo, + int keyBufferPosition, + int resultRowPosition + ) { super( baseInfo.getName(), @@ -37,10 +46,16 @@ public class GroupByColumnSelectorPlus extends ColumnSelectorPlus 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 5afc7aad78a..3c5f8fefe9b 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 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 f92ec9497f6..f0cfaf1ff28 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 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 e7a1f8e903b..65c48419a19 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 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 2802e3a8aec..69f6c868b3f 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 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 5adbdb1f14f..9b9d53a8495 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 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 3cc415368eb..087c778b18e 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 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 6ddbd99b4e8..afacd224d65 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 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 6a9b4289821..f63f9cc6895 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 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 96b9988e5ec..6d76e0c56f0 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 process( + public static Sequence process( final GroupByQuery query, final StorageAdapter storageAdapter, final ByteBuffer processingBuffer, @@ -107,10 +104,10 @@ public class VectorGroupByEngine } return new BaseSequence<>( - new BaseSequence.IteratorMaker>() + new BaseSequence.IteratorMaker>() { @Override - public CloseableIterator make() + public CloseableIterator 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() + return new CloseableIterator() { @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 iterFromMake) + public void cleanup(CloseableIterator iterFromMake) { try { iterFromMake.close(); @@ -192,7 +189,7 @@ public class VectorGroupByEngine ); } - private static class VectorGroupByEngineIterator implements CloseableIterator + private static class VectorGroupByEngineIterator implements CloseableIterator { private final GroupByQuery query; private final GroupByQueryConfig querySpecificConfig; @@ -218,7 +215,7 @@ public class VectorGroupByEngine private int partiallyAggregatedRows = -1; @Nullable - private CloseableGrouperIterator delegate = null; + private CloseableGrouperIterator 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 initNewDelegate() + private CloseableGrouperIterator 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 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 8450589f814..350d6ef14e1 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 f035db3dbde..9c11ab9b1de 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 havingSpecs; @@ -50,23 +48,15 @@ public class AndHavingSpec extends BaseHavingSpec } @Override - public void setRowSignature(Map rowSignature) + public void setQuery(GroupByQuery query) { for (HavingSpec havingSpec : havingSpecs) { - havingSpec.setRowSignature(rowSignature); + havingSpec.setQuery(query); } } @Override - public void setAggregators(Map 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 cf916bf0255..9039b21a315 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 rowSupplier = new SettableSupplier<>(); - private Map rowSignature = new HashMap<>(); - private Map aggregators = new HashMap<>(); - private Transformer transformer = null; + private Int2ObjectMap> 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 rowSignature) + public void setQuery(GroupByQuery query) { - this.rowSignature = rowSignature; + this.finalizers = new Int2ObjectArrayMap<>(query.getAggregatorSpecs().size()); + + 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 void setAggregators(final Map aggregators) - { - this.aggregators = aggregators; - } - - @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> 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 rowSignature, - final Map aggregators, - final boolean finalize - ) - { - final List 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 getDimensions() - { - return Collections.emptyList(); - } - - @Override - public long getTimestampFromEpoch() - { - return row.getTimestampFromEpoch(); - } - - @Override - public DateTime getTimestamp() - { - return row.getTimestamp(); - } - - @Override - public List 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 4dfc6e63c60..80a9669994b 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) { - List dimRowValList = row.getDimension(dimension); + columnNumber = query.getResultRowPositionLookup().getInt(dimension); + } + + @Override + public boolean eval(ResultRow row) + { + if (columnNumber < 0) { + return Strings.isNullOrEmpty(value); + } + + List 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 00c471b200b..0bd0d452334 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 aggregators; + private volatile int columnNumber; @JsonCreator public EqualToHavingSpec( @@ -62,15 +64,20 @@ public class EqualToHavingSpec extends BaseHavingSpec } @Override - public void setAggregators(Map 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 aa276a9466c..be141e9dfa0 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 ">" operator in a "having" clause. This is similar to SQL's "having aggregation > 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 aggregators; + private volatile int columnNumber; @JsonCreator public GreaterThanHavingSpec( @@ -62,15 +64,20 @@ public class GreaterThanHavingSpec extends BaseHavingSpec } @Override - public void setAggregators(Map 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 e75641f5419..c179eb68e60 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 rowSignature); - - void setAggregators(Map 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 e1227e7f043..57a4a6d690a 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 computeAggregatorsMap(List aggregatorSpecs) + { + Map 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 3c937cfba9f..8bc3f97bbc3 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 "<" operator in a "having" clause. This is similar to SQL's "having aggregation < 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 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 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 fa2c15d4bc4..a05ebcab4e6 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 81d7a63eaf8..cd8f4c646e0 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 rowSignature) + public void setQuery(GroupByQuery query) { - havingSpec.setRowSignature(rowSignature); + havingSpec.setQuery(query); } @Override - public void setAggregators(Map aggregators) - { - havingSpec.setAggregators(aggregators); - } - - @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 e6483490d7e..097619e5fc2 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 havingSpecs; @@ -50,23 +48,15 @@ public class OrHavingSpec extends BaseHavingSpec } @Override - public void setRowSignature(Map rowSignature) + public void setQuery(GroupByQuery query) { for (HavingSpec havingSpec : havingSpecs) { - havingSpec.setRowSignature(rowSignature); + havingSpec.setQuery(query); } } @Override - public void setAggregators(Map 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 7a4c5f3a66a..49c49edb9fd 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 dimensions) @@ -119,23 +124,18 @@ public class DefaultLimitSpec implements LimitSpec } @Override - public Function, Sequence> build( - List dimensions, - List aggs, - List postAggs, - Granularity granularity, - boolean sortByDimsFirst - ) + public Function, Sequence> build(final GroupByQuery query) { - // Can avoid re-sorting if the natural ordering is good enough. + final List dimensions = query.getDimensions(); + // Can avoid re-sorting if the natural ordering is good enough. boolean sortingNeeded = dimensions.size() < columns.size(); final Set 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 ordering = makeComparator(dimensions, aggs, postAggs, sortByDimsFirst); + final Ordering 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 makeComparator( + private Ordering makeComparator( + Object2IntMap rowOrderLookup, + boolean hasTimestamp, List dimensions, List aggs, List postAggs, boolean sortByDimsFirst ) { - Ordering timeOrdering = new Ordering() - { - @Override - public int compare(Row left, Row right) + final Ordering timeOrdering; + + if (hasTimestamp) { + timeOrdering = new Ordering() { - 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 dimensionsMap = new HashMap<>(); for (DimensionSpec spec : dimensions) { @@ -235,17 +250,23 @@ public class DefaultLimitSpec implements LimitSpec postAggregatorsMap.put(postAgg.getName(), postAgg); } - Ordering ordering = null; + Ordering ordering = null; for (OrderByColumnSpec columnSpec : columns) { String columnName = columnSpec.getDimension(); - Ordering nextOrdering = null; + Ordering 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()); + 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 metricOrdering(final String column, final Comparator comparator) + private Ordering metricOrdering(final int column, final Comparator 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 nullFriendlyComparator = NullHandling.sqlCompatible() + ? Comparator.nullsFirst(comparator) + : Comparator.nullsLast(comparator); + + //noinspection unchecked + return Ordering.from(Comparator.comparing(row -> (T) row.get(column), nullFriendlyComparator)); } - private Ordering dimensionOrdering(final String dimension, final StringComparator comparator) + private Ordering 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 values = row.getDimension(column); - return values.isEmpty() ? null : values.get(0); + final List 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> + private static class LimitingFn implements Function, Sequence> { - private int limit; + private final int limit; public LimitingFn(int limit) { @@ -311,41 +335,41 @@ public class DefaultLimitSpec implements LimitSpec } @Override - public Sequence apply(Sequence input) + public Sequence apply(Sequence input) { return input.limit(limit); } } - private static class SortingFn implements Function, Sequence> + private static class SortingFn implements Function, Sequence> { - private final Ordering ordering; + private final Ordering ordering; - public SortingFn(Ordering ordering) + public SortingFn(Ordering ordering) { this.ordering = ordering; } @Override - public Sequence apply(@Nullable Sequence input) + public Sequence apply(@Nullable Sequence input) { return Sequences.sort(input, ordering); } } - private static class TopNFunction implements Function, Sequence> + private static class TopNFunction implements Function, Sequence> { - private final Ordering ordering; + private final Ordering ordering; private final int limit; - public TopNFunction(Ordering ordering, int limit) + public TopNFunction(Ordering ordering, int limit) { this.ordering = ordering; this.limit = limit; } @Override - public Sequence apply(final Sequence input) + public Sequence apply(final Sequence 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 d7243359e3a..8f0c9b5b09f 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> build( - List dimensions, - List aggs, - List postAggs, - Granularity granularity, - boolean sortByDimsFirst - ); + Function, Sequence> 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 fcfc2919ee8..9bec0cf195d 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> build( - List dimensions, - List aggs, - List postAggs, - Granularity granularity, - boolean sortByDimsFirst - ) + public Function, Sequence> 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 b8dace90440..6125dd0897b 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 createIntervalChunkingRunner( + QueryRunner createIntervalChunkingRunner( IntervalChunkingQueryRunnerDecorator decorator, - QueryRunner runner, + QueryRunner runner, GroupByQueryQueryToolChest toolChest ); - Sequence mergeResults(QueryRunner 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 mergeResults( + QueryRunner 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 createMergeFn(Query query) + default BinaryOperator createMergeFn(Query 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 createResultComparator(Query queryParam) + default Comparator createResultComparator(Query queryParam) { throw new UOE("%s doesn't provide a result comparator", this.getClass().getName()); } - Sequence applyPostProcessing(Sequence 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 applyPostProcessing(Sequence results, GroupByQuery query); - Sequence 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 processSubqueryResult( GroupByQuery subquery, GroupByQuery query, GroupByQueryResource resource, - Sequence subqueryResult, + Sequence subqueryResult, boolean wasQueryPushedDown ); - Sequence processSubtotalsSpec(GroupByQuery query, GroupByQueryResource resource, Sequence 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 processSubtotalsSpec( + GroupByQuery query, + GroupByQueryResource resource, + Sequence queryResult + ); - QueryRunner mergeRunners(ListeningExecutorService exec, Iterable> 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 mergeRunners(ListeningExecutorService exec, Iterable> queryRunners); - Sequence 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 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 69f8790fbf8..be9ba0a1394 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 createIntervalChunkingRunner( + public QueryRunner createIntervalChunkingRunner( final IntervalChunkingQueryRunnerDecorator decorator, - final QueryRunner runner, + final QueryRunner runner, final GroupByQueryQueryToolChest toolChest ) { @@ -109,14 +109,15 @@ public class GroupByStrategyV1 implements GroupByStrategy } @Override - public Sequence mergeResults( - final QueryRunner baseRunner, + public Sequence mergeResults( + final QueryRunner 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.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 applyPostProcessing(Sequence results, GroupByQuery query) + public Sequence applyPostProcessing(Sequence results, GroupByQuery query) { return query.postProcess(results); } @Override - public Sequence processSubqueryResult( + public Sequence processSubqueryResult( GroupByQuery subquery, GroupByQuery query, GroupByQueryResource resource, - Sequence subqueryResult, + Sequence 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>() + new Function>() { @Override - public Sequence apply(Interval interval) + public Sequence 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 processSubtotalsSpec( + public Sequence processSubtotalsSpec( GroupByQuery query, GroupByQueryResource resource, - Sequence queryResult + Sequence queryResult ) { throw new UnsupportedOperationException("subtotalsSpec is not supported for v1 groupBy strategy."); } @Override - public QueryRunner mergeRunners( + public QueryRunner mergeRunners( final ListeningExecutorService exec, - final Iterable> queryRunners + final Iterable> queryRunners ) { return new GroupByMergedQueryRunner<>(exec, configSupplier, queryWatcher, bufferPool, queryRunners); } @Override - public Sequence process(final GroupByQuery query, final StorageAdapter storageAdapter) + public Sequence 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 72801a07c28..4c417b0b5a8 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); + 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> 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); - } - } - } else { + 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> 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); + } } } @@ -200,9 +163,9 @@ public class GroupByStrategyV2 implements GroupByStrategy } @Override - public QueryRunner createIntervalChunkingRunner( + public QueryRunner createIntervalChunkingRunner( final IntervalChunkingQueryRunnerDecorator decorator, - final QueryRunner runner, + final QueryRunner runner, final GroupByQueryQueryToolChest toolChest ) { @@ -215,44 +178,47 @@ public class GroupByStrategyV2 implements GroupByStrategy } @Override - public Comparator createResultComparator(Query queryParam) + public Comparator createResultComparator(Query queryParam) { return ((GroupByQuery) queryParam).getRowOrdering(true); } @Override - public BinaryOperator createMergeFn(Query queryParam) + public BinaryOperator createMergeFn(Query queryParam) { return new GroupByBinaryFnV2((GroupByQuery) queryParam); } @Override - public Sequence mergeResults( - final QueryRunner baseRunner, + public Sequence mergeResults( + final QueryRunner baseRunner, final GroupByQuery query, final ResponseContext responseContext ) { // Merge streams using ResultMergeQueryRunner, then apply postaggregators, then apply limit (which may // involve materialization) - final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner<>( + final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner<>( baseRunner, this::createResultComparator, this::createMergeFn ); - // Fudge timestamp, maybe. - final DateTime fudgeTimestamp = getUniversalTimestamp(query); - ImmutableMap.Builder context = ImmutableMap.builder(); + // Set up downstream context. + final ImmutableMap.Builder 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() - { - @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 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()); + + // Copy everything that comes before the postaggregations. + for (int i = 0; i < query.getResultRowPostAggregatorStart(); i++) { + rowWithPostAggregations.set(i, row.get(i)); } - if (query.getPostAggregatorSpecs().isEmpty() && fudgeTimestamp == null) { - return row; + // 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 mapForPostAggregationComputation = rowWithPostAggregations.toMap(query); + + for (int i = 0; i < query.getPostAggregatorSpecs().size(); i++) { + final PostAggregator postAggregator = query.getPostAggregatorSpecs().get(i); + final Object value = postAggregator.compute(mapForPostAggregationComputation); + + rowWithPostAggregations.set(query.getResultRowPostAggregatorStart() + i, value); + mapForPostAggregationComputation.put(postAggregator.getName(), value); } - final Map newMap; - - if (query.getPostAggregatorSpecs().isEmpty()) { - newMap = ((MapBasedRow) row).getEvent(); - } else { - newMap = Maps.newLinkedHashMap(((MapBasedRow) row).getEvent()); - - for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { - newMap.put(postAggregator.getName(), postAggregator.compute(newMap)); - } - } - - return new MapBasedRow(fudgeTimestamp != null ? fudgeTimestamp : row.getTimestamp(), newMap); + return rowWithPostAggregations; } - } - ); + ); + } } @Override - public Sequence applyPostProcessing(Sequence results, GroupByQuery query) + public Sequence applyPostProcessing(Sequence 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 processSubqueryResult( + public Sequence processSubqueryResult( GroupByQuery subquery, GroupByQuery query, GroupByQueryResource resource, - Sequence subqueryResult, + Sequence 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 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 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() - { - @Override - public Sequence run(QueryPlus 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 processSubtotalsSpec( + public Sequence processSubtotalsSpec( GroupByQuery query, GroupByQueryResource resource, - Sequence queryResult + Sequence 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 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> subtotals = query.getSubtotalsSpec(); - Supplier 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> subtotalsResults = new ArrayList<>(subtotals.size()); - - Map queryDimensionSpecs = new HashMap(queryWithoutSubtotalsSpec.getDimensions().size()); - for (DimensionSpec dimSpec : queryWithoutSubtotalsSpec.getDimensions()) { - queryDimensionSpecs.put(dimSpec.getOutputName(), dimSpec); - } + List> subtotalsResults = new ArrayList<>(subtotals.size()); for (List subtotalSpec : subtotals) { - GroupByQuery subtotalQuery = queryWithoutSubtotalsSpec.withDimensionSpecs( - subtotalSpec.stream() - .map(s -> new DefaultDimensionSpec(s, s, queryDimensionSpecs.get(s).getOutputType())) - .collect(Collectors.toList()) - ); + final ImmutableSet dimsInSubtotalSpec = ImmutableSet.copyOf(subtotalSpec); + final List dimensions = query.getDimensions(); + final List newDimensions = new ArrayList<>(); - subtotalsResults.add(applyPostProcessing( - mergeResults(new QueryRunner() - { - @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) - { - return GroupByRowProcessor.getRowsFromGrouper( - queryWithoutSubtotalsSpec, - subtotalSpec, - grouperSupplier - ); - } - }, subtotalQuery, null), - subtotalQuery - ) + 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)); + } + } + + 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 mergeRunners(ListeningExecutorService exec, Iterable> queryRunners) + public QueryRunner mergeRunners( + final ListeningExecutorService exec, + final Iterable> queryRunners + ) { return new GroupByMergingQueryRunnerV2( configSupplier.get(), @@ -483,7 +457,7 @@ public class GroupByStrategyV2 implements GroupByStrategy } @Override - public Sequence process(GroupByQuery query, StorageAdapter storageAdapter) + public Sequence 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 416fb037d15..88428c38e16 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, TimeseriesQuery> { @@ -333,7 +334,6 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest results = (List) input; final Map retVal = Maps.newLinkedHashMap(); - Iterator aggsIter = aggs.iterator(); Iterator resultIter = results.iterator(); final Number timestampNumber = (Number) resultIter.next(); @@ -345,12 +345,11 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest { + (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 06aa42700fc..3f0b5c7f7bf 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, TopNQuery> { @@ -388,7 +389,6 @@ public class TopNQueryQueryToolChest extends QueryToolChest result = (List) inputIter.next(); final Map vals = Maps.newLinkedHashMap(); - Iterator aggIter = aggs.iterator(); Iterator resultIter = result.iterator(); // Must convert generic Jackson-deserialized type into the proper type. @@ -398,12 +398,11 @@ public class TopNQueryQueryToolChest extends QueryToolChest { + (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 8e21296b90d..dc09a5633f8 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 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 048299e6e26..159187788f6 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 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 a80eae85f1f..cb1a8b68569 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 2f75700a864..56892584c3c 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 result = helper.runQueryOnSegmentsObjs( + Sequence 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 expectedResults = Arrays.asList( + List 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 result = helper.runQueryOnSegmentsObjs( + Sequence 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 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 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 result = helper.runQueryOnSegmentsObjs( + Sequence 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 expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", null, "count", 2L) + List 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 result = helper.runQueryOnSegmentsObjs( + Sequence 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 expectedResults; + List 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 result = helper.runQueryOnSegmentsObjs( + Sequence 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 expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t3", "count", 4L) + List 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 result = helper.runQueryOnSegmentsObjs( + Sequence 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 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 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 result = helper.runQueryOnSegmentsObjs( + Sequence 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 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 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 result = helper.runQueryOnSegmentsObjs( + Sequence 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 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 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 result = helper.runQueryOnSegmentsObjs( + Sequence 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 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 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 result = helper.runQueryOnSegmentsObjs( + Sequence 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 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 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 result = helper.runQueryOnSegmentsObjs( + Sequence 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 ); - List 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 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"); @@ -689,7 +690,7 @@ public class MultiValuedDimensionTest .setContext(context) .build(); - Sequence result = helper.runQueryOnSegmentsObjs( + Sequence result = helper.runQueryOnSegmentsObjs( ImmutableList.of( new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")), new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2")) @@ -697,15 +698,15 @@ public class MultiValuedDimensionTest query ); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foo", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "t1foo", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "t2foo", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "t3foo", "count", 4L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "t4foo", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "t5foo", "count", 4L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "t6foo", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "t7foo", "count", 2L) + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foo", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "t1foo", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "t2foo", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "t3foo", "count", 4L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "t4foo", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "t5foo", "count", 4L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "t6foo", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "t7foo", "count", 2L) ); TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-auto"); @@ -736,7 +737,7 @@ public class MultiValuedDimensionTest .setContext(context) .build(); - Sequence result = helper.runQueryOnSegmentsObjs( + Sequence result = helper.runQueryOnSegmentsObjs( ImmutableList.of( new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")), new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2")) @@ -744,9 +745,9 @@ public class MultiValuedDimensionTest query ); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", NullHandling.replaceWithDefault() ? -1L : null, "count", 6L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", 1L, "count", 2L) + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", NullHandling.replaceWithDefault() ? -1L : null, "count", 6L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", 1L, "count", 2L) ); TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-auto"); @@ -777,7 +778,7 @@ public class MultiValuedDimensionTest .setContext(context) .build(); - Sequence result = helper.runQueryOnSegmentsObjs( + Sequence result = helper.runQueryOnSegmentsObjs( ImmutableList.of( new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")), new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2")) @@ -785,11 +786,11 @@ public class MultiValuedDimensionTest query ); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foo", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foot1, foot2, foot3", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foot3, foot4, foot5", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foot5, foot6, foot7", "count", 2L) + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foo", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foot1, foot2, foot3", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foot3, foot4, foot5", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foot5, foot6, foot7", "count", 2L) ); TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-array-fn"); @@ -820,7 +821,7 @@ public class MultiValuedDimensionTest .setContext(context) .build(); - Sequence result = helper.runQueryOnSegmentsObjs( + Sequence result = helper.runQueryOnSegmentsObjs( ImmutableList.of( new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")), new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2")) @@ -828,11 +829,11 @@ public class MultiValuedDimensionTest query ); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foo", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foot1, foot2, foot3", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foot3, foot4, foot5", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foot5, foot6, foot7", "count", 2L) + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foo", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foot1, foot2, foot3", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foot3, foot4, foot5", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foot5, foot6, foot7", "count", 2L) ); TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-arrayfn-auto"); @@ -863,7 +864,7 @@ public class MultiValuedDimensionTest .setContext(context) .build(); - Sequence result = helper.runQueryOnSegmentsObjs( + Sequence result = helper.runQueryOnSegmentsObjs( ImmutableList.of( new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")), new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2")) @@ -872,17 +873,18 @@ public class MultiValuedDimensionTest ); - List expectedResults = Arrays.asList( + List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow( + query, "1970-01-01T00:00:00.000Z", "tt", NullHandling.replaceWithDefault() ? null : "", "count", 2L ), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "t1t2t3", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "t3t4t5", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "t5t6t7", "count", 2L) + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "t1t2t3", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "t3t4t5", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "t5t6t7", "count", 2L) ); TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-arrayfn-auto"); @@ -913,7 +915,7 @@ public class MultiValuedDimensionTest .setContext(context) .build(); - Sequence result = helper.runQueryOnSegmentsObjs( + Sequence result = helper.runQueryOnSegmentsObjs( ImmutableList.of( new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")), new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2")) @@ -921,11 +923,11 @@ public class MultiValuedDimensionTest query ); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foo", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foot1, foot2, foot3", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foot3, foot4, foot5", "count", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", "foot5, foot6, foot7", "count", 2L) + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foo", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foot1, foot2, foot3", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foot3, foot4, foot5", "count", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970", "tt", "foot5, foot6, foot7", "count", 2L) ); TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-arrayfn-auto"); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java index 8d134b36741..717876976e5 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java @@ -35,7 +35,6 @@ import org.apache.commons.io.IOUtils; import org.apache.commons.io.LineIterator; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.Row; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.java.util.common.IAE; @@ -59,6 +58,7 @@ import org.apache.druid.query.context.ResponseContext; 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.ResultRow; import org.apache.druid.query.select.SelectQueryConfig; import org.apache.druid.query.select.SelectQueryEngine; import org.apache.druid.query.select.SelectQueryQueryToolChest; @@ -325,7 +325,7 @@ public class AggregationTestHelper implements Closeable ); } - public Sequence createIndexAndRunQueryOnSegment( + public Sequence createIndexAndRunQueryOnSegment( File inputDataFile, String parserJson, String aggregators, @@ -340,7 +340,7 @@ public class AggregationTestHelper implements Closeable return runQueryOnSegments(Collections.singletonList(segmentDir), groupByQueryJson); } - public Sequence createIndexAndRunQueryOnSegment( + public Sequence createIndexAndRunQueryOnSegment( File inputDataFile, String parserJson, String aggregators, @@ -356,7 +356,7 @@ public class AggregationTestHelper implements Closeable return runQueryOnSegments(Collections.singletonList(segmentDir), groupByQueryJson); } - public Sequence createIndexAndRunQueryOnSegment( + public Sequence createIndexAndRunQueryOnSegment( InputStream inputDataStream, String parserJson, String aggregators, @@ -378,7 +378,7 @@ public class AggregationTestHelper implements Closeable ); } - public Sequence createIndexAndRunQueryOnSegment( + public Sequence createIndexAndRunQueryOnSegment( InputStream inputDataStream, String parserJson, String aggregators, @@ -561,14 +561,24 @@ public class AggregationTestHelper implements Closeable } } - //Simulates running group-by query on individual segments as historicals would do, json serialize the results - //from each segment, later deserialize and merge and finally return the results - public Sequence runQueryOnSegments(final List segmentDirs, final String queryJson) throws Exception + public Query readQuery(final String queryJson) { - return runQueryOnSegments(segmentDirs, mapper.readValue(queryJson, Query.class)); + try { + return mapper.readValue(queryJson, Query.class); + } + catch (IOException e) { + throw new RuntimeException(e); + } } - public Sequence runQueryOnSegments(final List segmentDirs, final Query query) + //Simulates running group-by query on individual segments as historicals would do, json serialize the results + //from each segment, later deserialize and merge and finally return the results + public Sequence runQueryOnSegments(final List segmentDirs, final String queryJson) + { + return runQueryOnSegments(segmentDirs, readQuery(queryJson)); + } + + public Sequence runQueryOnSegments(final List segmentDirs, final Query query) { final List segments = Lists.transform( segmentDirs, @@ -597,7 +607,7 @@ public class AggregationTestHelper implements Closeable } } - public Sequence runQueryOnSegmentsObjs(final List segments, final Query query) + public Sequence runQueryOnSegmentsObjs(final List segments, final Query query) { final FinalizeResultsQueryRunner baseRunner = new FinalizeResultsQueryRunner( toolChest.postMergeQueryDecoration( @@ -635,19 +645,19 @@ public class AggregationTestHelper implements Closeable return baseRunner.run(QueryPlus.wrap(query)); } - public QueryRunner makeStringSerdeQueryRunner( + public QueryRunner makeStringSerdeQueryRunner( final ObjectMapper mapper, final QueryToolChest toolChest, - final QueryRunner baseRunner + final QueryRunner baseRunner ) { - return new QueryRunner() + return new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext map) + public Sequence run(QueryPlus queryPlus, ResponseContext map) { try { - Sequence resultSeq = baseRunner.run(queryPlus); + Sequence resultSeq = baseRunner.run(queryPlus, ResponseContext.createEmpty()); final Yielder yielder = resultSeq.toYielder( null, new YieldingAccumulator() @@ -691,7 +701,7 @@ public class AggregationTestHelper implements Closeable ObjectCodec objectCodec = jp.getCodec(); while (jp.nextToken() != JsonToken.END_ARRAY) { - result.add(objectCodec.readValue(jp, toolChest.getResultTypeReference())); + result.add(objectCodec.readValue(jp, toolChest.getBaseResultType())); } return result; } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregationTest.java index e3b3fa8f885..d4699aa6031 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregationTest.java @@ -19,13 +19,13 @@ package org.apache.druid.query.aggregation.hyperloglog; -import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.jackson.AggregatorsModule; 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; @@ -110,7 +110,7 @@ public class HyperUniquesAggregationTest + "\"intervals\": [ \"1970/2050\" ]" + "}"; - Sequence seq = helper.createIndexAndRunQueryOnSegment( + Sequence seq = helper.createIndexAndRunQueryOnSegment( new File(this.getClass().getClassLoader().getResource("druid.sample.tsv").getFile()), parseSpec, metricSpec, @@ -120,9 +120,9 @@ public class HyperUniquesAggregationTest query ); - MapBasedRow row = (MapBasedRow) seq.toList().get(0); - Assert.assertEquals(3.0, row.getMetric("index_hll").floatValue(), 0.1); - Assert.assertEquals(3.0, row.getMetric("index_unique_count").floatValue(), 0.1); + final ResultRow resultRow = seq.toList().get(0); + Assert.assertEquals("index_hll", 3.0, ((Number) resultRow.get(0)).floatValue(), 0.1); + Assert.assertEquals("index_unique_count", 3.0, ((Number) resultRow.get(1)).floatValue(), 0.1); } } @@ -175,7 +175,7 @@ public class HyperUniquesAggregationTest + "\"intervals\": [ \"1970/2050\" ]" + "}"; - Sequence seq = helper.createIndexAndRunQueryOnSegment( + Sequence seq = helper.createIndexAndRunQueryOnSegment( new File(this.getClass().getClassLoader().getResource("druid.hll.sample.tsv").getFile()), parseSpec, metricSpec, @@ -185,9 +185,9 @@ public class HyperUniquesAggregationTest query ); - MapBasedRow row = (MapBasedRow) seq.toList().get(0); - Assert.assertEquals(4.0, row.getMetric("index_hll").floatValue(), 0.1); - Assert.assertEquals(4.0, row.getMetric("index_unique_count").floatValue(), 0.1); + final ResultRow resultRow = seq.toList().get(0); + Assert.assertEquals("index_hll", 4.0, ((Number) resultRow.get(0)).floatValue(), 0.1); + Assert.assertEquals("index_unique_count", 4.0, ((Number) resultRow.get(1)).floatValue(), 0.1); } } } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/post/FinalizingFieldAccessPostAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/post/FinalizingFieldAccessPostAggregatorTest.java index 6f22c7246cc..496028f8a25 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/post/FinalizingFieldAccessPostAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/post/FinalizingFieldAccessPostAggregatorTest.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.jackson.AggregatorsModule; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Comparators; @@ -35,6 +34,7 @@ import org.apache.druid.query.aggregation.CountAggregator; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.segment.TestHelper; import org.easymock.EasyMock; import org.junit.Assert; @@ -235,7 +235,7 @@ public class FinalizingFieldAccessPostAggregatorTest + "\"intervals\": [ \"1970/2050\" ]" + "}"; - Sequence seq = helper.createIndexAndRunQueryOnSegment( + Sequence seq = helper.createIndexAndRunQueryOnSegment( new File(this.getClass().getClassLoader().getResource("druid.sample.tsv").getFile()), parseSpec, metricSpec, @@ -245,10 +245,10 @@ public class FinalizingFieldAccessPostAggregatorTest query ); - MapBasedRow row = (MapBasedRow) seq.toList().get(0); - Assert.assertEquals(3.0, row.getMetric("hll_market").floatValue(), 0.1); - Assert.assertEquals(9.0, row.getMetric("hll_quality").floatValue(), 0.1); - Assert.assertEquals(12.0, row.getMetric("uniq_add").floatValue(), 0.1); + final ResultRow resultRow = seq.toList().get(0); + Assert.assertEquals("hll_market", 3.0, ((Number) resultRow.get(0)).floatValue(), 0.1); + Assert.assertEquals("hll_quality", 9.0, ((Number) resultRow.get(1)).floatValue(), 0.1); + Assert.assertEquals("uniq_add", 12.0, ((Number) resultRow.get(2)).floatValue(), 0.1); } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java index c776cecc710..35a59459bff 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java @@ -33,7 +33,6 @@ import org.apache.druid.collections.CloseableDefaultBlockingPool; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.Row; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; @@ -103,8 +102,8 @@ public class GroupByLimitPushDownInsufficientBufferTest private static final IndexIO INDEX_IO; private File tmpDir; - private QueryRunnerFactory groupByFactory; - private QueryRunnerFactory tooSmallGroupByFactory; + private QueryRunnerFactory groupByFactory; + private QueryRunnerFactory tooSmallGroupByFactory; private List incrementalIndices = new ArrayList<>(); private List groupByIndices = new ArrayList<>(); private ExecutorService executorService; @@ -431,27 +430,27 @@ public class GroupByLimitPushDownInsufficientBufferTest { // one segment's results use limit push down, the other doesn't because of insufficient buffer capacity - QueryToolChest toolChest = groupByFactory.getToolchest(); - QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + QueryToolChest toolChest = groupByFactory.getToolchest(); + QueryRunner theRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( groupByFactory.mergeRunners(executorService, getRunner1()) ), (QueryToolChest) toolChest ); - QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( + QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( tooSmallGroupByFactory.mergeRunners(executorService, getRunner2()) ), (QueryToolChest) toolChest ); - QueryRunner theRunner3 = new FinalizeResultsQueryRunner<>( + QueryRunner theRunner3 = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { return Sequences .simple( @@ -486,20 +485,23 @@ public class GroupByLimitPushDownInsufficientBufferTest .setGranularity(Granularities.ALL) .build(); - Sequence queryResult = theRunner3.run(QueryPlus.wrap(query)); - List results = queryResult.toList(); + Sequence queryResult = theRunner3.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + List results = queryResult.toList(); - Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "1970-01-01T00:00:00.000Z", "dimA", "zortaxx", "metA", 999L ); - Row expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "1970-01-01T00:00:00.000Z", "dimA", "zebra", "metA", 180L ); - Row expectedRow2 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow2 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "1970-01-01T00:00:00.000Z", "dimA", "world", "metA", 150L @@ -516,8 +518,8 @@ public class GroupByLimitPushDownInsufficientBufferTest { // one segment's results use limit push down, the other doesn't because of insufficient buffer capacity - QueryToolChest toolChest = groupByFactory.getToolchest(); - QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + QueryToolChest toolChest = groupByFactory.getToolchest(); + QueryRunner theRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( groupByFactory.mergeRunners(executorService, getRunner1()) ), @@ -525,19 +527,19 @@ public class GroupByLimitPushDownInsufficientBufferTest ); - QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( + QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( tooSmallGroupByFactory.mergeRunners(executorService, getRunner2()) ), (QueryToolChest) toolChest ); - QueryRunner theRunner3 = new FinalizeResultsQueryRunner<>( + QueryRunner theRunner3 = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { return Sequences .simple( @@ -574,26 +576,29 @@ public class GroupByLimitPushDownInsufficientBufferTest .setGranularity(Granularities.ALL) .setContext( ImmutableMap.of( - GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, - true + GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, + true ) ) .build(); - Sequence queryResult = theRunner3.run(QueryPlus.wrap(query)); - List results = queryResult.toList(); + Sequence queryResult = theRunner3.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + List results = queryResult.toList(); - Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "1970-01-01T00:00:00.000Z", "dimA", "zortaxx", "metA", 999L ); - Row expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "1970-01-01T00:00:00.000Z", "dimA", "foo", "metA", 200L ); - Row expectedRow2 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow2 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "1970-01-01T00:00:00.000Z", "dimA", "mango", "metA", 190L @@ -605,11 +610,11 @@ public class GroupByLimitPushDownInsufficientBufferTest Assert.assertEquals(expectedRow2, results.get(2)); } - private List> getRunner1() + private List> getRunner1() { - List> runners = new ArrayList<>(); + List> runners = new ArrayList<>(); QueryableIndex index = groupByIndices.get(0); - QueryRunner runner = makeQueryRunner( + QueryRunner runner = makeQueryRunner( groupByFactory, SegmentId.dummy(index.toString()), new QueryableIndexSegment(index, SegmentId.dummy(index.toString())) @@ -618,11 +623,11 @@ public class GroupByLimitPushDownInsufficientBufferTest return runners; } - private List> getRunner2() + private List> getRunner2() { - List> runners = new ArrayList<>(); + List> runners = new ArrayList<>(); QueryableIndex index2 = groupByIndices.get(1); - QueryRunner tooSmallRunner = makeQueryRunner( + QueryRunner tooSmallRunner = makeQueryRunner( tooSmallGroupByFactory, SegmentId.dummy(index2.toString()), new QueryableIndexSegment(index2, SegmentId.dummy(index2.toString())) diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java index 0f12908d370..92a21582a6e 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java @@ -33,7 +33,6 @@ import org.apache.druid.collections.CloseableDefaultBlockingPool; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.Row; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; @@ -113,8 +112,8 @@ public class GroupByLimitPushDownMultiNodeMergeTest private static final IndexIO INDEX_IO; private File tmpDir; - private QueryRunnerFactory groupByFactory; - private QueryRunnerFactory groupByFactory2; + private QueryRunnerFactory groupByFactory; + private QueryRunnerFactory groupByFactory2; private List incrementalIndices = new ArrayList<>(); private List groupByIndices = new ArrayList<>(); private ExecutorService executorService; @@ -461,27 +460,27 @@ public class GroupByLimitPushDownMultiNodeMergeTest @Test public void testDescendingNumerics() { - QueryToolChest toolChest = groupByFactory.getToolchest(); - QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + QueryToolChest toolChest = groupByFactory.getToolchest(); + QueryRunner theRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( groupByFactory.mergeRunners(executorService, getRunner1(2)) ), (QueryToolChest) toolChest ); - QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( + QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( groupByFactory2.mergeRunners(executorService, getRunner2(3)) ), (QueryToolChest) toolChest ); - QueryRunner finalRunner = new FinalizeResultsQueryRunner<>( + QueryRunner finalRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { return Sequences .simple( @@ -550,31 +549,35 @@ public class GroupByLimitPushDownMultiNodeMergeTest .setGranularity(Granularities.ALL) .build(); - Sequence queryResult = finalRunner.run(QueryPlus.wrap(query)); - List results = queryResult.toList(); + Sequence queryResult = finalRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + List results = queryResult.toList(); - Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "2017-07-14T02:40:00.000Z", "d0", 2027L, "d1", 3L, "d2", 17L, "a0", 2L ); - Row expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "2017-07-14T02:40:00.000Z", "d0", 2024L, "d1", 1L, "d2", 14L, "a0", 2L ); - Row expectedRow2 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow2 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "2017-07-14T02:40:00.000Z", "d0", 2020L, "d1", 11L, "d2", 13L, "a0", 2L ); - Row expectedRow3 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow3 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "2017-07-14T02:40:00.000Z", "d0", 2017L, "d1", 9L, @@ -594,27 +597,27 @@ public class GroupByLimitPushDownMultiNodeMergeTest { // one segment's results use limit push down, the other doesn't because of insufficient buffer capacity - QueryToolChest toolChest = groupByFactory.getToolchest(); - QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + QueryToolChest toolChest = groupByFactory.getToolchest(); + QueryRunner theRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( groupByFactory.mergeRunners(executorService, getRunner1(0)) ), (QueryToolChest) toolChest ); - QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( + QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( groupByFactory2.mergeRunners(executorService, getRunner2(1)) ), (QueryToolChest) toolChest ); - QueryRunner finalRunner = new FinalizeResultsQueryRunner<>( + QueryRunner finalRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { return Sequences .simple( @@ -671,28 +674,32 @@ public class GroupByLimitPushDownMultiNodeMergeTest .setGranularity(Granularities.ALL) .build(); - Sequence queryResult = finalRunner.run(QueryPlus.wrap(query)); - List results = queryResult.toList(); + Sequence queryResult = finalRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + List results = queryResult.toList(); - Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "2017-07-14T02:40:00.000Z", "dimA", "mango", "hour", 1505260800000L, "metASum", 26L ); - Row expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "2017-07-14T02:40:00.000Z", "dimA", "pomegranate", "hour", 1505260800000L, "metASum", 7113L ); - Row expectedRow2 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow2 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "2017-07-14T02:40:00.000Z", "dimA", "mango", "hour", 1505264400000L, "metASum", 10L ); - Row expectedRow3 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow3 = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "2017-07-14T02:40:00.000Z", "dimA", "pomegranate", "hour", 1505264400000L, @@ -706,11 +713,11 @@ public class GroupByLimitPushDownMultiNodeMergeTest Assert.assertEquals(expectedRow3, results.get(3)); } - private List> getRunner1(int qIndexNumber) + private List> getRunner1(int qIndexNumber) { - List> runners = new ArrayList<>(); + List> runners = new ArrayList<>(); QueryableIndex index = groupByIndices.get(qIndexNumber); - QueryRunner runner = makeQueryRunner( + QueryRunner runner = makeQueryRunner( groupByFactory, SegmentId.dummy(index.toString()), new QueryableIndexSegment(index, SegmentId.dummy(index.toString())) @@ -719,11 +726,11 @@ public class GroupByLimitPushDownMultiNodeMergeTest return runners; } - private List> getRunner2(int qIndexNumber) + private List> getRunner2(int qIndexNumber) { - List> runners = new ArrayList<>(); + List> runners = new ArrayList<>(); QueryableIndex index2 = groupByIndices.get(qIndexNumber); - QueryRunner tooSmallRunner = makeQueryRunner( + QueryRunner tooSmallRunner = makeQueryRunner( groupByFactory2, SegmentId.dummy(index2.toString()), new QueryableIndexSegment(index2, SegmentId.dummy(index2.toString())) diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java index 39167c50215..f16ff6ac351 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java @@ -31,7 +31,6 @@ import org.apache.druid.collections.CloseableDefaultBlockingPool; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.Row; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; @@ -99,7 +98,7 @@ public class GroupByMultiSegmentTest private static final IndexIO INDEX_IO; private File tmpDir; - private QueryRunnerFactory groupByFactory; + private QueryRunnerFactory groupByFactory; private List incrementalIndices = new ArrayList<>(); private List groupByIndices = new ArrayList<>(); private ExecutorService executorService; @@ -314,8 +313,8 @@ public class GroupByMultiSegmentTest @Test public void testHavingAndNoLimitPushDown() { - QueryToolChest toolChest = groupByFactory.getToolchest(); - QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + QueryToolChest toolChest = groupByFactory.getToolchest(); + QueryRunner theRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( groupByFactory.mergeRunners(executorService, makeGroupByMultiRunners()) ), @@ -343,10 +342,11 @@ public class GroupByMultiSegmentTest .setGranularity(Granularities.ALL) .build(); - Sequence queryResult = theRunner.run(QueryPlus.wrap(query)); - List results = queryResult.toList(); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + List results = queryResult.toList(); - Row expectedRow = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow = GroupByQueryRunnerTestHelper.createExpectedRow( + query, "1970-01-01T00:00:00.000Z", "dimA", "world", "metA", 150L @@ -356,12 +356,12 @@ public class GroupByMultiSegmentTest Assert.assertEquals(expectedRow, results.get(0)); } - private List> makeGroupByMultiRunners() + private List> makeGroupByMultiRunners() { - List> runners = new ArrayList<>(); + List> runners = new ArrayList<>(); for (QueryableIndex qindex : groupByIndices) { - QueryRunner runner = makeQueryRunner( + QueryRunner runner = makeQueryRunner( groupByFactory, SegmentId.dummy(qindex.toString()), new QueryableIndexSegment(qindex, SegmentId.dummy(qindex.toString())) @@ -422,11 +422,13 @@ public class GroupByMultiSegmentTest public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator() { - return new IntervalChunkingQueryRunnerDecorator(null, null, null) { + return new IntervalChunkingQueryRunnerDecorator(null, null, null) + { @Override public QueryRunner decorate(final QueryRunner delegate, QueryToolChest> toolChest) { - return new QueryRunner() { + return new QueryRunner() + { @Override public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java index ed170d78fab..d3183f5d53c 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -28,7 +28,6 @@ import com.google.common.collect.Lists; import org.apache.druid.collections.CloseableDefaultBlockingPool; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.collections.ReferenceCountingResourceHolder; -import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.DruidProcessingConfig; @@ -183,7 +182,7 @@ public class GroupByQueryMergeBufferTest } ); - private final QueryRunner runner; + private final QueryRunner runner; @AfterClass public static void teardownClass() @@ -196,13 +195,13 @@ public class GroupByQueryMergeBufferTest public static Collection constructorFeeder() { final List args = new ArrayList<>(); - for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { args.add(new Object[]{runner}); } return args; } - public GroupByQueryMergeBufferTest(QueryRunner runner) + public GroupByQueryMergeBufferTest(QueryRunner runner) { this.runner = factory.mergeRunners(Execs.directExecutor(), ImmutableList.of(runner)); } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java index 94842e08925..5195afdae5f 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java @@ -24,9 +24,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import org.apache.druid.collections.SerializablePair; -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.StringUtils; import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -101,12 +101,12 @@ public class GroupByQueryQueryToolChestTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( + final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( null, QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() ).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( + final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( null, QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() ).getCacheStrategy(query2); @@ -165,12 +165,12 @@ public class GroupByQueryQueryToolChestTest ) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( + final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( null, QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() ).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( + final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( null, QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() ).getCacheStrategy(query2); @@ -231,12 +231,12 @@ public class GroupByQueryQueryToolChestTest .setHavingSpec(new GreaterThanHavingSpec(QueryRunnerTestHelper.uniqueMetric, 10)) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( + final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( null, QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() ).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( + final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( null, QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() ).getCacheStrategy(query2); @@ -319,12 +319,12 @@ public class GroupByQueryQueryToolChestTest .setHavingSpec(andHavingSpec2) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( + final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( null, QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() ).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( + final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( null, QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() ).getCacheStrategy(query2); @@ -414,12 +414,12 @@ public class GroupByQueryQueryToolChestTest .setHavingSpec(havingSpec2) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( + final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( null, QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() ).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( + final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( null, QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() ).getCacheStrategy(query2); @@ -481,12 +481,12 @@ public class GroupByQueryQueryToolChestTest )) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( + final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( null, QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() ).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( + final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( null, QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() ).getCacheStrategy(query2); @@ -507,6 +507,113 @@ public class GroupByQueryQueryToolChestTest doTestCacheStrategy(ValueType.LONG, 2L); } + @Test + public void testResultSerde() throws Exception + { + final GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Collections.singletonList(DefaultDimensionSpec.of("test"))) + .setAggregatorSpecs(Collections.singletonList(QueryRunnerTestHelper.rowsCount)) + .setPostAggregatorSpecs(Collections.singletonList(new ConstantPostAggregator("post", 10))) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( + null, + QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + ); + + final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + final ObjectMapper arraysObjectMapper = toolChest.decorateObjectMapper( + objectMapper, + query.withOverriddenContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true)) + ); + final ObjectMapper mapsObjectMapper = toolChest.decorateObjectMapper( + objectMapper, + query.withOverriddenContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, false)) + ); + + final Object[] rowObjects = {DateTimes.of("2000").getMillis(), "foo", 100, 10}; + final ResultRow resultRow = ResultRow.of(rowObjects); + + Assert.assertEquals( + resultRow, + arraysObjectMapper.readValue( + StringUtils.format("[%s, \"foo\", 100, 10]", DateTimes.of("2000").getMillis()), + ResultRow.class + ) + ); + + Assert.assertEquals( + resultRow, + arraysObjectMapper.readValue( + StringUtils.format( + "{\"version\":\"v1\"," + + "\"timestamp\":\"%s\"," + + "\"event\":" + + " {\"test\":\"foo\", \"rows\":100, \"post\":10}" + + "}", + DateTimes.of("2000") + ), + ResultRow.class + ) + ); + + Assert.assertArrayEquals( + rowObjects, + objectMapper.readValue( + arraysObjectMapper.writeValueAsBytes(resultRow), + Object[].class + ) + ); + + Assert.assertEquals( + resultRow.toMapBasedRow(query), + objectMapper.readValue( + mapsObjectMapper.writeValueAsBytes(resultRow), + Row.class + ) + ); + + Assert.assertEquals( + "arrays read arrays", + resultRow, + arraysObjectMapper.readValue( + arraysObjectMapper.writeValueAsBytes(resultRow), + ResultRow.class + ) + ); + + Assert.assertEquals( + "arrays read maps", + resultRow, + arraysObjectMapper.readValue( + mapsObjectMapper.writeValueAsBytes(resultRow), + ResultRow.class + ) + ); + + Assert.assertEquals( + "maps read arrays", + resultRow, + mapsObjectMapper.readValue( + arraysObjectMapper.writeValueAsBytes(resultRow), + ResultRow.class + ) + ); + + Assert.assertEquals( + "maps read maps", + resultRow, + mapsObjectMapper.readValue( + mapsObjectMapper.writeValueAsBytes(resultRow), + ResultRow.class + ) + ); + } + private AggregatorFactory getComplexAggregatorFactoryForValueType(final ValueType valueType) { switch (valueType) { @@ -558,24 +665,15 @@ public class GroupByQueryQueryToolChestTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - CacheStrategy strategy = + CacheStrategy strategy = new GroupByQueryQueryToolChest(null, null).getCacheStrategy( query1 ); - final Row result1 = new MapBasedRow( - // test timestamps that result in integer size millis - DateTimes.utc(123L), - ImmutableMap.of( - "test", dimValue, - "rows", 1, - "complexMetric", getIntermediateComplexValue(valueType, dimValue) - ) - ); + // test timestamps that result in integer size millis + final ResultRow result1 = ResultRow.of(123L, dimValue, 1, getIntermediateComplexValue(valueType, dimValue)); - Object preparedValue = strategy.prepareForSegmentLevelCache().apply( - result1 - ); + Object preparedValue = strategy.prepareForSegmentLevelCache().apply(result1); ObjectMapper objectMapper = TestHelper.makeJsonMapper(); Object fromCacheValue = objectMapper.readValue( @@ -583,43 +681,19 @@ public class GroupByQueryQueryToolChestTest strategy.getCacheObjectClazz() ); - Row fromCacheResult = strategy.pullFromSegmentLevelCache().apply(fromCacheValue); + ResultRow fromCacheResult = strategy.pullFromSegmentLevelCache().apply(fromCacheValue); Assert.assertEquals(result1, fromCacheResult); - final Row result2 = new MapBasedRow( - // test timestamps that result in integer size millis - DateTimes.utc(123L), - ImmutableMap.of( - "test", dimValue, - "rows", 1, - "complexMetric", dimValue, - "post", 10 - ) - ); + // test timestamps that result in integer size millis + final ResultRow result2 = ResultRow.of(123L, dimValue, 1, dimValue, 10); // Please see the comments on aggregator serde and type handling in CacheStrategy.fetchAggregatorsFromCache() - final Row typeAdjustedResult2; + final ResultRow typeAdjustedResult2; if (valueType == ValueType.FLOAT) { - typeAdjustedResult2 = new MapBasedRow( - DateTimes.utc(123L), - ImmutableMap.of( - "test", dimValue, - "rows", 1, - "complexMetric", 2.1d, - "post", 10 - ) - ); + typeAdjustedResult2 = ResultRow.of(123L, dimValue, 1, 2.1d, 10); } else if (valueType == ValueType.LONG) { - typeAdjustedResult2 = new MapBasedRow( - DateTimes.utc(123L), - ImmutableMap.of( - "test", dimValue, - "rows", 1, - "complexMetric", 2, - "post", 10 - ) - ); + typeAdjustedResult2 = ResultRow.of(123L, dimValue, 1, 2, 10); } else { typeAdjustedResult2 = result2; } @@ -634,7 +708,7 @@ public class GroupByQueryQueryToolChestTest strategy.getCacheObjectClazz() ); - Row fromResultCacheResult = strategy.pullFromCache(true).apply(fromResultCacheValue); + ResultRow fromResultCacheResult = strategy.pullFromCache(true).apply(fromResultCacheValue); Assert.assertEquals(typeAdjustedResult2, fromResultCacheResult); } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java index a1e871f1099..5c62fed9847 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java @@ -20,7 +20,6 @@ package org.apache.druid.query.groupby; import com.google.common.collect.ImmutableList; -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.StringInputRowParser; @@ -54,6 +53,7 @@ import java.util.Arrays; import java.util.List; /** + * */ public class GroupByQueryRunnerFactoryTest { @@ -126,11 +126,11 @@ public class GroupByQueryRunnerFactoryTest } ); - Sequence result = mergedRunner.run(QueryPlus.wrap(query)); + Sequence result = mergedRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); - List 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", 4L) + List 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", 4L) ); TestHelper.assertExpectedObjects(expectedResults, result.toList(), ""); @@ -160,7 +160,7 @@ public class GroupByQueryRunnerFactoryTest "2011-01-12T00:00:00.000Z,product_1,t1", "2011-01-13T00:00:00.000Z,product_2,t2", "2011-01-14T00:00:00.000Z,product_3,t2", - }; + }; for (String row : rows) { incrementalIndex.add(parser.parse(row)); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java index e804de9cf49..2f855384283 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java @@ -27,7 +27,6 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.collections.CloseableDefaultBlockingPool; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.collections.ReferenceCountingResourceHolder; -import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.DruidProcessingConfig; @@ -159,7 +158,7 @@ public class GroupByQueryRunnerFailureTest } ); - private QueryRunner runner; + private QueryRunner runner; @AfterClass public static void teardownClass() @@ -172,13 +171,13 @@ public class GroupByQueryRunnerFailureTest public static Collection constructorFeeder() { final List args = new ArrayList<>(); - for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { args.add(new Object[]{runner}); } return args; } - public GroupByQueryRunnerFailureTest(QueryRunner runner) + public GroupByQueryRunnerFailureTest(QueryRunner runner) { this.runner = factory.mergeRunners(Execs.directExecutor(), ImmutableList.of(runner)); } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index c0fd03e1879..d1574885a60 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java @@ -33,7 +33,7 @@ import com.google.common.collect.Sets; import org.apache.druid.collections.CloseableDefaultBlockingPool; 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.Rows; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -106,11 +106,11 @@ import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.RegexDimFilter; import org.apache.druid.query.filter.SearchQueryDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.query.groupby.having.BaseHavingSpec; import org.apache.druid.query.groupby.having.DimFilterHavingSpec; import org.apache.druid.query.groupby.having.DimensionSelectorHavingSpec; import org.apache.druid.query.groupby.having.EqualToHavingSpec; import org.apache.druid.query.groupby.having.GreaterThanHavingSpec; +import org.apache.druid.query.groupby.having.HavingSpec; import org.apache.druid.query.groupby.having.OrHavingSpec; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.LimitSpec; @@ -187,7 +187,7 @@ public class GroupByQueryRunnerTest private static final Closer resourceCloser = Closer.create(); - private final QueryRunner runner; + private final QueryRunner runner; private final String runnerName; private final GroupByQueryRunnerFactory factory; private final GroupByQueryConfig config; @@ -418,7 +418,7 @@ public class GroupByQueryRunnerTest final Pair factoryAndCloser = makeQueryRunnerFactory(config); final GroupByQueryRunnerFactory factory = factoryAndCloser.lhs; resourceCloser.register(factoryAndCloser.rhs); - for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { for (boolean vectorize : ImmutableList.of(false, true)) { final String testName = StringUtils.format("config=%s, runner=%s, vectorize=%s", config, runner, vectorize); @@ -470,8 +470,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "automotive", @@ -484,7 +485,8 @@ public class GroupByQueryRunnerTest "idxDouble", 135.88510131835938d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "business", @@ -497,7 +499,8 @@ public class GroupByQueryRunnerTest "idxDouble", 118.57034 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "entertainment", @@ -510,7 +513,8 @@ public class GroupByQueryRunnerTest "idxDouble", 158.747224 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "health", @@ -523,7 +527,8 @@ public class GroupByQueryRunnerTest "idxDouble", 120.134704 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "mezzanine", @@ -536,7 +541,8 @@ public class GroupByQueryRunnerTest "idxDouble", 2871.8866900000003d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "news", @@ -549,7 +555,8 @@ public class GroupByQueryRunnerTest "idxDouble", 121.58358d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "premium", @@ -562,7 +569,8 @@ public class GroupByQueryRunnerTest "idxDouble", 2900.798647d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "technology", @@ -575,7 +583,8 @@ public class GroupByQueryRunnerTest "idxDouble", 78.622547d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "travel", @@ -589,7 +598,8 @@ public class GroupByQueryRunnerTest 119.922742d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "automotive", @@ -602,7 +612,8 @@ public class GroupByQueryRunnerTest "idxDouble", 147.42593d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "business", @@ -615,7 +626,8 @@ public class GroupByQueryRunnerTest "idxDouble", 112.987027d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "entertainment", @@ -628,7 +640,8 @@ public class GroupByQueryRunnerTest "idxDouble", 166.016049d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "health", @@ -641,7 +654,8 @@ public class GroupByQueryRunnerTest "idxDouble", 113.446008d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "mezzanine", @@ -654,7 +668,8 @@ public class GroupByQueryRunnerTest "idxDouble", 2448.830613d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "news", @@ -667,7 +682,8 @@ public class GroupByQueryRunnerTest "idxDouble", 114.290141d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "premium", @@ -680,7 +696,8 @@ public class GroupByQueryRunnerTest "idxDouble", 2506.415148d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "technology", @@ -693,7 +710,8 @@ public class GroupByQueryRunnerTest "idxDouble", 97.387433d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "travel", @@ -708,7 +726,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "groupBy"); } @@ -728,8 +746,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Collections.singletonList( + makeRow( + query, "2011-04-01", "alias0", null, "alias1", "foo", @@ -737,7 +756,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "missing-column"); } @@ -765,8 +784,9 @@ public class GroupByQueryRunnerTest ) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "travel", @@ -777,7 +797,8 @@ public class GroupByQueryRunnerTest "idx", 119L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "technology", @@ -788,7 +809,8 @@ public class GroupByQueryRunnerTest "idx", 78L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "premium", @@ -799,7 +821,8 @@ public class GroupByQueryRunnerTest "idx", 2900L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "news", @@ -810,7 +833,8 @@ public class GroupByQueryRunnerTest "idx", 121L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "mezzanine", @@ -821,7 +845,8 @@ public class GroupByQueryRunnerTest "idx", 2870L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "health", @@ -832,7 +857,8 @@ public class GroupByQueryRunnerTest "idx", 120L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "entertainment", @@ -843,7 +869,8 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "business", @@ -854,7 +881,8 @@ public class GroupByQueryRunnerTest "idx", 118L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "automotive", @@ -866,7 +894,8 @@ public class GroupByQueryRunnerTest 135L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "travel", @@ -877,7 +906,8 @@ public class GroupByQueryRunnerTest "idx", 126L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "technology", @@ -888,7 +918,8 @@ public class GroupByQueryRunnerTest "idx", 97L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "premium", @@ -899,7 +930,8 @@ public class GroupByQueryRunnerTest "idx", 2505L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "news", @@ -910,7 +942,8 @@ public class GroupByQueryRunnerTest "idx", 114L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "mezzanine", @@ -921,7 +954,8 @@ public class GroupByQueryRunnerTest "idx", 2447L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "health", @@ -932,7 +966,8 @@ public class GroupByQueryRunnerTest "idx", 113L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "entertainment", @@ -943,7 +978,8 @@ public class GroupByQueryRunnerTest "idx", 166L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "business", @@ -954,7 +990,8 @@ public class GroupByQueryRunnerTest "idx", 112L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "automotive", @@ -967,7 +1004,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "string-postAgg"); } @@ -1000,10 +1037,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotivex", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "businessx", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotivex", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "businessx", "rows", 1L, "idx", 118L), + makeRow( + query, "2011-04-01", "alias", "entertainmentx", @@ -1012,16 +1050,17 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "healthx", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzaninex", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "newsx", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premiumx", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technologyx", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travelx", "rows", 1L, "idx", 119L), + makeRow(query, "2011-04-01", "alias", "healthx", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzaninex", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "newsx", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premiumx", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technologyx", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travelx", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotivex", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "businessx", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-02", "alias", "automotivex", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "businessx", "rows", 1L, "idx", 112L), + makeRow( + query, "2011-04-02", "alias", "entertainmentx", @@ -1030,15 +1069,15 @@ public class GroupByQueryRunnerTest "idx", 166L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "healthx", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzaninex", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "newsx", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premiumx", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technologyx", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travelx", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "healthx", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzaninex", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "newsx", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premiumx", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technologyx", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travelx", "rows", 1L, "idx", 126L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "virtual-column"); } @@ -1053,29 +1092,29 @@ public class GroupByQueryRunnerTest .setGranularity(new DurationGranularity(86400L, 0L)) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + makeRow(query, "2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + makeRow(query, "2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "duration-granularity"); } @@ -1109,36 +1148,36 @@ public class GroupByQueryRunnerTest .overrideContext(ImmutableMap.of("sortByDimsFirst", true)) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "sort-by-dimensions-first"); } @@ -1154,19 +1193,19 @@ public class GroupByQueryRunnerTest .overrideContext(ImmutableMap.of("chunkPeriod", "P1D")) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 282L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 230L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 324L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 233L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 5317L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 235L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 5405L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 175L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 245L) + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 2L, "idx", 282L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 2L, "idx", 230L), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 324L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 2L, "idx", 233L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 5317L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 2L, "idx", 235L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 6L, "idx", 5405L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 2L, "idx", 175L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 2L, "idx", 245L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "chunk-period"); } @@ -1180,29 +1219,29 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel"), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive"), + makeRow(query, "2011-04-01", "alias", "business"), + makeRow(query, "2011-04-01", "alias", "entertainment"), + makeRow(query, "2011-04-01", "alias", "health"), + makeRow(query, "2011-04-01", "alias", "mezzanine"), + makeRow(query, "2011-04-01", "alias", "news"), + makeRow(query, "2011-04-01", "alias", "premium"), + makeRow(query, "2011-04-01", "alias", "technology"), + makeRow(query, "2011-04-01", "alias", "travel"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology"), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel") + makeRow(query, "2011-04-02", "alias", "automotive"), + makeRow(query, "2011-04-02", "alias", "business"), + makeRow(query, "2011-04-02", "alias", "entertainment"), + makeRow(query, "2011-04-02", "alias", "health"), + makeRow(query, "2011-04-02", "alias", "mezzanine"), + makeRow(query, "2011-04-02", "alias", "news"), + makeRow(query, "2011-04-02", "alias", "premium"), + makeRow(query, "2011-04-02", "alias", "technology"), + makeRow(query, "2011-04-02", "alias", "travel") ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "no-aggs"); } @@ -1220,19 +1259,19 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "a", "rows", 2L, "idx", 282L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "b", "rows", 2L, "idx", 230L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "e", "rows", 2L, "idx", 324L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "h", "rows", 2L, "idx", 233L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "m", "rows", 6L, "idx", 5317L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "n", "rows", 2L, "idx", 235L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "p", "rows", 6L, "idx", 5405L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "preferred", "rows", 26L, "idx", 12446L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "t", "rows", 4L, "idx", 420L) + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "a", "rows", 2L, "idx", 282L), + makeRow(query, "2011-04-01", "alias", "b", "rows", 2L, "idx", 230L), + makeRow(query, "2011-04-01", "alias", "e", "rows", 2L, "idx", 324L), + makeRow(query, "2011-04-01", "alias", "h", "rows", 2L, "idx", 233L), + makeRow(query, "2011-04-01", "alias", "m", "rows", 6L, "idx", 5317L), + makeRow(query, "2011-04-01", "alias", "n", "rows", 2L, "idx", 235L), + makeRow(query, "2011-04-01", "alias", "p", "rows", 6L, "idx", 5405L), + makeRow(query, "2011-04-01", "alias", "preferred", "rows", 26L, "idx", 12446L), + makeRow(query, "2011-04-01", "alias", "t", "rows", 4L, "idx", 420L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "multi-value-dim"); } @@ -1253,8 +1292,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "a", @@ -1265,7 +1305,8 @@ public class GroupByQueryRunnerTest "idx", 282L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "a", @@ -1276,7 +1317,8 @@ public class GroupByQueryRunnerTest "idx", 282L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "preferred", @@ -1287,7 +1329,8 @@ public class GroupByQueryRunnerTest "idx", 282L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "preferred", @@ -1300,7 +1343,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "two-multi-value-dims"); } @@ -1320,8 +1363,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "quality", "automotive", @@ -1332,7 +1376,8 @@ public class GroupByQueryRunnerTest "idx", 282L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "business", @@ -1343,7 +1388,8 @@ public class GroupByQueryRunnerTest "idx", 230L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "entertainment", @@ -1354,7 +1400,8 @@ public class GroupByQueryRunnerTest "idx", 324L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "health", @@ -1365,7 +1412,8 @@ public class GroupByQueryRunnerTest "idx", 233L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "mezzanine", @@ -1376,7 +1424,8 @@ public class GroupByQueryRunnerTest "idx", 5317L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "news", @@ -1387,7 +1436,8 @@ public class GroupByQueryRunnerTest "idx", 235L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "premium", @@ -1398,7 +1448,8 @@ public class GroupByQueryRunnerTest "idx", 5405L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "automotive", @@ -1409,7 +1460,8 @@ public class GroupByQueryRunnerTest "idx", 282L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "business", @@ -1420,7 +1472,8 @@ public class GroupByQueryRunnerTest "idx", 230L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "entertainment", @@ -1431,7 +1484,8 @@ public class GroupByQueryRunnerTest "idx", 324L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "health", @@ -1442,7 +1496,8 @@ public class GroupByQueryRunnerTest "idx", 233L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "mezzanine", @@ -1453,7 +1508,8 @@ public class GroupByQueryRunnerTest "idx", 5317L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "news", @@ -1464,7 +1520,8 @@ public class GroupByQueryRunnerTest "idx", 235L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "premium", @@ -1475,7 +1532,8 @@ public class GroupByQueryRunnerTest "idx", 5405L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "technology", @@ -1486,7 +1544,8 @@ public class GroupByQueryRunnerTest "idx", 175L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "travel", @@ -1497,7 +1556,8 @@ public class GroupByQueryRunnerTest "idx", 245L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "technology", @@ -1508,7 +1568,8 @@ public class GroupByQueryRunnerTest "idx", 175L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "travel", @@ -1521,7 +1582,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "one-multi-value-dim"); } @@ -1541,8 +1602,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "quality", "automotive", @@ -1553,7 +1615,8 @@ public class GroupByQueryRunnerTest "idx", 282L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "automotive", @@ -1564,7 +1627,8 @@ public class GroupByQueryRunnerTest "idx", 282L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "business", @@ -1575,7 +1639,8 @@ public class GroupByQueryRunnerTest "idx", 230L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "business", @@ -1586,7 +1651,8 @@ public class GroupByQueryRunnerTest "idx", 230L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "entertainment", @@ -1597,7 +1663,8 @@ public class GroupByQueryRunnerTest "idx", 324L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "entertainment", @@ -1608,7 +1675,8 @@ public class GroupByQueryRunnerTest "idx", 324L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "health", @@ -1619,7 +1687,8 @@ public class GroupByQueryRunnerTest "idx", 233L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "health", @@ -1630,7 +1699,8 @@ public class GroupByQueryRunnerTest "idx", 233L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "mezzanine", @@ -1641,7 +1711,8 @@ public class GroupByQueryRunnerTest "idx", 5317L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "mezzanine", @@ -1652,7 +1723,8 @@ public class GroupByQueryRunnerTest "idx", 5317L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "news", @@ -1663,7 +1735,8 @@ public class GroupByQueryRunnerTest "idx", 235L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "news", @@ -1674,7 +1747,8 @@ public class GroupByQueryRunnerTest "idx", 235L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "premium", @@ -1685,7 +1759,8 @@ public class GroupByQueryRunnerTest "idx", 5405L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "premium", @@ -1696,7 +1771,8 @@ public class GroupByQueryRunnerTest "idx", 5405L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "technology", @@ -1707,7 +1783,8 @@ public class GroupByQueryRunnerTest "idx", 175L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "technology", @@ -1718,7 +1795,8 @@ public class GroupByQueryRunnerTest "idx", 175L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "travel", @@ -1729,7 +1807,8 @@ public class GroupByQueryRunnerTest "idx", 245L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "quality", "travel", @@ -1742,7 +1821,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "one-multi-value-dim-different-order"); } @@ -1758,14 +1837,15 @@ public class GroupByQueryRunnerTest .overrideContext(ImmutableMap.of("maxResults", 1)) .build(); - List expectedResults = null; + List expectedResults = null; if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { expectedException.expect(ResourceLimitExceededException.class); } else { expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow( + query, "2011-04-01", "alias", "entertainment", @@ -1774,16 +1854,17 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + makeRow( + query, "2011-04-02", "alias", "entertainment", @@ -1792,16 +1873,16 @@ public class GroupByQueryRunnerTest "idx", 166L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); } - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "override-maxResults"); } @@ -1817,29 +1898,29 @@ public class GroupByQueryRunnerTest .overrideContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 60000)) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + makeRow(query, "2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + makeRow(query, "2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "override-timeout"); } @@ -1855,15 +1936,16 @@ public class GroupByQueryRunnerTest .overrideContext(ImmutableMap.of("maxOnDiskStorage", 0, "bufferGrouperMaxSize", 1)) .build(); - List expectedResults = null; + List expectedResults = null; if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { expectedException.expect(ResourceLimitExceededException.class); expectedException.expectMessage("Not enough aggregation buffer space to execute this query"); } else { expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow( + query, "2011-04-01", "alias", "entertainment", @@ -1872,16 +1954,17 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + makeRow( + query, "2011-04-02", "alias", "entertainment", @@ -1890,16 +1973,16 @@ public class GroupByQueryRunnerTest "idx", 166L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); } - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "overide-maxOnDiskStorage"); } @@ -1915,15 +1998,16 @@ public class GroupByQueryRunnerTest .overrideContext(ImmutableMap.of("maxOnDiskStorage", 0, "maxMergingDictionarySize", 1)) .build(); - List expectedResults = null; + List expectedResults = null; if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { expectedException.expect(ResourceLimitExceededException.class); expectedException.expectMessage("Not enough dictionary space to execute this query"); } else { expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow( + query, "2011-04-01", "alias", "entertainment", @@ -1932,16 +2016,17 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + makeRow( + query, "2011-04-02", "alias", "entertainment", @@ -1950,16 +2035,16 @@ public class GroupByQueryRunnerTest "idx", 166L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); } - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "dictionary-space"); } @@ -1975,7 +2060,7 @@ public class GroupByQueryRunnerTest .overrideContext(ImmutableMap.of("maxOnDiskStorage", 1, "maxMergingDictionarySize", 1)) .build(); - List expectedResults = null; + List expectedResults = null; if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { expectedException.expect(ResourceLimitExceededException.class); if (config.getMaxOnDiskStorage() > 0) { @@ -1986,9 +2071,10 @@ public class GroupByQueryRunnerTest } } else { expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow( + query, "2011-04-01", "alias", "entertainment", @@ -1997,16 +2083,17 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + makeRow( + query, "2011-04-02", "alias", "entertainment", @@ -2015,16 +2102,16 @@ public class GroupByQueryRunnerTest "idx", 166L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); } - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "disk-space"); } @@ -2100,10 +2187,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive0", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business0", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive0", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business0", "rows", 1L, "idx", 118L), + makeRow( + query, "2011-04-01", "alias", "entertainment0", @@ -2112,16 +2200,17 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health0", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine0", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news0", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium0", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology0", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel0", "rows", 1L, "idx", 119L), + makeRow(query, "2011-04-01", "alias", "health0", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine0", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news0", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium0", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology0", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel0", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive0", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business0", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-02", "alias", "automotive0", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business0", "rows", 1L, "idx", 112L), + makeRow( + query, "2011-04-02", "alias", "entertainment0", @@ -2130,15 +2219,15 @@ public class GroupByQueryRunnerTest "idx", 166L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health0", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine0", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news0", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium0", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology0", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel0", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "health0", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine0", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news0", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium0", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology0", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel0", "rows", 1L, "idx", 126L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "rebucket-rename"); } @@ -2169,10 +2258,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive0", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business0", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive0", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business0", "rows", 1L, "idx", 118L), + makeRow( + query, "2011-04-01", "alias", "entertainment0", @@ -2181,16 +2271,17 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health0", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine0", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news0", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium0", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology0", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel0", "rows", 1L, "idx", 119L), + makeRow(query, "2011-04-01", "alias", "health0", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine0", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news0", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium0", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology0", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel0", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive0", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business0", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-02", "alias", "automotive0", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business0", "rows", 1L, "idx", 112L), + makeRow( + query, "2011-04-02", "alias", "entertainment0", @@ -2199,15 +2290,15 @@ public class GroupByQueryRunnerTest "idx", 166L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health0", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine0", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news0", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium0", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology0", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel0", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "health0", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine0", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news0", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium0", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology0", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel0", "rows", 1L, "idx", 126L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "non-injective"); } @@ -2227,53 +2318,41 @@ public class GroupByQueryRunnerTest map.put("travel", "travel0"); GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.dataSource) - .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird).setDimensions(new ExtractionDimensionSpec( - "quality", - "alias", - new LookupExtractionFn(new MapLookupExtractor(map, false), true, null, true, false) - )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + new ExtractionDimensionSpec( + "quality", + "alias", + new LookupExtractionFn(new MapLookupExtractor(map, false), true, null, true, false) + ) + ) + .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive0", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business0", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "entertainment0", - "rows", - 1L, - "idx", - 158L - ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health0", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine0", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news0", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium0", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology0", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel0", "rows", 1L, "idx", 119L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive0", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business0", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-01", "alias", "entertainment0", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-01", "alias", "health0", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine0", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news0", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium0", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology0", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel0", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive0", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business0", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "entertainment0", - "rows", - 1L, - "idx", - 166L - ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health0", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine0", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news0", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium0", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology0", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel0", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "automotive0", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business0", "rows", 1L, "idx", 112L), + makeRow(query, "2011-04-02", "alias", "entertainment0", "rows", 1L, "idx", 166L), + makeRow(query, "2011-04-02", "alias", "health0", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine0", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news0", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium0", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology0", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel0", "rows", 1L, "idx", 126L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "retain-missing"); } @@ -2301,45 +2380,29 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive0", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business0", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "entertainment0", - "rows", - 1L, - "idx", - 158L - ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health0", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine0", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news0", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium0", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology0", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel0", "rows", 1L, "idx", 119L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive0", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business0", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-01", "alias", "entertainment0", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-01", "alias", "health0", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine0", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news0", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium0", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology0", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel0", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive0", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business0", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "entertainment0", - "rows", - 1L, - "idx", - 166L - ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health0", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine0", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news0", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium0", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology0", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel0", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "automotive0", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business0", "rows", 1L, "idx", 112L), + makeRow(query, "2011-04-02", "alias", "entertainment0", "rows", 1L, "idx", 166L), + makeRow(query, "2011-04-02", "alias", "health0", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine0", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news0", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium0", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology0", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel0", "rows", 1L, "idx", 126L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "rename-and-missing-string"); } @@ -2366,10 +2429,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive0", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business0", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive0", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business0", "rows", 1L, "idx", 118L), + makeRow( + query, "2011-04-01", "alias", "entertainment0", @@ -2378,16 +2442,17 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health0", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine0", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news0", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium0", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology0", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel0", "rows", 1L, "idx", 119L), + makeRow(query, "2011-04-01", "alias", "health0", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine0", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news0", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium0", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology0", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel0", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive0", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business0", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-02", "alias", "automotive0", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business0", "rows", 1L, "idx", 112L), + makeRow( + query, "2011-04-02", "alias", "entertainment0", @@ -2396,15 +2461,15 @@ public class GroupByQueryRunnerTest "idx", 166L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health0", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine0", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news0", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium0", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology0", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel0", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "health0", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine0", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news0", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium0", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology0", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel0", "rows", 1L, "idx", 126L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "simple-rename"); } @@ -2418,8 +2483,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Collections.singletonList( + makeRow( + query, "2011-04-01", "rows", 26L, @@ -2428,7 +2494,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "uniques"); } @@ -2450,8 +2516,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Collections.singletonList( + makeRow( + query, "2011-04-01", "rows", 26L, @@ -2460,7 +2527,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "unique-postagg-same-name"); } @@ -2477,8 +2544,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Collections.singletonList( + makeRow( + query, "2011-04-01", "rows", 26L, @@ -2487,7 +2555,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "cardinality"); } @@ -2508,9 +2576,10 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.monthGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-01-01", "market", "spot", "first", 100L, "last", 155L), - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow(query, "2011-01-01", "market", "spot", "first", 100L, "last", 155L), + makeRow( + query, "2011-01-01", "market", "total_market", @@ -2519,9 +2588,10 @@ public class GroupByQueryRunnerTest "last", 1127L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-01-01", "market", "upfront", "first", 800L, "last", 943L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-02-01", "market", "spot", "first", 132L, "last", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-01-01", "market", "upfront", "first", 800L, "last", 943L), + makeRow(query, "2011-02-01", "market", "spot", "first", 132L, "last", 114L), + makeRow( + query, "2011-02-01", "market", "total_market", @@ -2530,7 +2600,8 @@ public class GroupByQueryRunnerTest "last", 1292L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-02-01", "market", "upfront", @@ -2539,8 +2610,9 @@ public class GroupByQueryRunnerTest "last", 1101L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-03-01", "market", "spot", "first", 153L, "last", 125L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-03-01", "market", "spot", "first", 153L, "last", 125L), + makeRow( + query, "2011-03-01", "market", "total_market", @@ -2549,7 +2621,8 @@ public class GroupByQueryRunnerTest "last", 1366L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-03-01", "market", "upfront", @@ -2558,8 +2631,9 @@ public class GroupByQueryRunnerTest "last", 1063L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "market", "spot", "first", 135L, "last", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-01", "market", "spot", "first", 135L, "last", 120L), + makeRow( + query, "2011-04-01", "market", "total_market", @@ -2568,10 +2642,10 @@ public class GroupByQueryRunnerTest "last", 1029L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "market", "upfront", "first", 1447L, "last", 780L) + makeRow(query, "2011-04-01", "market", "upfront", "first", 1447L, "last", 780L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "first-last-aggs"); } @@ -2595,8 +2669,8 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = ImmutableList.of(); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + List expectedResults = ImmutableList.of(); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); Assert.assertEquals(expectedResults, results); } @@ -2628,24 +2702,24 @@ public class GroupByQueryRunnerTest .setDimensions(new ExtractionDimensionSpec("quality", "alias", nullExtractionFn)) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", null, "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "a", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "b", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "e", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "h", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "n", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "p", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "t", "rows", 2L, "idx", 197L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", null, "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "a", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "b", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-01", "alias", "e", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-01", "alias", "h", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "n", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "p", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "t", "rows", 2L, "idx", 197L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", null, "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "a", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "b", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "e", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "h", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "n", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "p", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "t", "rows", 2L, "idx", 223L) + makeRow(query, "2011-04-02", "alias", null, "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "a", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "b", "rows", 1L, "idx", 112L), + makeRow(query, "2011-04-02", "alias", "e", "rows", 1L, "idx", 166L), + makeRow(query, "2011-04-02", "alias", "h", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "n", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "p", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "t", "rows", 2L, "idx", 223L) ); TestHelper.assertExpectedObjects( @@ -2687,24 +2761,24 @@ public class GroupByQueryRunnerTest .setDimensions(new ExtractionDimensionSpec("quality", "alias", emptyStringExtractionFn)) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "a", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "b", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "e", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "h", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "n", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "p", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "t", "rows", 2L, "idx", 197L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "a", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "b", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-01", "alias", "e", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-01", "alias", "h", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "n", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "p", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "t", "rows", 2L, "idx", 197L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "a", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "b", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "e", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "h", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "n", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "p", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "t", "rows", 2L, "idx", 223L) + makeRow(query, "2011-04-02", "alias", "", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "a", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "b", "rows", 1L, "idx", 112L), + makeRow(query, "2011-04-02", "alias", "e", "rows", 1L, "idx", 166L), + makeRow(query, "2011-04-02", "alias", "h", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "n", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "p", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "t", "rows", 2L, "idx", 223L) ); TestHelper.assertExpectedObjects( @@ -2736,8 +2810,9 @@ public class GroupByQueryRunnerTest ) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, new DateTime("2011-03-31", tz), "alias", "automotive", @@ -2746,7 +2821,8 @@ public class GroupByQueryRunnerTest "idx", 135L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-03-31", tz), "alias", "business", @@ -2755,7 +2831,8 @@ public class GroupByQueryRunnerTest "idx", 118L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-03-31", tz), "alias", "entertainment", @@ -2764,7 +2841,8 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-03-31", tz), "alias", "health", @@ -2773,7 +2851,8 @@ public class GroupByQueryRunnerTest "idx", 120L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-03-31", tz), "alias", "mezzanine", @@ -2782,7 +2861,8 @@ public class GroupByQueryRunnerTest "idx", 2870L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-03-31", tz), "alias", "news", @@ -2791,7 +2871,8 @@ public class GroupByQueryRunnerTest "idx", 121L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-03-31", tz), "alias", "premium", @@ -2800,7 +2881,8 @@ public class GroupByQueryRunnerTest "idx", 2900L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-03-31", tz), "alias", "technology", @@ -2809,7 +2891,8 @@ public class GroupByQueryRunnerTest "idx", 78L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-03-31", tz), "alias", "travel", @@ -2819,7 +2902,8 @@ public class GroupByQueryRunnerTest 119L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-04-01", tz), "alias", "automotive", @@ -2828,7 +2912,8 @@ public class GroupByQueryRunnerTest "idx", 147L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-04-01", tz), "alias", "business", @@ -2837,7 +2922,8 @@ public class GroupByQueryRunnerTest "idx", 112L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-04-01", tz), "alias", "entertainment", @@ -2846,7 +2932,8 @@ public class GroupByQueryRunnerTest "idx", 166L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-04-01", tz), "alias", "health", @@ -2855,7 +2942,8 @@ public class GroupByQueryRunnerTest "idx", 113L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-04-01", tz), "alias", "mezzanine", @@ -2864,7 +2952,8 @@ public class GroupByQueryRunnerTest "idx", 2447L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-04-01", tz), "alias", "news", @@ -2873,7 +2962,8 @@ public class GroupByQueryRunnerTest "idx", 114L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-04-01", tz), "alias", "premium", @@ -2882,7 +2972,8 @@ public class GroupByQueryRunnerTest "idx", 2505L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-04-01", tz), "alias", "technology", @@ -2891,7 +2982,8 @@ public class GroupByQueryRunnerTest "idx", 97L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, new DateTime("2011-04-01", tz), "alias", "travel", @@ -2902,7 +2994,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "timezone"); } @@ -2920,10 +3012,10 @@ public class GroupByQueryRunnerTest final GroupByQuery allGranQuery = builder.copy().setGranularity(Granularities.ALL).build(); QueryRunner mergedRunner = factory.getToolchest().mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( @@ -2942,36 +3034,36 @@ public class GroupByQueryRunnerTest } ); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) + List expectedResults = Arrays.asList( + makeRow(fullQuery, "2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + makeRow(fullQuery, "2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + makeRow(fullQuery, "2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + makeRow(fullQuery, "2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + makeRow(fullQuery, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + makeRow(fullQuery, "2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + makeRow(fullQuery, "2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + makeRow(fullQuery, "2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + makeRow(fullQuery, "2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); ResponseContext context = ResponseContext.createEmpty(); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged"); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery)), "merged"); - List allGranExpectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 2L, "idx", 217L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 2L, "idx", 319L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 2L, "idx", 216L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 2L, "idx", 221L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 6L, "idx", 4416L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 2L, "idx", 177L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) + List allGranExpectedResults = Arrays.asList( + makeRow(allGranQuery, "2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), + makeRow(allGranQuery, "2011-04-02", "alias", "business", "rows", 2L, "idx", 217L), + makeRow(allGranQuery, "2011-04-02", "alias", "entertainment", "rows", 2L, "idx", 319L), + makeRow(allGranQuery, "2011-04-02", "alias", "health", "rows", 2L, "idx", 216L), + makeRow(allGranQuery, "2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + makeRow(allGranQuery, "2011-04-02", "alias", "news", "rows", 2L, "idx", 221L), + makeRow(allGranQuery, "2011-04-02", "alias", "premium", "rows", 6L, "idx", 4416L), + makeRow(allGranQuery, "2011-04-02", "alias", "technology", "rows", 2L, "idx", 177L), + makeRow(allGranQuery, "2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) ); TestHelper.assertExpectedObjects( allGranExpectedResults, - mergedRunner.run(QueryPlus.wrap(allGranQuery), context), + mergedRunner.run(QueryPlus.wrap(allGranQuery)), "merged" ); } @@ -2996,19 +3088,100 @@ public class GroupByQueryRunnerTest final GroupByQuery fullQuery = builder.build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) + List expectedResults = Arrays.asList( + makeRow( + fullQuery, + "2011-04-01", + "alias", + "automotive", + "rows", + 2L, + "idx", + 269L + ), + makeRow( + fullQuery, + "2011-04-01", + "alias", + "business", + "rows", + 2L, + "idx", + 217L + ), + makeRow( + fullQuery, + "2011-04-01", + "alias", + "entertainment", + "rows", + 2L, + "idx", + 319L + ), + makeRow( + fullQuery, + "2011-04-01", + "alias", + "health", + "rows", + 2L, + "idx", + 216L + ), + makeRow( + fullQuery, + "2011-04-01", + "alias", + "mezzanine", + "rows", + 6L, + "idx", + 4420L + ), + makeRow( + fullQuery, + "2011-04-01", + "alias", + "news", + "rows", + 2L, + "idx", + 221L + ), + makeRow( + fullQuery, + "2011-04-01", + "alias", + "premium", + "rows", + 6L, + "idx", + 4416L + ), + makeRow( + fullQuery, + "2011-04-01", + "alias", + "technology", + "rows", + 2L, + "idx", + 177L + ), + makeRow( + fullQuery, + "2011-04-01", + "alias", + "travel", + "rows", + 2L, + "idx", + 243L + ) ); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, limit), @@ -3032,25 +3205,25 @@ public class GroupByQueryRunnerTest GroupByQuery fullQuery = builder.build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + List expectedResults = Arrays.asList( + makeRow(fullQuery, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(fullQuery, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(fullQuery, "2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + makeRow(fullQuery, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + makeRow(fullQuery, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(fullQuery, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(fullQuery, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + makeRow(fullQuery, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow(fullQuery, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + makeRow(fullQuery, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + makeRow(fullQuery, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(fullQuery, "2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + makeRow(fullQuery, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + makeRow(fullQuery, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, limit), @@ -3085,25 +3258,25 @@ public class GroupByQueryRunnerTest GroupByQuery fullQuery = builder.build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 6090L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 6030L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 333L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 285L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 255L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 252L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 251L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 248L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 165L), + List expectedResults = Arrays.asList( + makeRow(fullQuery, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 6090L), + makeRow(fullQuery, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 6030L), + makeRow(fullQuery, "2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 333L), + makeRow(fullQuery, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 285L), + makeRow(fullQuery, "2011-04-01", "alias", "news", "rows", 1L, "idx", 255L), + makeRow(fullQuery, "2011-04-01", "alias", "health", "rows", 1L, "idx", 252L), + makeRow(fullQuery, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 251L), + makeRow(fullQuery, "2011-04-01", "alias", "business", "rows", 1L, "idx", 248L), + makeRow(fullQuery, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 165L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 5262L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 5141L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 348L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 309L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 265L) + makeRow(fullQuery, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 5262L), + makeRow(fullQuery, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 5141L), + makeRow(fullQuery, "2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 348L), + makeRow(fullQuery, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 309L), + makeRow(fullQuery, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 265L) ); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, limit), @@ -3136,45 +3309,34 @@ public class GroupByQueryRunnerTest new DefaultLimitSpec(OrderByColumnSpec.descending("rows", "idx"), null), }; - final Comparator idxComparator = - new Comparator() - { - @Override - public int compare(Row o1, Row o2) - { - return Float.compare(o1.getMetric("idx").floatValue(), o2.getMetric("idx").floatValue()); - } - }; + GroupByQuery baseQuery = makeQueryBuilder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04") + .setDimensions(new DefaultDimensionSpec("quality", "alias")) + .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) + .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) + .build(); - Comparator rowsIdxComparator = - new Comparator() - { - - @Override - public int compare(Row o1, Row o2) - { - int value = Float.compare(o1.getMetric("rows").floatValue(), o2.getMetric("rows").floatValue()); - if (value != 0) { - return value; - } - - return idxComparator.compare(o1, o2); - } - }; - - List allResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) + List allResults = Arrays.asList( + makeRow(baseQuery, "2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + makeRow(baseQuery, "2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + makeRow(baseQuery, "2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + makeRow(baseQuery, "2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + makeRow(baseQuery, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + makeRow(baseQuery, "2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + makeRow(baseQuery, "2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + makeRow(baseQuery, "2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + makeRow(baseQuery, "2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); - List> expectedResults = Lists.newArrayList( + final int idxPosition = baseQuery.getResultRowPositionLookup().getInt("idx"); + final int rowsPosition = baseQuery.getResultRowPositionLookup().getInt("rows"); + + Comparator idxComparator = Comparator.comparing(row -> ((Number) row.get(idxPosition)).floatValue()); + Comparator rowsComparator = Comparator.comparing(row -> ((Number) row.get(rowsPosition)).floatValue()); + Comparator rowsIdxComparator = Ordering.from(rowsComparator).thenComparing(idxComparator); + + List> expectedResults = Lists.newArrayList( Ordering.from(idxComparator).sortedCopy(allResults), Ordering.from(rowsIdxComparator).sortedCopy(allResults), Ordering.from(idxComparator).reverse().sortedCopy(allResults), @@ -3182,27 +3344,21 @@ public class GroupByQueryRunnerTest ); for (int i = 0; i < orderBySpecs.length; ++i) { - doTestMergeResultsWithOrderBy(orderBySpecs[i], expectedResults.get(i)); + doTestMergeResultsWithOrderBy(baseQuery, orderBySpecs[i], expectedResults.get(i)); } } - private void doTestMergeResultsWithOrderBy(LimitSpec orderBySpec, List expectedResults) + private void doTestMergeResultsWithOrderBy( + GroupByQuery baseQuery, + LimitSpec limitSpec, + List expectedResults + ) { - GroupByQuery.Builder builder = makeQueryBuilder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setInterval("2011-04-02/2011-04-04") - .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) - .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) - .setLimitSpec(orderBySpec); - - final GroupByQuery fullQuery = builder.build(); - QueryRunner mergedRunner = factory.getToolchest().mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( @@ -3221,7 +3377,8 @@ public class GroupByQueryRunnerTest } ); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery)), "merged"); + final GroupByQuery query = baseQuery.withLimitSpec(limitSpec); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(query)), "merged"); } @Test @@ -3241,25 +3398,24 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L) + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L) ); - ResponseContext context = ResponseContext.createEmpty(); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit"); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit"); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), - mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build())), "limited" ); @@ -3270,7 +3426,8 @@ public class GroupByQueryRunnerTest ); builder.setLimit(Integer.MAX_VALUE).setAggregatorSpecs(aggregatorSpecs); - expectedResults = GroupByQueryRunnerTestHelper.createExpectedRows( + expectedResults = makeRows( + builder.build(), new String[]{"__time", "alias", "rows", "idx"}, new Object[]{"2011-04-01", "travel", 2L, 365.4876403808594D}, new Object[]{"2011-04-01", "technology", 2L, 267.3737487792969D}, @@ -3285,12 +3442,12 @@ public class GroupByQueryRunnerTest TestHelper.assertExpectedObjects( expectedResults, - mergeRunner.run(QueryPlus.wrap(builder.build()), context), + mergeRunner.run(QueryPlus.wrap(builder.build())), "no-limit" ); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), - mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build())), "limited" ); @@ -3309,12 +3466,12 @@ public class GroupByQueryRunnerTest TestHelper.assertExpectedObjects( expectedResults, - mergeRunner.run(QueryPlus.wrap(builder.build()), context), + mergeRunner.run(QueryPlus.wrap(builder.build())), "no-limit" ); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), - mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build())), "limited" ); } @@ -3333,24 +3490,23 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L) + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L) ); - ResponseContext context = ResponseContext.createEmpty(); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit"); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit"); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), - mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build())), "limited" ); } @@ -3369,7 +3525,8 @@ public class GroupByQueryRunnerTest GroupByQuery query = builder.build(); - List expectedResults = GroupByQueryRunnerTestHelper.createExpectedRows( + List expectedResults = makeRows( + query, new String[]{"__time", "alias", "rows", "idx"}, new Object[]{"2011-04-01", "mezzanine", 6L, 4423.6533203125D}, new Object[]{"2011-04-01", "premium", 6L, 4418.61865234375D}, @@ -3382,12 +3539,11 @@ public class GroupByQueryRunnerTest new Object[]{"2011-04-01", "technology", 2L, 178.24917602539062D} ); - ResponseContext context = ResponseContext.createEmpty(); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit"); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit"); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), - mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build())), "limited" ); } @@ -3414,24 +3570,23 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); - ResponseContext context = ResponseContext.createEmpty(); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit"); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit"); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), - mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build())), "limited" ); } @@ -3454,22 +3609,25 @@ public class GroupByQueryRunnerTest ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "1970-01-01T00:00:00.000Z", "marketalias", "upfront", "rows", 186L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01T00:00:00.000Z", "marketalias", "total_market", "rows", 186L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01T00:00:00.000Z", "marketalias", "spot", @@ -3478,7 +3636,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "order-limit"); } @@ -3503,19 +3661,20 @@ public class GroupByQueryRunnerTest ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "market", "upfront", "rows", 186L), - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow(query, "1970-01-01T00:00:00.000Z", "market", "upfront", "rows", 186L), + makeRow( + query, "1970-01-01T00:00:00.000Z", "market", "total_market", "rows", 186L ), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "market", "spot", "rows", 837L) + makeRow(query, "1970-01-01T00:00:00.000Z", "market", "spot", "rows", 837L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "order-limit"); } @@ -3548,8 +3707,9 @@ public class GroupByQueryRunnerTest ) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "1970-01-01T00:00:00.000Z", "market", "spot", @@ -3558,7 +3718,8 @@ public class GroupByQueryRunnerTest QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric, QueryRunnerTestHelper.UNIQUES_9 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01T00:00:00.000Z", "market", "upfront", @@ -3567,7 +3728,8 @@ public class GroupByQueryRunnerTest QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric, QueryRunnerTestHelper.UNIQUES_2 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01T00:00:00.000Z", "market", "total_market", @@ -3578,7 +3740,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "order-limit"); } @@ -3613,8 +3775,9 @@ public class GroupByQueryRunnerTest ) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Collections.singletonList( + makeRow( + query, "1970-01-01T00:00:00.000Z", "market", "spot", @@ -3625,7 +3788,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "order-limit"); } @@ -3663,8 +3826,9 @@ public class GroupByQueryRunnerTest ) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Collections.singletonList( + makeRow( + query, "1970-01-01T00:00:00.000Z", "market", "spot", @@ -3675,7 +3839,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "order-limit"); } @@ -3710,8 +3874,9 @@ public class GroupByQueryRunnerTest ) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "1970-01-01T00:00:00.000Z", "market", "spot", @@ -3720,7 +3885,8 @@ public class GroupByQueryRunnerTest QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric, QueryRunnerTestHelper.UNIQUES_9 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01T00:00:00.000Z", "market", "upfront", @@ -3729,7 +3895,8 @@ public class GroupByQueryRunnerTest QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric, QueryRunnerTestHelper.UNIQUES_2 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01T00:00:00.000Z", "market", "total_market", @@ -3740,7 +3907,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "order-limit"); } @@ -3777,28 +3944,28 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health0000", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health09", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health20", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health55", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health105", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health999", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel47", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel123", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel555", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health0000", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health09", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health20", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health55", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health105", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health999", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel47", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel123", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel555", "rows", 1L, "idx", 126L) + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "health0000", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "health09", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "health20", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-01", "alias", "health55", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "health105", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "health999", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "travel47", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-01", "alias", "travel123", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel555", "rows", 1L, "idx", 119L), + makeRow(query, "2011-04-02", "alias", "health0000", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "health09", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "health20", "rows", 1L, "idx", 112L), + makeRow(query, "2011-04-02", "alias", "health55", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "health105", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "health999", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "travel47", "rows", 1L, "idx", 166L), + makeRow(query, "2011-04-02", "alias", "travel123", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel555", "rows", 1L, "idx", 126L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "alphanumeric-dimension-order"); } @@ -3836,26 +4003,26 @@ public class GroupByQueryRunnerTest .overrideContext(ImmutableMap.of("sortByDimsFirst", true)) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "1", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "1", "rows", 1L, "idx", 126L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "1", "rows", 1L, "idx", 119L), + makeRow(query, "2011-04-02", "alias", "1", "rows", 1L, "idx", 126L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "2", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "2", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-01", "alias", "2", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-02", "alias", "2", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "3", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "3", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-01", "alias", "3", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-02", "alias", "3", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "4", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "4", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-01", "alias", "4", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-02", "alias", "4", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "5", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "5", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-01", "alias", "5", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-02", "alias", "5", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "6", "rows", 1L, "idx", 120L) + makeRow(query, "2011-04-01", "alias", "6", "rows", 1L, "idx", 120L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "lookup-limit"); } @@ -3882,12 +4049,12 @@ public class GroupByQueryRunnerTest ).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01T00:00:00.000Z", "market", "spot", "rows", 9L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02T00:00:00.000Z", "market", "spot", "rows", 9L) + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01T00:00:00.000Z", "market", "spot", "rows", 9L), + makeRow(query, "2011-04-02T00:00:00.000Z", "market", "spot", "rows", 9L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); Iterator resultsIter = results.iterator(); Iterator expectedResultsIter = expectedResults.iterator(); @@ -3903,31 +4070,6 @@ public class GroupByQueryRunnerTest @Test public void testPostAggMergedHavingSpec() { - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "mezzanine", - "rows", - 6L, - "index", - 4420L, - QueryRunnerTestHelper.addRowsIndexConstantMetric, - (double) (6L + 4420L + 1L) - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "premium", - "rows", - 6L, - "index", - 4416L, - QueryRunnerTestHelper.addRowsIndexConstantMetric, - (double) (6L + 4416L + 1L) - ) - ); - GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.dataSource) .setInterval("2011-04-02/2011-04-04") @@ -3942,12 +4084,38 @@ public class GroupByQueryRunnerTest ); final GroupByQuery fullQuery = builder.build(); + List expectedResults = Arrays.asList( + makeRow( + fullQuery, + "2011-04-01", + "alias", + "mezzanine", + "rows", + 6L, + "index", + 4420L, + QueryRunnerTestHelper.addRowsIndexConstantMetric, + (double) (6L + 4420L + 1L) + ), + makeRow( + fullQuery, + "2011-04-01", + "alias", + "premium", + "rows", + 6L, + "index", + 4416L, + QueryRunnerTestHelper.addRowsIndexConstantMetric, + (double) (6L + 4416L + 1L) + ) + ); QueryRunner mergedRunner = factory.getToolchest().mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( @@ -3986,56 +4154,17 @@ public class GroupByQueryRunnerTest ) ); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-01-25", - "alias", - "business", - "rows", - 3L, - "index", - 312.38165283203125 - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-01-25", - "alias", - "news", - "rows", - 3L, - "index", - 312.7834167480469 - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-01-25", - "alias", - "technology", - "rows", - 3L, - "index", - 324.6412353515625 - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-01-25", - "alias", - "travel", - "rows", - 3L, - "index", - 393.36322021484375 - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-01-25", - "alias", - "health", - "rows", - 3L, - "index", - 511.2996826171875 - ) + GroupByQuery fullQuery = builder.build(); + + List expectedResults = Arrays.asList( + makeRow(fullQuery, "2011-01-25", "alias", "business", "rows", 3L, "index", 312.38165283203125), + makeRow(fullQuery, "2011-01-25", "alias", "news", "rows", 3L, "index", 312.7834167480469), + makeRow(fullQuery, "2011-01-25", "alias", "technology", "rows", 3L, "index", 324.6412353515625), + makeRow(fullQuery, "2011-01-25", "alias", "travel", "rows", 3L, "index", 393.36322021484375), + makeRow(fullQuery, "2011-01-25", "alias", "health", "rows", 3L, "index", 511.2996826171875) ); - GroupByQuery fullQuery = builder.build(); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, fullQuery); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, fullQuery); TestHelper.assertExpectedObjects( expectedResults, results, @@ -4046,31 +4175,6 @@ public class GroupByQueryRunnerTest @Test public void testPostAggHavingSpec() { - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "mezzanine", - "rows", - 6L, - "index", - 4420L, - QueryRunnerTestHelper.addRowsIndexConstantMetric, - (double) (6L + 4420L + 1L) - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "premium", - "rows", - 6L, - "index", - 4416L, - QueryRunnerTestHelper.addRowsIndexConstantMetric, - (double) (6L + 4416L + 1L) - ) - ); - GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.dataSource) .setInterval("2011-04-02/2011-04-04") @@ -4087,6 +4191,34 @@ public class GroupByQueryRunnerTest ); final GroupByQuery fullQuery = builder.build(); + + List expectedResults = Arrays.asList( + makeRow( + fullQuery, + "2011-04-01", + "alias", + "mezzanine", + "rows", + 6L, + "index", + 4420L, + QueryRunnerTestHelper.addRowsIndexConstantMetric, + (double) (6L + 4420L + 1L) + ), + makeRow( + fullQuery, + "2011-04-01", + "alias", + "premium", + "rows", + 6L, + "index", + 4416L, + QueryRunnerTestHelper.addRowsIndexConstantMetric, + (double) (6L + 4416L + 1L) + ) + ); + TestHelper.assertExpectedObjects( expectedResults, GroupByQueryRunnerTestHelper.runQuery(factory, runner, fullQuery), @@ -4098,13 +4230,6 @@ public class GroupByQueryRunnerTest @Test public void testHavingSpec() { - - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L) - ); - GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.dataSource) .setInterval("2011-04-02/2011-04-04") @@ -4121,6 +4246,13 @@ public class GroupByQueryRunnerTest ); final GroupByQuery fullQuery = builder.build(); + + List expectedResults = Arrays.asList( + makeRow(fullQuery, "2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + makeRow(fullQuery, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + makeRow(fullQuery, "2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L) + ); + TestHelper.assertExpectedObjects( expectedResults, GroupByQueryRunnerTestHelper.runQuery(factory, runner, fullQuery), @@ -4131,12 +4263,6 @@ public class GroupByQueryRunnerTest @Test public void testDimFilterHavingSpec() { - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L) - ); - final DimFilterHavingSpec havingSpec = new DimFilterHavingSpec( new AndDimFilter( ImmutableList.of( @@ -4161,6 +4287,13 @@ public class GroupByQueryRunnerTest .setHavingSpec(havingSpec); final GroupByQuery fullQuery = builder.build(); + + List expectedResults = Arrays.asList( + makeRow(fullQuery, "2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + makeRow(fullQuery, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + makeRow(fullQuery, "2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L) + ); + TestHelper.assertExpectedObjects( expectedResults, GroupByQueryRunnerTestHelper.runQuery(factory, runner, fullQuery), @@ -4185,12 +4318,6 @@ public class GroupByQueryRunnerTest JavaScriptConfig.getEnabledInstance() ); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L) - ); - final DimFilterHavingSpec havingSpec = new DimFilterHavingSpec( new OrDimFilter( ImmutableList.of( @@ -4210,6 +4337,13 @@ public class GroupByQueryRunnerTest .setHavingSpec(havingSpec); final GroupByQuery fullQuery = builder.build(); + + List expectedResults = Arrays.asList( + makeRow(fullQuery, "2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + makeRow(fullQuery, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + makeRow(fullQuery, "2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L) + ); + TestHelper.assertExpectedObjects( expectedResults, GroupByQueryRunnerTestHelper.runQuery(factory, runner, fullQuery), @@ -4220,12 +4354,6 @@ public class GroupByQueryRunnerTest @Test public void testMergedHavingSpec() { - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L) - ); - GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.dataSource) .setInterval("2011-04-02/2011-04-04") @@ -4243,11 +4371,17 @@ public class GroupByQueryRunnerTest GroupByQuery fullQuery = builder.build(); + List expectedResults = Arrays.asList( + makeRow(fullQuery, "2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + makeRow(fullQuery, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + makeRow(fullQuery, "2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L) + ); + QueryRunner mergedRunner = factory.getToolchest().mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( @@ -4272,42 +4406,6 @@ public class GroupByQueryRunnerTest @Test public void testMergedPostAggHavingSpec() { - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "business", - "rows", - 2L, - "idx", - 217L, - "rows_times_10", - 20.0 - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "mezzanine", - "rows", - 6L, - "idx", - 4420L, - "rows_times_10", - 60.0 - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "premium", - "rows", - 6L, - "idx", - 4416L, - "rows_times_10", - 60.0 - ) - ); - GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.dataSource) .setInterval("2011-04-02/2011-04-04") @@ -4341,13 +4439,26 @@ public class GroupByQueryRunnerTest ) ); - GroupByQuery fullQuery = builder.build(); + GroupByQuery query = builder.build(); + + // Same query, but with expressions instead of arithmetic. + final GroupByQuery expressionQuery = query.withPostAggregatorSpecs( + Collections.singletonList( + new ExpressionPostAggregator("rows_times_10", "rows * 10.0", null, TestExprMacroTable.INSTANCE) + ) + ); + + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "business", "rows", 2L, "idx", 217L, "rows_times_10", 20.0), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L, "rows_times_10", 60.0), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L, "rows_times_10", 60.0) + ); QueryRunner mergedRunner = factory.getToolchest().mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( @@ -4374,23 +4485,17 @@ public class GroupByQueryRunnerTest factory.getToolchest().mergeResults( factory.getToolchest().preMergeQueryDecoration(mergedRunner) ) - ).run(QueryPlus.wrap(fullQuery), context), + ).run(QueryPlus.wrap(query)), "merged" ); - fullQuery = fullQuery.withPostAggregatorSpecs( - Collections.singletonList( - new ExpressionPostAggregator("rows_times_10", "rows * 10.0", null, TestExprMacroTable.INSTANCE) - ) - ); - TestHelper.assertExpectedObjects( expectedResults, factory.getToolchest().postMergeQueryDecoration( factory.getToolchest().mergeResults( factory.getToolchest().preMergeQueryDecoration(mergedRunner) ) - ).run(QueryPlus.wrap(fullQuery), context), + ).run(QueryPlus.wrap(expressionQuery)), "merged" ); } @@ -4398,82 +4503,6 @@ public class GroupByQueryRunnerTest @Test public void testCustomAggregatorHavingSpec() { - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "automotive", - "rows", - 1L, - "idxDouble", - 135.885094d - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "entertainment", - "rows", - 1L, - "idxDouble", - 158.747224d - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "mezzanine", - "rows", - 3L, - "idxDouble", - 2871.8866900000003d - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "premium", - "rows", - 3L, - "idxDouble", - 2900.798647d - ), - - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "automotive", - "rows", - 1L, - "idxDouble", - 147.425935d - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "entertainment", - "rows", - 1L, - "idxDouble", - 166.016049d - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "mezzanine", - "rows", - 3L, - "idxDouble", - 2448.830613d - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "premium", - "rows", - 3L, - "idxDouble", - 2506.415148d - ) - ); - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) @@ -4493,6 +4522,17 @@ public class GroupByQueryRunnerTest ) .build(); + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idxDouble", 135.885094d), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 1L, "idxDouble", 158.747224d), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idxDouble", 2871.8866900000003d), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idxDouble", 2900.798647d), + makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idxDouble", 147.425935d), + makeRow(query, "2011-04-02", "alias", "entertainment", "rows", 1L, "idxDouble", 166.016049d), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idxDouble", 2448.830613d), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idxDouble", 2506.415148d) + ); + TestHelper.assertExpectedObjects( expectedResults, GroupByQueryRunnerTestHelper.runQuery(factory, runner, query), @@ -4513,11 +4553,11 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L) + List expectedResults = Collections.singletonList( + makeRow(query, "2011-04-01", "quality", "automotive", "rows", 2L) ); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit"); } @@ -4533,8 +4573,9 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "billy", null, @@ -4543,8 +4584,9 @@ public class GroupByQueryRunnerTest "rows", 2L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "business", "rows", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-01", "billy", null, "quality", "business", "rows", 2L), + makeRow( + query, "2011-04-01", "billy", null, @@ -4553,11 +4595,12 @@ public class GroupByQueryRunnerTest "rows", 2L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "health", "rows", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "mezzanine", "rows", 6L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "news", "rows", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "premium", "rows", 6L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-01", "billy", null, "quality", "health", "rows", 2L), + makeRow(query, "2011-04-01", "billy", null, "quality", "mezzanine", "rows", 6L), + makeRow(query, "2011-04-01", "billy", null, "quality", "news", "rows", 2L), + makeRow(query, "2011-04-01", "billy", null, "quality", "premium", "rows", 6L), + makeRow( + query, "2011-04-01", "billy", null, @@ -4566,10 +4609,10 @@ public class GroupByQueryRunnerTest "rows", 2L ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "travel", "rows", 2L) + makeRow(query, "2011-04-01", "billy", null, "quality", "travel", "rows", 2L) ); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit"); } @@ -4603,30 +4646,30 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + makeRow(query, "2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + makeRow(query, "2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "sub-query"); } @@ -4666,30 +4709,30 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + makeRow(query, "2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + makeRow(query, "2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-multiple-intervals"); } @@ -4730,30 +4773,30 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + makeRow(query, "2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + makeRow(query, "2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-multiple-intervals"); } @@ -4794,13 +4837,13 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "a", "rows", 13L, "idx", 6619L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "a", "rows", 13L, "idx", 5827L) + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "a", "rows", 13L, "idx", 6619L), + makeRow(query, "2011-04-02", "alias", "a", "rows", 13L, "idx", 5827L) ); // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-extractionfn"); } @@ -4833,7 +4876,8 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = GroupByQueryRunnerTestHelper.createExpectedRows( + List expectedResults = makeRows( + query, new String[]{"__time", "rows", "idx", "indexMaxPlusTen"}, new Object[]{"2011-04-01", 9L, 2900.0, 2930.0}, new Object[]{"2011-04-02", 9L, 2505.0, 2535.0} @@ -4856,7 +4900,8 @@ public class GroupByQueryRunnerTest .build(); query = (GroupByQuery) query.withDataSource(new QueryDataSource(subquery)); - expectedResults = GroupByQueryRunnerTestHelper.createExpectedRows( + expectedResults = makeRows( + query, new String[]{"__time", "rows", "idx", "indexMaxPlusTen"}, new Object[]{"2011-04-01", 9L, 21.0, 2930.0}, new Object[]{"2011-04-02", 9L, 2.0, 2535.0} @@ -4903,16 +4948,16 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "idx1", 2900.0, "idx2", 2900.0, - "idx3", 5800.0, "idx4", 5800.0 + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "idx1", 2900.0, "idx2", 2900.0, + "idx3", 5800.0, "idx4", 5800.0 ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "idx1", 2505.0, "idx2", 2505.0, - "idx3", 5010.0, "idx4", 5010.0 + makeRow(query, "2011-04-02", "idx1", 2505.0, "idx2", 2505.0, + "idx3", 5010.0, "idx4", 5010.0 ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-multiple-aggs"); } @@ -4950,12 +4995,12 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "idx", 2900.0), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "idx", 2505.0) + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "idx", 2900.0), + makeRow(query, "2011-04-02", "idx", 2505.0) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-filter"); } @@ -4977,11 +5022,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "idx", 2505.0) + List expectedResults = Collections.singletonList( + makeRow(query, "2011-04-02", "idx", 2505.0) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-different-intervals"); } @@ -5068,7 +5113,7 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); Assert.assertFalse(results.iterator().hasNext()); } @@ -5121,8 +5166,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "automotive", @@ -5133,7 +5179,8 @@ public class GroupByQueryRunnerTest "idx", 1135L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "business", @@ -5144,7 +5191,8 @@ public class GroupByQueryRunnerTest "idx", 1118L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "entertainment", @@ -5155,7 +5203,8 @@ public class GroupByQueryRunnerTest "idx", 1158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "health", @@ -5166,7 +5215,8 @@ public class GroupByQueryRunnerTest "idx", 1120L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "mezzanine", @@ -5177,7 +5227,8 @@ public class GroupByQueryRunnerTest "idx", 3870L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "news", @@ -5188,7 +5239,8 @@ public class GroupByQueryRunnerTest "idx", 1121L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "premium", @@ -5199,7 +5251,8 @@ public class GroupByQueryRunnerTest "idx", 3900L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "technology", @@ -5210,7 +5263,8 @@ public class GroupByQueryRunnerTest "idx", 1078L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "travel", @@ -5222,7 +5276,8 @@ public class GroupByQueryRunnerTest 1119L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "automotive", @@ -5233,7 +5288,8 @@ public class GroupByQueryRunnerTest "idx", 1147L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "business", @@ -5244,7 +5300,8 @@ public class GroupByQueryRunnerTest "idx", 1112L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "entertainment", @@ -5255,7 +5312,8 @@ public class GroupByQueryRunnerTest "idx", 1166L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "health", @@ -5266,7 +5324,8 @@ public class GroupByQueryRunnerTest "idx", 1113L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "mezzanine", @@ -5277,7 +5336,8 @@ public class GroupByQueryRunnerTest "idx", 3447L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "news", @@ -5288,7 +5348,8 @@ public class GroupByQueryRunnerTest "idx", 1114L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "premium", @@ -5299,7 +5360,8 @@ public class GroupByQueryRunnerTest "idx", 3505L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "technology", @@ -5310,7 +5372,8 @@ public class GroupByQueryRunnerTest "idx", 1097L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "travel", @@ -5324,7 +5387,7 @@ public class GroupByQueryRunnerTest ); // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-postaggs"); } @@ -5350,8 +5413,16 @@ public class GroupByQueryRunnerTest ) ) .setHavingSpec( - new BaseHavingSpec() + new HavingSpec() { + private GroupByQuery query; + + @Override + public void setQuery(GroupByQuery query) + { + this.query = query; + } + @Override public byte[] getCacheKey() { @@ -5359,9 +5430,11 @@ public class GroupByQueryRunnerTest } @Override - public boolean eval(Row row) + public boolean eval(ResultRow row) { - return (row.getMetric("idx_subpostagg").floatValue() < 3800); + final String field = "idx_subpostagg"; + final int p = query.getResultRowPositionLookup().getInt(field); + return (Rows.objectToNumber(field, row.get(p)).floatValue() < 3800); } } ) @@ -5391,8 +5464,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "automotive", @@ -5403,7 +5477,8 @@ public class GroupByQueryRunnerTest "idx", 1135L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "business", @@ -5414,7 +5489,8 @@ public class GroupByQueryRunnerTest "idx", 1118L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "entertainment", @@ -5425,7 +5501,8 @@ public class GroupByQueryRunnerTest "idx", 1158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "health", @@ -5436,7 +5513,8 @@ public class GroupByQueryRunnerTest "idx", 1120L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "news", @@ -5447,7 +5525,8 @@ public class GroupByQueryRunnerTest "idx", 1121L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "technology", @@ -5458,7 +5537,8 @@ public class GroupByQueryRunnerTest "idx", 1078L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "travel", @@ -5470,7 +5550,8 @@ public class GroupByQueryRunnerTest 1119L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "automotive", @@ -5481,7 +5562,8 @@ public class GroupByQueryRunnerTest "idx", 1147L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "business", @@ -5492,7 +5574,8 @@ public class GroupByQueryRunnerTest "idx", 1112L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "entertainment", @@ -5503,7 +5586,8 @@ public class GroupByQueryRunnerTest "idx", 1166L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "health", @@ -5514,7 +5598,8 @@ public class GroupByQueryRunnerTest "idx", 1113L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "mezzanine", @@ -5525,7 +5610,8 @@ public class GroupByQueryRunnerTest "idx", 3447L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "news", @@ -5536,7 +5622,8 @@ public class GroupByQueryRunnerTest "idx", 1114L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "premium", @@ -5547,7 +5634,8 @@ public class GroupByQueryRunnerTest "idx", 3505L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "technology", @@ -5558,7 +5646,8 @@ public class GroupByQueryRunnerTest "idx", 1097L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "travel", @@ -5572,7 +5661,7 @@ public class GroupByQueryRunnerTest ); // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-postaggs"); } @@ -5618,8 +5707,10 @@ public class GroupByQueryRunnerTest ) ) .setHavingSpec( - new BaseHavingSpec() + new HavingSpec() { + private GroupByQuery query; + @Override public byte[] getCacheKey() { @@ -5627,9 +5718,17 @@ public class GroupByQueryRunnerTest } @Override - public boolean eval(Row row) + public void setQuery(GroupByQuery query) { - return (row.getMetric("idx_subpostagg").floatValue() < 3800); + this.query = query; + } + + @Override + public boolean eval(ResultRow row) + { + final String field = "idx_subpostagg"; + final int p = query.getResultRowPositionLookup().getInt(field); + return (Rows.objectToNumber(field, row.get(p)).floatValue() < 3800); } } ) @@ -5671,8 +5770,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "travel", @@ -5685,7 +5785,8 @@ public class GroupByQueryRunnerTest "js_outer_agg", 123.92274475097656 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "technology", @@ -5698,7 +5799,8 @@ public class GroupByQueryRunnerTest "js_outer_agg", 82.62254333496094 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "news", @@ -5711,7 +5813,8 @@ public class GroupByQueryRunnerTest "js_outer_agg", 125.58358001708984 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "health", @@ -5724,7 +5827,8 @@ public class GroupByQueryRunnerTest "js_outer_agg", 124.13470458984375 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "entertainment", @@ -5740,7 +5844,7 @@ public class GroupByQueryRunnerTest ); // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-multi-aggs"); } @@ -5764,10 +5868,10 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01", "rows", 837L) + List expectedResults = Collections.singletonList( + makeRow(query, "1970-01-01", "rows", 837L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-filter-agg"); } @@ -5801,18 +5905,18 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-02-01", "rows", 0L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-02-02", "rows", 0L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-02-03", "rows", 0L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-03-01", "rows", 0L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-03-02", "rows", 0L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-03-03", "rows", 0L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "rows", 13L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "rows", 0L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-03", "rows", 0L) + List expectedResults = Arrays.asList( + makeRow(query, "2011-02-01", "rows", 0L), + makeRow(query, "2011-02-02", "rows", 0L), + makeRow(query, "2011-02-03", "rows", 0L), + makeRow(query, "2011-03-01", "rows", 0L), + makeRow(query, "2011-03-02", "rows", 0L), + makeRow(query, "2011-03-03", "rows", 0L), + makeRow(query, "2011-04-01", "rows", 13L), + makeRow(query, "2011-04-02", "rows", 0L), + makeRow(query, "2011-04-03", "rows", 0L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-time-filter"); } @@ -5834,10 +5938,10 @@ public class GroupByQueryRunnerTest .overrideContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 10000)) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "count", 18L) + List expectedResults = Collections.singletonList( + makeRow(query, "2011-04-01", "count", 18L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-timeout"); } @@ -5859,10 +5963,10 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "count", 18L) + List expectedResults = Collections.singletonList( + makeRow(query, "2011-04-01", "count", 18L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-virtual"); } @@ -5889,10 +5993,10 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01", "car", QueryRunnerTestHelper.UNIQUES_9) + List expectedResults = Collections.singletonList( + makeRow(query, "1970-01-01", "car", QueryRunnerTestHelper.UNIQUES_9) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-cardinality"); } @@ -5929,10 +6033,10 @@ public class GroupByQueryRunnerTest expectedException.expectMessage("Unknown column in order clause"); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); } else { - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "count", 18L) + List expectedResults = Collections.singletonList( + makeRow(query, "2011-04-01", "count", 18L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-count-agg"); } } @@ -5963,28 +6067,28 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "js_agg", 139D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "business", "js_agg", 122D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "entertainment", "js_agg", 162D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "health", "js_agg", 124D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "mezzanine", "js_agg", 2893D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "news", "js_agg", 125D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "premium", "js_agg", 2923D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "technology", "js_agg", 82D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "js_agg", 123D), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "quality", "automotive", "js_agg", 139D), + makeRow(query, "2011-04-01", "quality", "business", "js_agg", 122D), + makeRow(query, "2011-04-01", "quality", "entertainment", "js_agg", 162D), + makeRow(query, "2011-04-01", "quality", "health", "js_agg", 124D), + makeRow(query, "2011-04-01", "quality", "mezzanine", "js_agg", 2893D), + makeRow(query, "2011-04-01", "quality", "news", "js_agg", 125D), + makeRow(query, "2011-04-01", "quality", "premium", "js_agg", 2923D), + makeRow(query, "2011-04-01", "quality", "technology", "js_agg", 82D), + makeRow(query, "2011-04-01", "quality", "travel", "js_agg", 123D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "automotive", "js_agg", 151D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "business", "js_agg", 116D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "entertainment", "js_agg", 170D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "health", "js_agg", 117D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "mezzanine", "js_agg", 2470D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "news", "js_agg", 118D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "premium", "js_agg", 2528D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "technology", "js_agg", 101D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "travel", "js_agg", 130D) + makeRow(query, "2011-04-02", "quality", "automotive", "js_agg", 151D), + makeRow(query, "2011-04-02", "quality", "business", "js_agg", 116D), + makeRow(query, "2011-04-02", "quality", "entertainment", "js_agg", 170D), + makeRow(query, "2011-04-02", "quality", "health", "js_agg", 117D), + makeRow(query, "2011-04-02", "quality", "mezzanine", "js_agg", 2470D), + makeRow(query, "2011-04-02", "quality", "news", "js_agg", 118D), + makeRow(query, "2011-04-02", "quality", "premium", "js_agg", 2528D), + makeRow(query, "2011-04-02", "quality", "technology", "js_agg", 101D), + makeRow(query, "2011-04-02", "quality", "travel", "js_agg", 130D) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-javascript"); } @@ -6014,28 +6118,28 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "js_agg", 136D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "business", "js_agg", 119D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "entertainment", "js_agg", 159D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "health", "js_agg", 121D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "mezzanine", "js_agg", 2873D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "news", "js_agg", 122D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "premium", "js_agg", 2903D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "technology", "js_agg", 79D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "js_agg", 120D), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "quality", "automotive", "js_agg", 136D), + makeRow(query, "2011-04-01", "quality", "business", "js_agg", 119D), + makeRow(query, "2011-04-01", "quality", "entertainment", "js_agg", 159D), + makeRow(query, "2011-04-01", "quality", "health", "js_agg", 121D), + makeRow(query, "2011-04-01", "quality", "mezzanine", "js_agg", 2873D), + makeRow(query, "2011-04-01", "quality", "news", "js_agg", 122D), + makeRow(query, "2011-04-01", "quality", "premium", "js_agg", 2903D), + makeRow(query, "2011-04-01", "quality", "technology", "js_agg", 79D), + makeRow(query, "2011-04-01", "quality", "travel", "js_agg", 120D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "automotive", "js_agg", 148D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "business", "js_agg", 113D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "entertainment", "js_agg", 167D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "health", "js_agg", 114D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "mezzanine", "js_agg", 2450D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "news", "js_agg", 115D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "premium", "js_agg", 2508D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "technology", "js_agg", 98D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "quality", "travel", "js_agg", 127D) + makeRow(query, "2011-04-02", "quality", "automotive", "js_agg", 148D), + makeRow(query, "2011-04-02", "quality", "business", "js_agg", 113D), + makeRow(query, "2011-04-02", "quality", "entertainment", "js_agg", 167D), + makeRow(query, "2011-04-02", "quality", "health", "js_agg", 114D), + makeRow(query, "2011-04-02", "quality", "mezzanine", "js_agg", 2450D), + makeRow(query, "2011-04-02", "quality", "news", "js_agg", 115D), + makeRow(query, "2011-04-02", "quality", "premium", "js_agg", 2508D), + makeRow(query, "2011-04-02", "quality", "technology", "js_agg", 98D), + makeRow(query, "2011-04-02", "quality", "travel", "js_agg", 127D) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-javascript"); } @@ -6066,8 +6170,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "automotive", @@ -6078,7 +6183,8 @@ public class GroupByQueryRunnerTest "uniq", 1.0002442201269182 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "business", @@ -6089,7 +6195,8 @@ public class GroupByQueryRunnerTest "uniq", 1.0002442201269182 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "entertainment", @@ -6100,7 +6207,8 @@ public class GroupByQueryRunnerTest "uniq", 1.0002442201269182 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "health", @@ -6111,7 +6219,8 @@ public class GroupByQueryRunnerTest "uniq", 1.0002442201269182 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "mezzanine", @@ -6122,7 +6231,8 @@ public class GroupByQueryRunnerTest "uniq", 1.0002442201269182 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "news", @@ -6133,7 +6243,8 @@ public class GroupByQueryRunnerTest "uniq", 1.0002442201269182 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "premium", @@ -6144,7 +6255,8 @@ public class GroupByQueryRunnerTest "uniq", 1.0002442201269182 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "technology", @@ -6155,7 +6267,8 @@ public class GroupByQueryRunnerTest "uniq", 1.0002442201269182 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "travel", @@ -6169,7 +6282,7 @@ public class GroupByQueryRunnerTest ); // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-hyperunique"); } @@ -6210,8 +6323,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Collections.singletonList( + makeRow( + query, "2011-04-01", "rows", 26L, @@ -6225,7 +6339,7 @@ public class GroupByQueryRunnerTest ); // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-hyperunique"); } @@ -6259,14 +6373,14 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.monthGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-01-01", "first", 100L, "last", 943L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-02-01", "first", 132L, "last", 1101L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-03-01", "first", 153L, "last", 1063L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "first", 135L, "last", 780L) + List expectedResults = Arrays.asList( + makeRow(query, "2011-01-01", "first", 100L, "last", 943L), + makeRow(query, "2011-02-01", "first", 132L, "last", 1101L), + makeRow(query, "2011-03-01", "first", 153L, "last", 1063L), + makeRow(query, "2011-04-01", "first", 135L, "last", 780L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subquery-firstlast"); } @@ -6283,7 +6397,12 @@ public class GroupByQueryRunnerTest GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setVirtualColumns(new ExpressionVirtualColumn("alias", "quality", ValueType.STRING, TestExprMacroTable.INSTANCE)) + .setVirtualColumns(new ExpressionVirtualColumn( + "alias", + "quality", + ValueType.STRING, + TestExprMacroTable.INSTANCE + )) .setDimensions(Lists.newArrayList( new DefaultDimensionSpec("quality", "quality"), new DefaultDimensionSpec("market", "market"), @@ -6305,8 +6424,9 @@ public class GroupByQueryRunnerTest )) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "automotive", @@ -6319,7 +6439,8 @@ public class GroupByQueryRunnerTest "idxDouble", 135.88510131835938d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "business", @@ -6332,7 +6453,8 @@ public class GroupByQueryRunnerTest "idxDouble", 118.57034 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "entertainment", @@ -6345,7 +6467,8 @@ public class GroupByQueryRunnerTest "idxDouble", 158.747224 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "health", @@ -6358,7 +6481,8 @@ public class GroupByQueryRunnerTest "idxDouble", 120.134704 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "mezzanine", @@ -6371,7 +6495,8 @@ public class GroupByQueryRunnerTest "idxDouble", 2871.8866900000003d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "news", @@ -6384,7 +6509,8 @@ public class GroupByQueryRunnerTest "idxDouble", 121.58358d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "premium", @@ -6397,7 +6523,8 @@ public class GroupByQueryRunnerTest "idxDouble", 2900.798647d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "technology", @@ -6410,7 +6537,8 @@ public class GroupByQueryRunnerTest "idxDouble", 78.622547d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "travel", @@ -6424,7 +6552,8 @@ public class GroupByQueryRunnerTest 119.922742d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "automotive", @@ -6437,7 +6566,8 @@ public class GroupByQueryRunnerTest "idxDouble", 147.42593d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "business", @@ -6450,7 +6580,8 @@ public class GroupByQueryRunnerTest "idxDouble", 112.987027d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "entertainment", @@ -6463,7 +6594,8 @@ public class GroupByQueryRunnerTest "idxDouble", 166.016049d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "health", @@ -6476,7 +6608,8 @@ public class GroupByQueryRunnerTest "idxDouble", 113.446008d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "mezzanine", @@ -6489,7 +6622,8 @@ public class GroupByQueryRunnerTest "idxDouble", 2448.830613d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "news", @@ -6502,7 +6636,8 @@ public class GroupByQueryRunnerTest "idxDouble", 114.290141d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "premium", @@ -6515,7 +6650,8 @@ public class GroupByQueryRunnerTest "idxDouble", 2506.415148d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "technology", @@ -6528,7 +6664,8 @@ public class GroupByQueryRunnerTest "idxDouble", 97.387433d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "travel", @@ -6542,7 +6679,8 @@ public class GroupByQueryRunnerTest 126.411364d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "spot", @@ -6555,7 +6693,8 @@ public class GroupByQueryRunnerTest "idx", 640L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "total_market", @@ -6568,7 +6707,8 @@ public class GroupByQueryRunnerTest "idx", 1314L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "upfront", @@ -6581,7 +6721,8 @@ public class GroupByQueryRunnerTest "idx", 1447L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "spot", @@ -6594,7 +6735,8 @@ public class GroupByQueryRunnerTest "idx", 265L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "total_market", @@ -6607,7 +6749,8 @@ public class GroupByQueryRunnerTest "idx", 1522L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "upfront", @@ -6620,7 +6763,8 @@ public class GroupByQueryRunnerTest "idx", 1234L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "spot", @@ -6634,7 +6778,8 @@ public class GroupByQueryRunnerTest 197L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "spot", @@ -6647,7 +6792,8 @@ public class GroupByQueryRunnerTest "idx", 648L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "total_market", @@ -6660,7 +6806,8 @@ public class GroupByQueryRunnerTest "idx", 1193L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "upfront", @@ -6673,7 +6820,8 @@ public class GroupByQueryRunnerTest "idx", 1144L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "spot", @@ -6686,7 +6834,8 @@ public class GroupByQueryRunnerTest "idx", 249L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "total_market", @@ -6699,7 +6848,8 @@ public class GroupByQueryRunnerTest "idx", 1321L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "upfront", @@ -6712,7 +6862,8 @@ public class GroupByQueryRunnerTest "idx", 1049L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "spot", @@ -6726,7 +6877,8 @@ public class GroupByQueryRunnerTest 223L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "idxDouble", 6626.151575318359, @@ -6737,7 +6889,8 @@ public class GroupByQueryRunnerTest "idx", 6619L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "idxDouble", 5833.209713, @@ -6750,7 +6903,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subtotal"); } @@ -6758,15 +6911,22 @@ public class GroupByQueryRunnerTest @Test public void testGroupByWithSubtotalsSpecWithRenamedDimensionAndFilter() { + // Cannot vectorize due to expression virtual columns. + cannotVectorize(); + if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { return; } - GroupByQuery query = GroupByQuery - .builder() + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setVirtualColumns(new ExpressionVirtualColumn("alias", "quality", ValueType.STRING, TestExprMacroTable.INSTANCE)) + .setVirtualColumns(new ExpressionVirtualColumn( + "alias", + "quality", + ValueType.STRING, + TestExprMacroTable.INSTANCE + )) .setDimensions(Lists.newArrayList( new DefaultDimensionSpec("quality", "quality"), new DefaultDimensionSpec("market", "market"), @@ -6788,8 +6948,9 @@ public class GroupByQueryRunnerTest )) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias_renamed", "automotive", @@ -6802,7 +6963,8 @@ public class GroupByQueryRunnerTest "idxDouble", 135.88510131835938d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias_renamed", "automotive", @@ -6816,7 +6978,8 @@ public class GroupByQueryRunnerTest 147.42593d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "rows", 1L, @@ -6827,7 +6990,8 @@ public class GroupByQueryRunnerTest "idxDouble", 135.88510131835938d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "rows", 1L, @@ -6840,7 +7004,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subtotal"); } @@ -6874,8 +7038,9 @@ public class GroupByQueryRunnerTest )) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01T00:00:00.000Z", "idxDouble", 135.885094, @@ -6888,7 +7053,8 @@ public class GroupByQueryRunnerTest "idx", 135L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "idxDouble", 118.57034, @@ -6901,7 +7067,8 @@ public class GroupByQueryRunnerTest "idx", 118L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "idxDouble", 158.747224, @@ -6914,7 +7081,8 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "idxDouble", 120.134704, @@ -6927,7 +7095,8 @@ public class GroupByQueryRunnerTest "idx", 120L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "idxDouble", 2871.8866900000003, @@ -6940,7 +7109,8 @@ public class GroupByQueryRunnerTest "idx", 2870L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "idxDouble", 121.583581, @@ -6953,7 +7123,8 @@ public class GroupByQueryRunnerTest "idx", 121L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "idxDouble", 2900.798647, @@ -6966,7 +7137,8 @@ public class GroupByQueryRunnerTest "idx", 2900L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "idxDouble", 78.622547, @@ -6979,7 +7151,8 @@ public class GroupByQueryRunnerTest "idx", 78L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "idxDouble", 119.922742, @@ -6992,7 +7165,8 @@ public class GroupByQueryRunnerTest "idx", 119L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "idxDouble", 147.425935, @@ -7005,7 +7179,8 @@ public class GroupByQueryRunnerTest "idx", 147L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "idxDouble", 112.987027, @@ -7018,7 +7193,8 @@ public class GroupByQueryRunnerTest "idx", 112L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "idxDouble", 166.016049, @@ -7031,7 +7207,8 @@ public class GroupByQueryRunnerTest "idx", 166L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "idxDouble", 113.446008, @@ -7044,7 +7221,8 @@ public class GroupByQueryRunnerTest "idx", 113L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "idxDouble", 2448.830613, @@ -7057,7 +7235,8 @@ public class GroupByQueryRunnerTest "idx", 2447L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "idxDouble", 114.290141, @@ -7070,7 +7249,8 @@ public class GroupByQueryRunnerTest "idx", 114L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "idxDouble", 2506.415148, @@ -7083,7 +7263,8 @@ public class GroupByQueryRunnerTest "idx", 2505L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "idxDouble", 97.387433, @@ -7096,7 +7277,8 @@ public class GroupByQueryRunnerTest "idx", 97L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "idxDouble", 126.411364, @@ -7110,7 +7292,8 @@ public class GroupByQueryRunnerTest 126L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "spot", @@ -7123,7 +7306,8 @@ public class GroupByQueryRunnerTest "idx", 640L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "total_market", @@ -7136,7 +7320,8 @@ public class GroupByQueryRunnerTest "idx", 1314L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "upfront", @@ -7149,7 +7334,8 @@ public class GroupByQueryRunnerTest "idx", 1447L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "spot", @@ -7162,7 +7348,8 @@ public class GroupByQueryRunnerTest "idx", 265L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "total_market", @@ -7175,7 +7362,8 @@ public class GroupByQueryRunnerTest "idx", 1522L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "upfront", @@ -7188,7 +7376,8 @@ public class GroupByQueryRunnerTest "idx", 1234L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "spot", @@ -7201,7 +7390,8 @@ public class GroupByQueryRunnerTest "idx", 197L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "spot", @@ -7214,7 +7404,8 @@ public class GroupByQueryRunnerTest "idx", 648L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "total_market", @@ -7227,7 +7418,8 @@ public class GroupByQueryRunnerTest "idx", 1193L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "upfront", @@ -7240,7 +7432,8 @@ public class GroupByQueryRunnerTest "idx", 1144L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "spot", @@ -7253,7 +7446,8 @@ public class GroupByQueryRunnerTest "idx", 249L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "total_market", @@ -7266,7 +7460,8 @@ public class GroupByQueryRunnerTest "idx", 1321L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "upfront", @@ -7279,7 +7474,8 @@ public class GroupByQueryRunnerTest "idx", 1049L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "market", "spot", @@ -7293,7 +7489,8 @@ public class GroupByQueryRunnerTest 223L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "idxDouble", 6626.151569, @@ -7304,7 +7501,8 @@ public class GroupByQueryRunnerTest "idx", 6619L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02T00:00:00.000Z", "idxDouble", 5833.209717999999, @@ -7317,7 +7515,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subtotal-long-dim"); } @@ -7353,8 +7551,9 @@ public class GroupByQueryRunnerTest .setLimit(1) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "technology", @@ -7367,7 +7566,8 @@ public class GroupByQueryRunnerTest "idxDouble", 78.622547d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "market", "spot", @@ -7380,7 +7580,8 @@ public class GroupByQueryRunnerTest "idx", 197L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01T00:00:00.000Z", "idxDouble", 6626.151575318359, @@ -7393,7 +7594,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "subtotal-order-limit"); } @@ -7414,8 +7615,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Collections.singletonList( + makeRow( + query, "2011-04-01", "rows", 26L, @@ -7426,7 +7628,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "time"); } @@ -7460,8 +7662,9 @@ public class GroupByQueryRunnerTest ) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "1970-01-01", "dayOfWeek", "Friday", @@ -7474,7 +7677,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 13337.574157714844 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Monday", @@ -7487,7 +7691,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 13675.738830566406 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Saturday", @@ -7500,7 +7705,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 13611.751281738281 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Sunday", @@ -7513,7 +7719,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 13703.541015625 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Thursday", @@ -7526,7 +7733,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 14406.127197265625 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Tuesday", @@ -7539,7 +7747,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 13317.471435546875 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Wednesday", @@ -7552,7 +7761,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 14398.368591308594 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Friday", @@ -7565,7 +7775,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 27324.8623046875 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Monday", @@ -7578,7 +7789,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 27646.58447265625 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Saturday", @@ -7591,7 +7803,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 27847.83154296875 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Sunday", @@ -7604,7 +7817,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 24818.223876953125 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Thursday", @@ -7617,7 +7831,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 28591.748901367188 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Tuesday", @@ -7630,7 +7845,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 26995.280639648438 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Wednesday", @@ -7645,7 +7861,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "time-extraction"); } @@ -7713,8 +7929,9 @@ public class GroupByQueryRunnerTest ) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "1970-01-01", "dayOfWeek", null, @@ -7727,7 +7944,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 14398.368591308594 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Friday", @@ -7740,7 +7958,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 13337.574157714844 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Monday", @@ -7753,7 +7972,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 13675.738830566406 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Saturday", @@ -7766,7 +7986,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 13611.751281738281 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Sunday", @@ -7779,7 +8000,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 13703.541015625 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Thursday", @@ -7792,7 +8014,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 14406.127197265625 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Tuesday", @@ -7805,7 +8028,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 13317.471435546875 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", null, @@ -7818,7 +8042,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 29014.5751953125 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Friday", @@ -7831,7 +8056,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 27324.8623046875 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Monday", @@ -7844,7 +8070,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 27646.58447265625 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Saturday", @@ -7857,7 +8084,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 27847.83154296875 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Sunday", @@ -7870,7 +8098,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 24818.223876953125 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Thursday", @@ -7883,7 +8112,8 @@ public class GroupByQueryRunnerTest "addRowsIndexConstant", 28591.748901367188 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01", "dayOfWeek", "Tuesday", @@ -7898,19 +8128,30 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "time-extraction"); } @Test public void testBySegmentResults() { + GroupByQuery.Builder builder = makeQueryBuilder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04") + .setDimensions(new DefaultDimensionSpec("quality", "alias")) + .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) + .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) + .setDimFilter(new SelectorDimFilter("quality", "mezzanine", null)) + .setContext(ImmutableMap.of("bySegment", true)); + final GroupByQuery fullQuery = builder.build(); + int segmentCount = 32; Result singleSegmentResult = new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new BySegmentResultValueClass<>( Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + fullQuery, "2011-04-01", "alias", "mezzanine", @@ -7928,18 +8169,9 @@ public class GroupByQueryRunnerTest for (int i = 0; i < segmentCount; i++) { bySegmentResults.add(singleSegmentResult); } - GroupByQuery.Builder builder = makeQueryBuilder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setInterval("2011-04-02/2011-04-04") - .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) - .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) - .setDimFilter(new SelectorDimFilter("quality", "mezzanine", null)) - .overrideContext(ImmutableMap.of("bySegment", true)); - final GroupByQuery fullQuery = builder.build(); QueryToolChest toolChest = factory.getToolchest(); - List> singleSegmentRunners = new ArrayList<>(); + List> singleSegmentRunners = new ArrayList<>(); for (int i = 0; i < segmentCount; i++) { singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner)); } @@ -7963,15 +8195,31 @@ public class GroupByQueryRunnerTest @Test public void testBySegmentResultsUnOptimizedDimextraction() { - // Cannot vectorize due to extraction dimension spec. - cannotVectorize(); + GroupByQuery.Builder builder = makeQueryBuilder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04").setDimensions(new ExtractionDimensionSpec( + "quality", + "alias", + new LookupExtractionFn( + new MapLookupExtractor(ImmutableMap.of("mezzanine", "mezzanine0"), false), + false, + null, + false, + false + ) + )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) + .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) + .setDimFilter(new SelectorDimFilter("quality", "mezzanine", null)) + .setContext(ImmutableMap.of("bySegment", true)); + final GroupByQuery fullQuery = builder.build(); int segmentCount = 32; Result singleSegmentResult = new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new BySegmentResultValueClass<>( Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + fullQuery, "2011-04-01", "alias", "mezzanine0", @@ -7989,26 +8237,9 @@ public class GroupByQueryRunnerTest for (int i = 0; i < segmentCount; i++) { bySegmentResults.add(singleSegmentResult); } - GroupByQuery.Builder builder = makeQueryBuilder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setInterval("2011-04-02/2011-04-04").setDimensions(new ExtractionDimensionSpec( - "quality", - "alias", - new LookupExtractionFn( - new MapLookupExtractor(ImmutableMap.of("mezzanine", "mezzanine0"), false), - false, - null, - false, - false - ) - )).setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) - .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) - .setDimFilter(new SelectorDimFilter("quality", "mezzanine", null)) - .overrideContext(ImmutableMap.of("bySegment", true)); - final GroupByQuery fullQuery = builder.build(); QueryToolChest toolChest = factory.getToolchest(); - List> singleSegmentRunners = new ArrayList<>(); + List> singleSegmentRunners = new ArrayList<>(); for (int i = 0; i < segmentCount; i++) { singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner)); } @@ -8031,29 +8262,6 @@ public class GroupByQueryRunnerTest @Test public void testBySegmentResultsOptimizedDimextraction() { - int segmentCount = 32; - Result singleSegmentResult = new Result<>( - DateTimes.of("2011-01-12T00:00:00.000Z"), - new BySegmentResultValueClass<>( - Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "mezzanine0", - "rows", - 6L, - "idx", - 4420L - ) - ), - QueryRunnerTestHelper.segmentId.toString(), - Intervals.of("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z") - ) - ); - List bySegmentResults = new ArrayList<>(); - for (int i = 0; i < segmentCount; i++) { - bySegmentResults.add(singleSegmentResult); - } GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.dataSource) .setInterval("2011-04-02/2011-04-04").setDimensions(new ExtractionDimensionSpec( @@ -8071,9 +8279,34 @@ public class GroupByQueryRunnerTest .setDimFilter(new SelectorDimFilter("quality", "mezzanine", null)) .overrideContext(ImmutableMap.of("bySegment", true)); final GroupByQuery fullQuery = builder.build(); + + int segmentCount = 32; + Result> singleSegmentResult = new Result<>( + DateTimes.of("2011-01-12T00:00:00.000Z"), + new BySegmentResultValueClass<>( + Collections.singletonList( + makeRow( + fullQuery, + "2011-04-01", + "alias", + "mezzanine0", + "rows", + 6L, + "idx", + 4420L + ) + ), + QueryRunnerTestHelper.segmentId.toString(), + Intervals.of("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z") + ) + ); + List bySegmentResults = new ArrayList<>(); + for (int i = 0; i < segmentCount; i++) { + bySegmentResults.add(singleSegmentResult); + } QueryToolChest toolChest = factory.getToolchest(); - List> singleSegmentRunners = new ArrayList<>(); + List> singleSegmentRunners = new ArrayList<>(); for (int i = 0; i < segmentCount; i++) { singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner)); } @@ -8127,29 +8360,29 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .setDimFilter(new OrDimFilter(dimFilters)) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + makeRow(query, "2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + makeRow(query, "2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + makeRow(query, "2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "dim-extraction"); } @@ -8182,24 +8415,24 @@ public class GroupByQueryRunnerTest .setDimFilter(new ExtractionDimFilter("quality", "", lookupExtractionFn, null)) .build(); - List expectedResults; + List expectedResults; if (NullHandling.replaceWithDefault()) { expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L) + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L) ); } else { // Only empty string should match, nulls will not match expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L) + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L) ); } - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "dim-extraction"); } @@ -8214,13 +8447,18 @@ public class GroupByQueryRunnerTest .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("idx", "index")) + .setAggregatorSpecs( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) .setGranularity(QueryRunnerTestHelper.dayGran) - .setDimFilter(new ExtractionDimFilter("quality", "NOT_THERE", lookupExtractionFn, null)) + .setDimFilter( + new ExtractionDimFilter("quality", "NOT_THERE", lookupExtractionFn, null) + ) .build(); - List expectedResults = Collections.emptyList(); + List expectedResults = Collections.emptyList(); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "dim-extraction"); } @@ -8259,13 +8497,13 @@ public class GroupByQueryRunnerTest ) ).build(); - List expectedResults = Arrays + List expectedResults = Arrays .asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", null, "rows", 13L, "idx", 6619L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", null, "rows", 13L, "idx", 5827L) + makeRow(query, "2011-04-01", "alias", null, "rows", 13L, "idx", 6619L), + makeRow(query, "2011-04-02", "alias", null, "rows", 13L, "idx", 5827L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "dim-extraction"); } @@ -8290,14 +8528,18 @@ public class GroupByQueryRunnerTest .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs( - new FilteredAggregatorFactory(QueryRunnerTestHelper.rowsCount, filter), - new FilteredAggregatorFactory(new LongSumAggregatorFactory("idx", "index"), filter) - ) + .setAggregatorSpecs(new FilteredAggregatorFactory( + QueryRunnerTestHelper.rowsCount, + filter + ), new FilteredAggregatorFactory( + new LongSumAggregatorFactory("idx", "index"), + filter + )) .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "automotive", @@ -8306,7 +8548,8 @@ public class GroupByQueryRunnerTest "idx", NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "business", @@ -8315,7 +8558,8 @@ public class GroupByQueryRunnerTest "idx", NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "entertainment", @@ -8324,7 +8568,8 @@ public class GroupByQueryRunnerTest "idx", NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "health", @@ -8333,9 +8578,10 @@ public class GroupByQueryRunnerTest "idx", NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow( + query, "2011-04-01", "alias", "premium", @@ -8344,7 +8590,8 @@ public class GroupByQueryRunnerTest "idx", NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "technology", @@ -8353,7 +8600,8 @@ public class GroupByQueryRunnerTest "idx", NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "travel", @@ -8363,7 +8611,8 @@ public class GroupByQueryRunnerTest NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "automotive", @@ -8372,7 +8621,8 @@ public class GroupByQueryRunnerTest "idx", NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "business", @@ -8381,7 +8631,8 @@ public class GroupByQueryRunnerTest "idx", NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "entertainment", @@ -8390,7 +8641,8 @@ public class GroupByQueryRunnerTest "idx", NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "health", @@ -8399,9 +8651,10 @@ public class GroupByQueryRunnerTest "idx", NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + makeRow( + query, "2011-04-02", "alias", "premium", @@ -8410,7 +8663,8 @@ public class GroupByQueryRunnerTest "idx", NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "technology", @@ -8419,7 +8673,8 @@ public class GroupByQueryRunnerTest "idx", NullHandling.defaultLongValue() ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "travel", @@ -8430,7 +8685,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "agg-filter"); } @@ -8462,14 +8717,14 @@ public class GroupByQueryRunnerTest ) ) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L) + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "extraction-dim-filter"); } @@ -8498,43 +8753,19 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .setDimFilter(new ExtractionDimFilter("null_column", "EMPTY", lookupExtractionFn, null)) .build(); - List expectedResults = Arrays + List expectedResults = Arrays .asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", null, "rows", 13L, "idx", 6619L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", null, "rows", 13L, "idx", 5827L) + makeRow(query, "2011-04-01", "alias", null, "rows", 13L, "idx", 6619L), + makeRow(query, "2011-04-02", "alias", null, "rows", 13L, "idx", 5827L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "extraction-dim-filter"); } @Test public void testBySegmentResultsWithAllFiltersWithExtractionFns() { - int segmentCount = 32; - Result singleSegmentResult = new Result<>( - DateTimes.of("2011-01-12T00:00:00.000Z"), - new BySegmentResultValueClass<>( - Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "mezzanine", - "rows", - 6L, - "idx", - 4420L - ) - ), - QueryRunnerTestHelper.segmentId.toString(), - Intervals.of("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z") - ) - ); - List bySegmentResults = new ArrayList<>(); - for (int i = 0; i < segmentCount; i++) { - bySegmentResults.add(singleSegmentResult); - } - String extractionJsFn = "function(str) { return 'super-' + str; }"; String jsFn = "function(x) { return(x === 'super-mezzanine') }"; ExtractionFn extractionFn = new JavaScriptExtractionFn( @@ -8578,9 +8809,35 @@ public class GroupByQueryRunnerTest .setDimFilter(superFilter) .overrideContext(ImmutableMap.of("bySegment", true)); final GroupByQuery fullQuery = builder.build(); + + int segmentCount = 32; + Result singleSegmentResult = new Result<>( + DateTimes.of("2011-01-12T00:00:00.000Z"), + new BySegmentResultValueClass<>( + Collections.singletonList( + makeRow( + fullQuery, + "2011-04-01", + "alias", + "mezzanine", + "rows", + 6L, + "idx", + 4420L + ) + ), + QueryRunnerTestHelper.segmentId.toString(), + Intervals.of("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z") + ) + ); + List bySegmentResults = new ArrayList<>(); + for (int i = 0; i < segmentCount; i++) { + bySegmentResults.add(singleSegmentResult); + } + QueryToolChest toolChest = factory.getToolchest(); - List> singleSegmentRunners = new ArrayList<>(); + List> singleSegmentRunners = new ArrayList<>(); for (int i = 0; i < segmentCount; i++) { singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner)); } @@ -8637,12 +8894,12 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .setDimFilter(superFilter).build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", null, "rows", 13L, "idx", 6619L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", null, "rows", 13L, "idx", 5827L) + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", null, "rows", 13L, "idx", 6619L), + makeRow(query, "2011-04-02", "alias", null, "rows", 13L, "idx", 5827L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "extraction"); } @@ -8671,8 +8928,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "spot", @@ -8681,7 +8939,8 @@ public class GroupByQueryRunnerTest "numVals", 1.0002442201269182d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "total_market", @@ -8690,7 +8949,8 @@ public class GroupByQueryRunnerTest "numVals", 1.0002442201269182d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "upfront", @@ -8699,7 +8959,8 @@ public class GroupByQueryRunnerTest "numVals", 1.0002442201269182d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "spot", @@ -8708,7 +8969,8 @@ public class GroupByQueryRunnerTest "numVals", 1.0002442201269182d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "total_market", @@ -8717,7 +8979,8 @@ public class GroupByQueryRunnerTest "numVals", 1.0002442201269182d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "upfront", @@ -8728,7 +8991,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "cardinality-agg"); } @@ -8753,8 +9016,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "spot", @@ -8763,7 +9027,8 @@ public class GroupByQueryRunnerTest "numVals", 8.015665809687173d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "total_market", @@ -8772,7 +9037,8 @@ public class GroupByQueryRunnerTest "numVals", 2.000977198748901d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", "upfront", @@ -8781,7 +9047,8 @@ public class GroupByQueryRunnerTest "numVals", 2.000977198748901d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "spot", @@ -8790,7 +9057,8 @@ public class GroupByQueryRunnerTest "numVals", 9.019833517963864d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "total_market", @@ -8799,7 +9067,8 @@ public class GroupByQueryRunnerTest "numVals", 2.000977198748901d ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "upfront", @@ -8810,7 +9079,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "cardinality-agg"); } @@ -8836,19 +9105,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - Assert.assertEquals( - Functions.>identity(), - query.getLimitSpec().build( - query.getDimensions(), - query.getAggregatorSpecs(), - query.getPostAggregatorSpecs(), - query.getGranularity(), - query.getContextSortByDimsFirst() - ) - ); + Assert.assertEquals(Functions.>identity(), query.getLimitSpec().build(query)); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "ql_alias", 1200L, @@ -8857,7 +9118,8 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "ql_alias", 1200L, @@ -8867,7 +9129,7 @@ public class GroupByQueryRunnerTest 166L ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "long"); } @@ -8893,19 +9155,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - Assert.assertNotEquals( - Functions.>identity(), - query.getLimitSpec().build( - query.getDimensions(), - query.getAggregatorSpecs(), - query.getPostAggregatorSpecs(), - query.getGranularity(), - query.getContextSortByDimsFirst() - ) - ); + Assert.assertNotEquals(Functions.>identity(), query.getLimitSpec().build(query)); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "ql_alias", 1700L, @@ -8914,7 +9168,8 @@ public class GroupByQueryRunnerTest "idx", 175L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "ql_alias", 1200L, @@ -8924,7 +9179,7 @@ public class GroupByQueryRunnerTest 324L ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "long"); } @@ -8951,8 +9206,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "ql_alias", "super-1200", @@ -8961,7 +9217,8 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "ql_alias", "super-1200", @@ -8971,7 +9228,7 @@ public class GroupByQueryRunnerTest 166L ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "long-extraction"); } @@ -8992,8 +9249,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "time_alias", 1301616000000L, @@ -9002,7 +9260,8 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "time_alias", 1301702400000L, @@ -9012,7 +9271,7 @@ public class GroupByQueryRunnerTest 166L ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "long"); } @@ -9034,8 +9293,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "time_alias", "super-1301616000000", @@ -9044,7 +9304,8 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "time_alias", "super-1301702400000", @@ -9054,7 +9315,7 @@ public class GroupByQueryRunnerTest 166L ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "long-extraction"); } @@ -9080,19 +9341,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - Assert.assertEquals( - Functions.>identity(), - query.getLimitSpec().build( - query.getDimensions(), - query.getAggregatorSpecs(), - query.getPostAggregatorSpecs(), - query.getGranularity(), - query.getContextSortByDimsFirst() - ) - ); + Assert.assertEquals(Functions.>identity(), query.getLimitSpec().build(query)); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "index_alias", 158.747224f, @@ -9101,7 +9354,8 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "index_alias", 166.016049f, @@ -9112,7 +9366,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "float"); } @@ -9138,19 +9392,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - Assert.assertNotEquals( - Functions.>identity(), - query.getLimitSpec().build( - query.getDimensions(), - query.getAggregatorSpecs(), - query.getPostAggregatorSpecs(), - query.getGranularity(), - query.getContextSortByDimsFirst() - ) - ); + Assert.assertNotEquals(Functions.>identity(), query.getLimitSpec().build(query)); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "qf_alias", 17000.0f, @@ -9159,7 +9405,8 @@ public class GroupByQueryRunnerTest "idx", 175L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "qf_alias", 12000.0f, @@ -9169,7 +9416,7 @@ public class GroupByQueryRunnerTest 324L ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "float"); } @@ -9195,19 +9442,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - Assert.assertNotEquals( - Functions.>identity(), - query.getLimitSpec().build( - query.getDimensions(), - query.getAggregatorSpecs(), - query.getPostAggregatorSpecs(), - query.getGranularity(), - query.getContextSortByDimsFirst() - ) - ); + Assert.assertNotEquals(Functions.>identity(), query.getLimitSpec().build(query)); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", 17000.0d, @@ -9216,7 +9455,8 @@ public class GroupByQueryRunnerTest "idx", 175L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "alias", 12000.0d, @@ -9226,7 +9466,7 @@ public class GroupByQueryRunnerTest 324L ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "double"); } @@ -9253,10 +9493,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults; + List expectedResults; expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "index_alias", "super-158.747224", @@ -9265,7 +9506,8 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "index_alias", "super-166.016049", @@ -9276,7 +9518,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "float"); } @@ -9321,8 +9563,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Collections.singletonList( + makeRow( + query, "2011-04-01", "alias", "total_market", "time_alias", 1301616000000L, @@ -9332,7 +9575,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "havingspec-long-float"); } @@ -9356,8 +9599,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "ql_alias", "1200", @@ -9368,7 +9612,8 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "ql_alias", "1200", @@ -9380,7 +9625,7 @@ public class GroupByQueryRunnerTest 166L ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "long-float"); } @@ -9417,8 +9662,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + outerQuery, "2011-04-01", "time_alias2", 1301616000000L, "ql_alias_long", 1200L, @@ -9426,7 +9672,8 @@ public class GroupByQueryRunnerTest "ql_alias_float", 1200.0, "count", 1L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + outerQuery, "2011-04-01", "time_alias2", 1301702400000L, "ql_alias_long", 1200L, @@ -9436,7 +9683,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); TestHelper.assertExpectedObjects(expectedResults, results, "numeric-string"); } @@ -9473,16 +9720,18 @@ public class GroupByQueryRunnerTest .addOrderByColumn("ql") .build(); - List expectedResults; + List expectedResults; // "entertainment" rows are excluded by the decorated specs, they become empty rows expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "ql", NullHandling.defaultLongValue(), "qf", NullHandling.defaultDoubleValue(), "count", 2L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "ql", 170000L, "qf", 170000.0, @@ -9490,7 +9739,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "numeric-string"); } @@ -9524,16 +9773,18 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs(new CountAggregatorFactory("count")) .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults; + List expectedResults; if (NullHandling.replaceWithDefault()) { expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "ql", 0L, "qf", 0.0, "count", 2L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "ql", 1700L, "qf", 17000.0, @@ -9542,13 +9793,15 @@ public class GroupByQueryRunnerTest ); } else { expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "ql", null, "qf", null, "count", 2L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-01", "ql", 1700L, "qf", 17000.0, @@ -9557,7 +9810,7 @@ public class GroupByQueryRunnerTest ); } - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "numeric"); } @@ -9618,17 +9871,22 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Collections.singletonList( + makeRow( + outerQuery, "2011-04-01", - "quallong", 1200L, - "qualfloat", 12000.0, - "ql_alias_sum", 2400L, - "qf_alias_sum", 24000.0 + "quallong", + 1200L, + "qualfloat", + 12000.0, + "ql_alias_sum", + 2400L, + "qf_alias_sum", + 24000.0 ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); TestHelper.assertExpectedObjects(expectedResults, results, "numerics"); } @@ -9665,43 +9923,49 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + outerQuery, "2011-04-01", "market", "spot", "time_alias2", 1301616000000L, "time_alias_max", 1301616000000L, "index_alias_max", 158.74722290039062 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + outerQuery, "2011-04-01", "market", "spot", "time_alias2", 1301702400000L, "time_alias_max", 1301702400000L, "index_alias_max", 166.01605224609375 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + outerQuery, "2011-04-01", "market", "total_market", "time_alias2", 1301616000000L, "time_alias_max", 1301616000000L, "index_alias_max", 1522.043701171875 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + outerQuery, "2011-04-01", "market", "total_market", "time_alias2", 1301702400000L, "time_alias_max", 1301702400000L, "index_alias_max", 1321.375 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + outerQuery, "2011-04-01", "market", "upfront", "time_alias2", 1301616000000L, "time_alias_max", 1301616000000L, "index_alias_max", 1447.3411865234375 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + outerQuery, "2011-04-01", "market", "upfront", "time_alias2", 1301702400000L, @@ -9710,7 +9974,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); TestHelper.assertExpectedObjects(expectedResults, results, "numerics"); } @@ -9741,8 +10005,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", 13L, @@ -9751,7 +10016,8 @@ public class GroupByQueryRunnerTest "idx", 158L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", 13L, @@ -9762,7 +10028,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "string-long"); } @@ -9803,8 +10069,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "2011-04-01", "alias", "technology", "rows", 1L, @@ -9815,7 +10082,8 @@ public class GroupByQueryRunnerTest "qfLong", 17000L, "qfJs", 17000.0 ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "2011-04-02", "alias", "technology", "rows", 1L, @@ -9828,7 +10096,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "numeric-dims"); } @@ -9872,16 +10140,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", "technology", - "qf_outer", 17000.0f, - "rows", 2L - ) + List expectedResults = Collections.singletonList( + makeRow(outerQuery, "2011-04-01", "alias", "technology", "qf_outer", 17000.0f, "rows", 2L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); TestHelper.assertExpectedObjects(expectedResults, results, "extraction-fn"); } @@ -9925,16 +10188,11 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.allGran) .build(); - List expectedResults = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", "technology", - "time_week", 1301270400000L, - "rows", 2L - ) + List expectedResults = Collections.singletonList( + makeRow(outerQuery, "2011-04-01", "alias", "technology", "time_week", 1301270400000L, "rows", 2L) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); TestHelper.assertExpectedObjects(expectedResults, results, "extraction-fn"); } @@ -9963,15 +10221,17 @@ public class GroupByQueryRunnerTest .overrideContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, true)) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( + List expectedResults = Arrays.asList( + makeRow( + query, "1970-01-01T00:00:00.000Z", "marketalias", "upfront", "rows", 186L ), - GroupByQueryRunnerTestHelper.createExpectedRow( + makeRow( + query, "1970-01-01T00:00:00.000Z", "marketalias", "total_market", @@ -9980,7 +10240,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "order-limit"); } @@ -10007,16 +10267,16 @@ public class GroupByQueryRunnerTest final GroupByQuery allGranQuery = builder.build(); QueryRunner mergedRunner = factory.getToolchest().mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) ); @@ -10034,12 +10294,13 @@ public class GroupByQueryRunnerTest } } ); - List allGranExpectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 2L, "idx", 177L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 6L, "idx", 4416L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 2L, "idx", 221L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L) + Map context = new HashMap<>(); + List allGranExpectedResults = Arrays.asList( + makeRow(allGranQuery, "2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L), + makeRow(allGranQuery, "2011-04-02", "alias", "technology", "rows", 2L, "idx", 177L), + makeRow(allGranQuery, "2011-04-02", "alias", "premium", "rows", 6L, "idx", 4416L), + makeRow(allGranQuery, "2011-04-02", "alias", "news", "rows", 2L, "idx", 221L), + makeRow(allGranQuery, "2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L) ); TestHelper.assertExpectedObjects( @@ -10072,16 +10333,16 @@ public class GroupByQueryRunnerTest final GroupByQuery allGranQuery = builder.build(); QueryRunner mergedRunner = factory.getToolchest().mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) ); @@ -10100,15 +10361,15 @@ public class GroupByQueryRunnerTest } ); - List allGranExpectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 6L, "idx", 4416L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 2L, "idx", 319L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) + List allGranExpectedResults = Arrays.asList( + makeRow(allGranQuery, "2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + makeRow(allGranQuery, "2011-04-02", "alias", "premium", "rows", 6L, "idx", 4416L), + makeRow(allGranQuery, "2011-04-02", "alias", "entertainment", "rows", 2L, "idx", 319L), + makeRow(allGranQuery, "2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), + makeRow(allGranQuery, "2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) ); - Iterable results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList(); + Iterable results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList(); TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); } @@ -10140,16 +10401,16 @@ public class GroupByQueryRunnerTest final GroupByQuery allGranQuery = builder.build(); QueryRunner mergedRunner = factory.getToolchest().mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) ); @@ -10168,65 +10429,15 @@ public class GroupByQueryRunnerTest } ); - List allGranExpectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "travel", - "market", - "spot", - "rows", - 2L, - "idx", - 243L - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "technology", - "market", - "spot", - "rows", - 2L, - "idx", - 177L - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "premium", - "market", - "upfront", - "rows", - 2L, - "idx", - 1817L - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "premium", - "market", - "total_market", - "rows", - 2L, - "idx", - 2342L - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "premium", - "market", - "spot", - "rows", - 2L, - "idx", - 257L - ) + List allGranExpectedResults = Arrays.asList( + makeRow(allGranQuery, "2011-04-02", "alias", "travel", "market", "spot", "rows", 2L, "idx", 243L), + makeRow(allGranQuery, "2011-04-02", "alias", "technology", "market", "spot", "rows", 2L, "idx", 177L), + makeRow(allGranQuery, "2011-04-02", "alias", "premium", "market", "upfront", "rows", 2L, "idx", 1817L), + makeRow(allGranQuery, "2011-04-02", "alias", "premium", "market", "total_market", "rows", 2L, "idx", 2342L), + makeRow(allGranQuery, "2011-04-02", "alias", "premium", "market", "spot", "rows", 2L, "idx", 257L) ); - Iterable results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList(); + Iterable results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList(); TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); } @@ -10271,16 +10482,16 @@ public class GroupByQueryRunnerTest final GroupByQuery allGranQuery = builder.build(); QueryRunner mergedRunner = factory.getToolchest().mergeResults( - new QueryRunner() + new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) ); @@ -10299,65 +10510,15 @@ public class GroupByQueryRunnerTest } ); - List allGranExpectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "travel", - "market", - "spot", - "rows", - 2L, - "idx", - 243L - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "technology", - "market", - "spot", - "rows", - 2L, - "idx", - 177L - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "premium", - "market", - "total_market", - "rows", - 2L, - "idx", - 2342L - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "premium", - "market", - "upfront", - "rows", - 2L, - "idx", - 1817L - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "premium", - "market", - "spot", - "rows", - 2L, - "idx", - 257L - ) + List allGranExpectedResults = Arrays.asList( + makeRow(allGranQuery, "2011-04-02", "alias", "travel", "market", "spot", "rows", 2L, "idx", 243L), + makeRow(allGranQuery, "2011-04-02", "alias", "technology", "market", "spot", "rows", 2L, "idx", 177L), + makeRow(allGranQuery, "2011-04-02", "alias", "premium", "market", "total_market", "rows", 2L, "idx", 2342L), + makeRow(allGranQuery, "2011-04-02", "alias", "premium", "market", "upfront", "rows", 2L, "idx", 1817L), + makeRow(allGranQuery, "2011-04-02", "alias", "premium", "market", "spot", "rows", 2L, "idx", 257L) ); - Iterable results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList(); + Iterable results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList(); TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); } @@ -10425,7 +10586,7 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); Assert.assertFalse(results.iterator().hasNext()); } @@ -10480,24 +10641,24 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L) + makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L), + makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L) ); // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "limit-pushdown"); } @@ -10547,29 +10708,9 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "alias", - "technology", - "qualityLen", - 10L, - "rows", - 2L, - "idx", - 156L - ), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-02", - "alias", - "technology", - "qualityLen", - 10L, - "rows", - 2L, - "idx", - 194L - ) + List expectedResults = Arrays.asList( + makeRow(query, "2011-04-01", "alias", "technology", "qualityLen", 10L, "rows", 2L, "idx", 156L), + makeRow(query, "2011-04-02", "alias", "technology", "qualityLen", 10L, "rows", 2L, "idx", 194L) ); ChainedExecutionQueryRunner ceqr = new ChainedExecutionQueryRunner( @@ -10577,15 +10718,34 @@ public class GroupByQueryRunnerTest (query1, future) -> { return; }, - ImmutableList.>of(runner, runner) + ImmutableList.>of(runner, runner) ); - QueryRunner mergingRunner = factory.mergeRunners(Execs.directExecutor(), ImmutableList.of(ceqr)); + QueryRunner mergingRunner = factory.mergeRunners(Execs.directExecutor(), ImmutableList.of(ceqr)); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, mergingRunner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, mergingRunner, query); TestHelper.assertExpectedObjects(expectedResults, results, "type-conversion"); } + private static ResultRow makeRow(final GroupByQuery query, final String timestamp, final Object... vals) + { + return GroupByQueryRunnerTestHelper.createExpectedRow(query, timestamp, vals); + } + + private static ResultRow makeRow(final GroupByQuery query, final DateTime timestamp, final Object... vals) + { + return GroupByQueryRunnerTestHelper.createExpectedRow(query, timestamp, vals); + } + + private static List makeRows( + final GroupByQuery query, + final String[] columnNames, + final Object[]... values + ) + { + return GroupByQueryRunnerTestHelper.createExpectedRows(query, columnNames, values); + } + /** * Use this method instead of makeQueryBuilder() to make sure the context is set properly. Also, avoid * setContext in tests. Only use overrideContext. diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTestHelper.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTestHelper.java index 277c28d5dea..dd152533454 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTestHelper.java @@ -20,9 +20,6 @@ package org.apache.druid.query.groupby; import com.google.common.base.Preconditions; -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.guava.Sequence; import org.apache.druid.query.FinalizeResultsQueryRunner; @@ -37,11 +34,10 @@ import org.joda.time.chrono.ISOChronology; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; import java.util.List; -import java.util.Map; /** + * */ public class GroupByQueryRunnerTestHelper { @@ -58,38 +54,59 @@ public class GroupByQueryRunnerTestHelper return queryResult.toList(); } - public static Row createExpectedRow(final String timestamp, Object... vals) + public static ResultRow createExpectedRow(final GroupByQuery query, final String timestamp, Object... vals) { - return createExpectedRow(DateTimes.of(timestamp), vals); + return createExpectedRow(query, DateTimes.of(timestamp), vals); } - public static Row createExpectedRow(final DateTime timestamp, Object... vals) + /** + * Create a {@link ResultRow} for a given {@link GroupByQuery}. The size of the row will include space + * for postaggregations. + */ + public static ResultRow createExpectedRow(final GroupByQuery query, final DateTime timestamp, Object... vals) { Preconditions.checkArgument(vals.length % 2 == 0); - Map theVals = new HashMap<>(); - for (int i = 0; i < vals.length; i += 2) { - theVals.put(vals[i].toString(), vals[i + 1]); + final ResultRow row = ResultRow.create(query.getResultRowSizeWithPostAggregators()); + + if (query.getResultRowHasTimestamp()) { + row.set(0, timestamp.getMillis()); } - return new MapBasedRow(timestamp, theVals); + for (int i = 0; i < vals.length; i += 2) { + final int position = query.getResultRowPositionLookup().getInt(vals[i].toString()); + row.set(position, vals[i + 1]); + } + + return row; } - public static List createExpectedRows(String[] columnNames, Object[]... values) + /** + * Create a collection of {@link ResultRow} objects for a given {@link GroupByQuery}. The size of the rows will + * include space for postaggregations. + */ + public static List createExpectedRows( + final GroupByQuery query, + final String[] columnNames, + final Object[]... values + ) { - int timeIndex = Arrays.asList(columnNames).indexOf(ColumnHolder.TIME_COLUMN_NAME); + final int timeIndex = Arrays.asList(columnNames).indexOf(ColumnHolder.TIME_COLUMN_NAME); Preconditions.checkArgument(timeIndex >= 0); - List expected = new ArrayList<>(); + List expected = new ArrayList<>(); for (Object[] value : values) { Preconditions.checkArgument(value.length == columnNames.length); - Map theVals = Maps.newHashMapWithExpectedSize(value.length); + ResultRow row = ResultRow.create(query.getResultRowSizeWithPostAggregators()); for (int i = 0; i < columnNames.length; i++) { if (i != timeIndex) { - theVals.put(columnNames[i], value[i]); + final int position = query.getResultRowPositionLookup().getInt(columnNames[i]); + row.set(position, value[i]); + } else if (query.getResultRowHasTimestamp()) { + row.set(0, new DateTime(value[i], ISOChronology.getInstanceUTC()).getMillis()); } } - expected.add(new MapBasedRow(new DateTime(value[timeIndex], ISOChronology.getInstanceUTC()), theVals)); + expected.add(row); } return expected; } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java index 5fd314da088..9f0ce6d7350 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java @@ -21,10 +21,7 @@ package org.apache.druid.query.groupby; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Ordering; -import org.apache.druid.data.input.MapBasedRow; -import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.BaseQuery; @@ -93,10 +90,10 @@ public class GroupByQueryTest .addDimension(new DefaultDimensionSpec("baz", "baz", ValueType.STRING)) .build(); - final Ordering rowOrdering = query.getRowOrdering(false); + final Ordering rowOrdering = query.getRowOrdering(false); final int compare = rowOrdering.compare( - new MapBasedRow(0L, ImmutableMap.of("foo", 1, "bar", 1f, "baz", "a")), - new MapBasedRow(0L, ImmutableMap.of("foo", 1L, "bar", 1d, "baz", "b")) + ResultRow.of(1, 1f, "a"), + ResultRow.of(1L, 1d, "b") ); Assert.assertEquals(-1, compare); } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index fba0343aea3..f2cf1c41b05 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -22,7 +22,6 @@ package org.apache.druid.query.groupby; import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.MapBasedRow; -import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; @@ -83,14 +82,14 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest final List constructors = new ArrayList<>(); - for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { final QueryRunner modifiedRunner = new QueryRunner() { @Override public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { TimeseriesQuery tsQuery = (TimeseriesQuery) queryPlus.getQuery(); - QueryRunner newRunner = factory.mergeRunners( + QueryRunner newRunner = factory.mergeRunners( Execs.directExecutor(), ImmutableList.of(runner) ); QueryToolChest toolChest = factory.getToolchest(); @@ -114,15 +113,16 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest return Sequences.map( newRunner.run(queryPlus.withQuery(newQuery), responseContext), - new Function>() + new Function>() { @Override - public Result apply(final Row input) + public Result apply(final ResultRow input) { - MapBasedRow row = (MapBasedRow) input; + final MapBasedRow mapBasedRow = input.toMapBasedRow(newQuery); return new Result<>( - row.getTimestamp(), new TimeseriesResultValue(row.getEvent()) + mapBasedRow.getTimestamp(), + new TimeseriesResultValue(mapBasedRow.getEvent()) ); } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java index 735f2d27f77..129e14c31cd 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java @@ -35,7 +35,6 @@ 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.MapBasedInputRow; -import org.apache.druid.data.input.Row; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; @@ -106,8 +105,8 @@ public class NestedQueryPushDownTest private static final IndexMergerV9 INDEX_MERGER_V9; public static final ObjectMapper JSON_MAPPER; private File tmpDir; - private QueryRunnerFactory groupByFactory; - private QueryRunnerFactory groupByFactory2; + private QueryRunnerFactory groupByFactory; + private QueryRunnerFactory groupByFactory2; private List incrementalIndices = new ArrayList<>(); private List groupByIndices = new ArrayList<>(); private ExecutorService executorService; @@ -418,15 +417,17 @@ public class NestedQueryPushDownTest .setGranularity(Granularities.ALL) .build(); - Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty()); - List results = queryResult.toList(); + Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery); + List results = queryResult.toList(); - Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + nestedQuery, "2017-07-14T02:40:00.000Z", "dimB", "sour", "totalSum", 2000L ); - Row expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + nestedQuery, "2017-07-14T02:40:00.000Z", "dimB", "sweet", "totalSum", 6000L @@ -471,15 +472,17 @@ public class NestedQueryPushDownTest .setGranularity(Granularities.ALL) .build(); - Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty()); - List results = queryResult.toList(); + Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery); + List results = queryResult.toList(); - Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + nestedQuery, "2017-07-14T02:40:00.000Z", "renamedDimB", "sour", "maxBSum", 20L ); - Row expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + nestedQuery, "2017-07-14T02:40:00.000Z", "renamedDimB", "sweet", "maxBSum", 60L @@ -533,8 +536,8 @@ public class NestedQueryPushDownTest .setQuerySegmentSpec(intervalSpec) .build(); - Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty()); - List results = queryResult.toList(); + Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery); + List results = queryResult.toList(); Assert.assertEquals(0, results.size()); } @@ -577,13 +580,14 @@ public class NestedQueryPushDownTest .setQuerySegmentSpec(intervalSpec) .build(); - Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + nestedQuery, "2017-07-14T02:40:00.000Z", "finalSum", 4000L, "newDimA", "mango" ); - Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty()); - List results = queryResult.toList(); + Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery); + List results = queryResult.toList(); Assert.assertEquals(1, results.size()); Assert.assertEquals(expectedRow0, results.get(0)); @@ -627,13 +631,14 @@ public class NestedQueryPushDownTest .setQuerySegmentSpec(intervalSpec) .build(); - Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + nestedQuery, "2017-07-14T02:40:00.000Z", "finalSum", 4000L, "newDimA", "mango" ); - Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty()); - List results = queryResult.toList(); + Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery); + List results = queryResult.toList(); Assert.assertEquals(1, results.size()); Assert.assertEquals(expectedRow0, results.get(0)); @@ -673,18 +678,20 @@ public class NestedQueryPushDownTest .setQuerySegmentSpec(intervalSpec) .build(); - Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + nestedQuery, "2017-07-14T02:40:00.000Z", "finalSum", 4000L, "extractedDimA", "p" ); - Row expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow( + nestedQuery, "2017-07-14T02:40:00.000Z", "finalSum", 4000L, "extractedDimA", "replacement" ); - Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty()); - List results = queryResult.toList(); + Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery); + List results = queryResult.toList(); Assert.assertEquals(2, results.size()); Assert.assertEquals(expectedRow0, results.get(0)); @@ -724,37 +731,39 @@ public class NestedQueryPushDownTest .setQuerySegmentSpec(intervalSpec) .build(); - Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( + nestedQuery, "2017-07-14T02:40:00.000Z", "dimB", "sweet", "finalSum", 90L ); - Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty()); - List results = queryResult.toList(); + Sequence queryResult = runNestedQueryWithForcePushDown(nestedQuery); + List results = queryResult.toList(); Assert.assertEquals(1, results.size()); Assert.assertEquals(expectedRow0, results.get(0)); } - private Sequence runNestedQueryWithForcePushDown(GroupByQuery nestedQuery, ResponseContext context) + private Sequence runNestedQueryWithForcePushDown(GroupByQuery nestedQuery) { - QueryToolChest toolChest = groupByFactory.getToolchest(); + ResponseContext context = ResponseContext.createEmpty(); + QueryToolChest toolChest = groupByFactory.getToolchest(); GroupByQuery pushDownQuery = nestedQuery; - QueryRunner segment1Runner = new FinalizeResultsQueryRunner( + QueryRunner segment1Runner = new FinalizeResultsQueryRunner( toolChest.mergeResults( groupByFactory.mergeRunners(executorService, getQueryRunnerForSegment1()) ), (QueryToolChest) toolChest ); - QueryRunner segment2Runner = new FinalizeResultsQueryRunner( + QueryRunner segment2Runner = new FinalizeResultsQueryRunner( toolChest.mergeResults( groupByFactory2.mergeRunners(executorService, getQueryRunnerForSegment2()) ), (QueryToolChest) toolChest ); - QueryRunner queryRunnerForSegments = new FinalizeResultsQueryRunner<>( + QueryRunner queryRunnerForSegments = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( (queryPlus, responseContext) -> Sequences .simple( @@ -786,7 +795,7 @@ public class NestedQueryPushDownTest GroupByQueryConfig.CTX_KEY_FORCE_PUSH_DOWN_NESTED_QUERY, false )); - Sequence pushDownQueryResults = strategy.mergeResults( + Sequence pushDownQueryResults = strategy.mergeResults( queryRunnerForSegments, queryWithPushDownDisabled, context @@ -826,7 +835,7 @@ public class NestedQueryPushDownTest .setContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_FORCE_PUSH_DOWN_NESTED_QUERY, true)) .setGranularity(Granularities.ALL) .build(); - QueryToolChest toolChest = groupByFactory.getToolchest(); + QueryToolChest toolChest = groupByFactory.getToolchest(); GroupByQuery rewrittenQuery = ((GroupByQueryQueryToolChest) toolChest).rewriteNestedQueryForPushDown(nestedQuery); Assert.assertEquals(outputNameB, rewrittenQuery.getDimensions().get(0).getDimension()); Assert.assertEquals(outputNameAgg, rewrittenQuery.getAggregatorSpecs().get(0).getName()); @@ -845,11 +854,11 @@ public class NestedQueryPushDownTest } - private List> getQueryRunnerForSegment1() + private List> getQueryRunnerForSegment1() { - List> runners = new ArrayList<>(); + List> runners = new ArrayList<>(); QueryableIndex index = groupByIndices.get(0); - QueryRunner runner = makeQueryRunnerForSegment( + QueryRunner runner = makeQueryRunnerForSegment( groupByFactory, SegmentId.dummy(index.toString()), new QueryableIndexSegment(index, SegmentId.dummy(index.toString())) @@ -858,11 +867,11 @@ public class NestedQueryPushDownTest return runners; } - private List> getQueryRunnerForSegment2() + private List> getQueryRunnerForSegment2() { - List> runners = new ArrayList<>(); + List> runners = new ArrayList<>(); QueryableIndex index2 = groupByIndices.get(1); - QueryRunner tooSmallRunner = makeQueryRunnerForSegment( + QueryRunner tooSmallRunner = makeQueryRunnerForSegment( groupByFactory2, SegmentId.dummy(index2.toString()), new QueryableIndexSegment(index2, SegmentId.dummy(index2.toString())) diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/BaseHavingSpec.java b/processing/src/test/java/org/apache/druid/query/groupby/ResultRowTest.java similarity index 59% rename from processing/src/main/java/org/apache/druid/query/groupby/having/BaseHavingSpec.java rename to processing/src/test/java/org/apache/druid/query/groupby/ResultRowTest.java index e4db0cbdb78..e32e61e2d18 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/having/BaseHavingSpec.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/ResultRowTest.java @@ -17,24 +17,21 @@ * under the License. */ -package org.apache.druid.query.groupby.having; +package org.apache.druid.query.groupby; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.segment.column.ValueType; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; -import java.util.Map; - -public abstract class BaseHavingSpec implements HavingSpec +public class ResultRowTest { - @Override - public void setRowSignature(Map rowSignature) + @Test + public void testSerde() throws Exception { - // Do nothing. - } - - @Override - public void setAggregators(Map aggregators) - { - + final ResultRow row = ResultRow.of(1, 2, 3); + final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + Assert.assertEquals(row, objectMapper.readValue("[1, 2, 3]", ResultRow.class)); + Assert.assertEquals(row, objectMapper.readValue(objectMapper.writeValueAsBytes(row), ResultRow.class)); } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/having/DimFilterHavingSpecTest.java b/processing/src/test/java/org/apache/druid/query/groupby/having/DimFilterHavingSpecTest.java index 8e8ee5a13bf..f0d82d67185 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/having/DimFilterHavingSpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/having/DimFilterHavingSpecTest.java @@ -20,10 +20,12 @@ package org.apache.druid.query.groupby.having; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.segment.column.ValueType; import org.hamcrest.CoreMatchers; import org.junit.Assert; @@ -49,20 +51,34 @@ public class DimFilterHavingSpecTest public void testSimple() { final DimFilterHavingSpec havingSpec = new DimFilterHavingSpec(new SelectorDimFilter("foo", "bar", null), null); - havingSpec.setRowSignature(null); + havingSpec.setQuery( + GroupByQuery.builder() + .setDataSource("dummy") + .setInterval("1000/3000") + .setDimensions(DefaultDimensionSpec.of("foo")) + .setGranularity(Granularities.ALL) + .build() + ); - Assert.assertTrue(havingSpec.eval(new MapBasedRow(0, ImmutableMap.of("foo", "bar")))); - Assert.assertFalse(havingSpec.eval(new MapBasedRow(0, ImmutableMap.of("foo", "baz")))); + Assert.assertTrue(havingSpec.eval(ResultRow.of("bar"))); + Assert.assertFalse(havingSpec.eval(ResultRow.of("baz"))); } @Test public void testRowSignature() { final DimFilterHavingSpec havingSpec = new DimFilterHavingSpec(new SelectorDimFilter("foo", "1", null), null); - havingSpec.setRowSignature(ImmutableMap.of("foo", ValueType.LONG)); + havingSpec.setQuery( + GroupByQuery.builder() + .setDataSource("dummy") + .setInterval("1000/3000") + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("foo", "foo", ValueType.LONG)) + .build() + ); - Assert.assertTrue(havingSpec.eval(new MapBasedRow(0, ImmutableMap.of("foo", 1L)))); - Assert.assertFalse(havingSpec.eval(new MapBasedRow(0, ImmutableMap.of("foo", 2L)))); + Assert.assertTrue(havingSpec.eval(ResultRow.of(1L))); + Assert.assertFalse(havingSpec.eval(ResultRow.of(2L))); } @Test(timeout = 60_000L) @@ -74,18 +90,13 @@ public class DimFilterHavingSpecTest final List> futures = new ArrayList<>(); for (int i = 0; i < 2; i++) { - final MapBasedRow row = new MapBasedRow(0, ImmutableMap.of("foo", String.valueOf(i))); + final ResultRow row = ResultRow.of(String.valueOf(i)); futures.add( exec.submit( - new Runnable() - { - @Override - public void run() - { - havingSpec.setRowSignature(null); - while (!Thread.interrupted()) { - havingSpec.eval(row); - } + () -> { + havingSpec.setQuery(GroupByQuery.builder().setDimensions(DefaultDimensionSpec.of("foo")).build()); + while (!Thread.interrupted()) { + havingSpec.eval(row); } } ) diff --git a/processing/src/test/java/org/apache/druid/query/groupby/having/DimensionSelectorHavingSpecTest.java b/processing/src/test/java/org/apache/druid/query/groupby/having/DimensionSelectorHavingSpecTest.java index b03be7bf386..5f3f7d5823c 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/having/DimensionSelectorHavingSpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/having/DimensionSelectorHavingSpecTest.java @@ -22,11 +22,10 @@ package org.apache.druid.query.groupby.having; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.data.input.MapBasedRow; -import org.apache.druid.data.input.Row; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.extraction.RegexDimExtractionFn; +import org.apache.druid.query.groupby.ResultRow; import org.junit.Assert; import org.junit.Test; @@ -35,9 +34,9 @@ import java.util.Map; public class DimensionSelectorHavingSpecTest { - private Row getTestRow(Object dimensionValue) + private ResultRow getTestRow(Object dimensionValue) { - return new MapBasedRow(0, ImmutableMap.of("dimension", dimensionValue)); + return ResultRow.of(dimensionValue); } @Test diff --git a/processing/src/test/java/org/apache/druid/query/groupby/having/HavingSpecTest.java b/processing/src/test/java/org/apache/druid/query/groupby/having/HavingSpecTest.java index 752126e1799..fbf42ca3765 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/having/HavingSpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/having/HavingSpecTest.java @@ -22,15 +22,14 @@ package org.apache.druid.query.groupby.having; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.Row; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.cache.CacheKeyBuilder; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.ResultRow; import org.junit.Assert; import org.junit.Test; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -38,11 +37,7 @@ import java.util.concurrent.atomic.AtomicInteger; public class HavingSpecTest { - private static final Row ROW = new MapBasedInputRow( - 0, - new ArrayList<>(), - ImmutableMap.of("metric", 10f) - ); + private static final ResultRow ROW = ResultRow.of(10f); @Test public void testHavingClauseSerde() @@ -139,9 +134,9 @@ public class HavingSpecTest Assert.assertFalse(spec.eval(getTestRow(Long.MAX_VALUE))); } - private Row getTestRow(Object metricValue) + private ResultRow getTestRow(Object metricValue) { - return new MapBasedInputRow(0, new ArrayList<>(), ImmutableMap.of("metric", metricValue)); + return ResultRow.of(metricValue); } @Test @@ -213,7 +208,7 @@ public class HavingSpecTest Assert.assertFalse(spec.eval(getTestRow(Long.MAX_VALUE))); } - private static class CountingHavingSpec extends BaseHavingSpec + private static class CountingHavingSpec implements HavingSpec { private final AtomicInteger counter; @@ -226,7 +221,13 @@ public class HavingSpecTest } @Override - public boolean eval(Row row) + public void setQuery(GroupByQuery query) + { + // Nothing to do. + } + + @Override + public boolean eval(ResultRow row) { counter.incrementAndGet(); return value; @@ -333,10 +334,10 @@ public class HavingSpecTest @Test public void testNotHavingSepc() { - NotHavingSpec spec = new NotHavingSpec(HavingSpec.NEVER); + NotHavingSpec spec = new NotHavingSpec(new NeverHavingSpec()); Assert.assertTrue(spec.eval(ROW)); - spec = new NotHavingSpec(HavingSpec.ALWAYS); + spec = new NotHavingSpec(new AlwaysHavingSpec()); Assert.assertFalse(spec.eval(ROW)); } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpecTest.java b/processing/src/test/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpecTest.java index 95d49cd51dd..8122a0e200c 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpecTest.java @@ -21,46 +21,46 @@ package org.apache.druid.query.groupby.orderby; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import org.apache.druid.data.input.MapBasedRow; -import org.apache.druid.data.input.Row; +import com.google.common.collect.ImmutableMap; 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.java.util.common.guava.Sequences; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; import org.apache.druid.query.aggregation.post.ConstantPostAggregator; -import org.apache.druid.query.aggregation.post.ExpressionPostAggregator; import org.apache.druid.query.dimension.DefaultDimensionSpec; -import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; -import java.util.HashMap; import java.util.List; -import java.util.Map; /** + * */ public class DefaultLimitSpecTest { - private final List testRowsList; - private final Sequence testRowsSequence; + private final List testRowsList; + private final List testRowsWithTimestampList; public DefaultLimitSpecTest() { testRowsList = ImmutableList.of( - createRow("2011-04-01", "k1", 10.0, "k2", 1L, "k3", 2L), - createRow("2011-04-01", "k1", 20.0, "k2", 3L, "k3", 1L), - createRow("2011-04-01", "k1", 9.0, "k2", 2L, "k3", 3L) + ResultRow.of(10.0, 1L, 2L), + ResultRow.of(20.0, 3L, 1L), + ResultRow.of(9.0, 2L, 3L) ); - testRowsSequence = Sequences.simple(testRowsList); + testRowsWithTimestampList = ImmutableList.of( + ResultRow.of(DateTimes.of("2011-04-01").getMillis(), 10.0, 1L, 2L), + ResultRow.of(DateTimes.of("2011-04-01").getMillis(), 20.0, 3L, 1L), + ResultRow.of(DateTimes.of("2011-04-01").getMillis(), 9.0, 2L, 3L) + ); } @Test @@ -93,7 +93,8 @@ public class DefaultLimitSpecTest ); Assert.assertEquals( new DefaultLimitSpec(ImmutableList.of(new OrderByColumnSpec("d", OrderByColumnSpec.Direction.DESCENDING, - StringComparators.NUMERIC)), 10), + StringComparators.NUMERIC + )), 10), spec ); @@ -110,7 +111,8 @@ public class DefaultLimitSpecTest Assert.assertEquals( new DefaultLimitSpec(ImmutableList.of(new OrderByColumnSpec("d", OrderByColumnSpec.Direction.DESCENDING, - StringComparators.NUMERIC)), 10), + StringComparators.NUMERIC + )), 10), spec ); @@ -125,7 +127,8 @@ public class DefaultLimitSpecTest ); Assert.assertEquals( new DefaultLimitSpec(ImmutableList.of(new OrderByColumnSpec("d", OrderByColumnSpec.Direction.ASCENDING, - StringComparators.LEXICOGRAPHIC)), 10), + StringComparators.LEXICOGRAPHIC + )), 10), spec ); @@ -140,7 +143,8 @@ public class DefaultLimitSpecTest ); Assert.assertEquals( new DefaultLimitSpec(ImmutableList.of(new OrderByColumnSpec("d", OrderByColumnSpec.Direction.ASCENDING, - StringComparators.LEXICOGRAPHIC)), 10), + StringComparators.LEXICOGRAPHIC + )), 10), spec ); @@ -155,17 +159,17 @@ public class DefaultLimitSpecTest 2 ); - Function, Sequence> limitFn = limitSpec.build( - ImmutableList.of(), - ImmutableList.of(), - ImmutableList.of(), - Granularities.NONE, - false + Function, Sequence> limitFn = limitSpec.build( + GroupByQuery.builder() + .setDataSource("dummy") + .setInterval("1000/3000") + .setGranularity(Granularities.NONE) + .build() ); Assert.assertEquals( - ImmutableList.of(testRowsList.get(0), testRowsList.get(1)), - limitFn.apply(testRowsSequence).toList() + ImmutableList.of(testRowsWithTimestampList.get(0), testRowsWithTimestampList.get(1)), + limitFn.apply(Sequences.simple(testRowsWithTimestampList)).toList() ); } @@ -177,17 +181,19 @@ public class DefaultLimitSpecTest 2 ); - Function, Sequence> limitFn = limitSpec.build( - ImmutableList.of(new DefaultDimensionSpec("k1", "k1", ValueType.DOUBLE)), - ImmutableList.of(), - ImmutableList.of(), - Granularities.ALL, - true + Function, Sequence> limitFn = limitSpec.build( + GroupByQuery.builder() + .setDataSource("dummy") + .setInterval("1000/3000") + .setDimensions(new DefaultDimensionSpec("k1", "k1", ValueType.DOUBLE)) + .setGranularity(Granularities.ALL) + .overrideContext(ImmutableMap.of(GroupByQuery.CTX_KEY_SORT_BY_DIMS_FIRST, true)) + .build() ); Assert.assertEquals( ImmutableList.of(testRowsList.get(0), testRowsList.get(1)), - limitFn.apply(testRowsSequence).toList() + limitFn.apply(Sequences.simple(testRowsList)).toList() ); } @@ -199,17 +205,19 @@ public class DefaultLimitSpecTest 2 ); - Function, Sequence> limitFn = limitSpec.build( - ImmutableList.of(new DefaultDimensionSpec("k1", "k1", ValueType.DOUBLE)), - ImmutableList.of(), - ImmutableList.of(), - Granularities.NONE, - true + Function, Sequence> limitFn = limitSpec.build( + GroupByQuery.builder() + .setDataSource("dummy") + .setInterval("1000/3000") + .setDimensions(new DefaultDimensionSpec("k1", "k1", ValueType.DOUBLE)) + .setGranularity(Granularities.NONE) + .overrideContext(ImmutableMap.of(GroupByQuery.CTX_KEY_SORT_BY_DIMS_FIRST, true)) + .build() ); Assert.assertEquals( - ImmutableList.of(testRowsList.get(2), testRowsList.get(0)), - limitFn.apply(testRowsSequence).toList() + ImmutableList.of(testRowsWithTimestampList.get(2), testRowsWithTimestampList.get(0)), + limitFn.apply(Sequences.simple(testRowsWithTimestampList)).toList() ); } @@ -221,19 +229,20 @@ public class DefaultLimitSpecTest 2 ); - Function, Sequence> limitFn = limitSpec.build( - ImmutableList.of(new DefaultDimensionSpec("k1", "k1")), - ImmutableList.of(), - ImmutableList.of(), - Granularities.NONE, - false + Function, Sequence> limitFn = limitSpec.build( + GroupByQuery.builder() + .setDataSource("dummy") + .setInterval("1000/3000") + .setDimensions(new DefaultDimensionSpec("k1", "k1", ValueType.DOUBLE)) + .setGranularity(Granularities.ALL) + .build() ); // Note: This test encodes the fact that limitSpec sorts numbers like strings; we might want to change this // in the future. Assert.assertEquals( ImmutableList.of(testRowsList.get(2), testRowsList.get(1)), - limitFn.apply(testRowsSequence).toList() + limitFn.apply(Sequences.simple(testRowsList)).toList() ); } @@ -247,91 +256,19 @@ public class DefaultLimitSpecTest 2 ); - Function, Sequence> limitFn = limitSpec.build( - ImmutableList.of( - new DefaultDimensionSpec("k1", "k1") - ), - ImmutableList.of( - new LongSumAggregatorFactory("k2", "k2") - ), - ImmutableList.of( - new ConstantPostAggregator("k3", 1L) - ), - Granularities.NONE, - false + Function, Sequence> limitFn = limitSpec.build( + GroupByQuery.builder() + .setDataSource("dummy") + .setInterval("1000/3000") + .setDimensions(new DefaultDimensionSpec("k1", "k1")) + .setAggregatorSpecs(new LongSumAggregatorFactory("k2", "k2")) + .setPostAggregatorSpecs(ImmutableList.of(new ConstantPostAggregator("k3", 1L))) + .setGranularity(Granularities.NONE) + .build() ); Assert.assertEquals( ImmutableList.of(testRowsList.get(0), testRowsList.get(1)), - limitFn.apply(testRowsSequence).toList() + limitFn.apply(Sequences.simple(testRowsList)).toList() ); - - // if there is an aggregator with same name then that is used to build ordering - limitFn = limitSpec.build( - ImmutableList.of( - new DefaultDimensionSpec("k1", "k1") - ), - ImmutableList.of( - new LongSumAggregatorFactory("k1", "k1") - ), - ImmutableList.of( - new ConstantPostAggregator("k3", 1L) - ), - Granularities.NONE, - false - ); - Assert.assertEquals( - ImmutableList.of(testRowsList.get(2), testRowsList.get(0)), - limitFn.apply(testRowsSequence).toList() - ); - - // if there is a post-aggregator with same name then that is used to build ordering - limitFn = limitSpec.build( - ImmutableList.of( - new DefaultDimensionSpec("k1", "k1") - ), - ImmutableList.of( - new LongSumAggregatorFactory("k2", "k2") - ), - ImmutableList.of( - new ArithmeticPostAggregator( - "k1", - "+", - ImmutableList.of( - new ConstantPostAggregator("x", 1), - new ConstantPostAggregator("y", 1)) - ) - ), - Granularities.NONE, - false - ); - Assert.assertEquals( - (List) ImmutableList.of(testRowsList.get(2), testRowsList.get(0)), - (List) limitFn.apply(testRowsSequence).toList() - ); - - // makes same result - limitFn = limitSpec.build( - ImmutableList.of(new DefaultDimensionSpec("k1", "k1")), - ImmutableList.of(new LongSumAggregatorFactory("k2", "k2")), - ImmutableList.of(new ExpressionPostAggregator("k1", "1 + 1", null, TestExprMacroTable.INSTANCE)), - Granularities.NONE, - false - ); - Assert.assertEquals( - (List) ImmutableList.of(testRowsList.get(2), testRowsList.get(0)), - (List) limitFn.apply(testRowsSequence).toList() - ); - } - - private Row createRow(String timestamp, Object... vals) - { - Preconditions.checkArgument(vals.length % 2 == 0); - - Map theVals = new HashMap<>(); - for (int i = 0; i < vals.length; i += 2) { - theVals.put(vals[i].toString(), vals[i + 1]); - } - - return new MapBasedRow(DateTimes.of(timestamp), theVals); } } diff --git a/processing/src/test/java/org/apache/druid/segment/TestHelper.java b/processing/src/test/java/org/apache/druid/segment/TestHelper.java index 3450e46407a..36d6115dae0 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/TestHelper.java @@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Result; import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.query.topn.TopNResultValue; import org.apache.druid.segment.column.ColumnConfig; @@ -46,6 +47,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; /** + * */ public class TestHelper { @@ -143,10 +145,10 @@ public class TestHelper final Object next = resultsIter.next(); final Object next2 = resultsIter2.next(); - if (expectedNext instanceof Row) { + if (expectedNext instanceof ResultRow) { // HACK! Special casing for groupBy - assertRow(failMsg, (Row) expectedNext, (Row) next); - assertRow(failMsg, (Row) expectedNext, (Row) next2); + assertRow(failMsg, (ResultRow) expectedNext, (ResultRow) next); + assertRow(failMsg, (ResultRow) expectedNext, (ResultRow) next2); } else if (expectedNext instanceof Result && (((Result) expectedNext).getValue()) instanceof TimeseriesResultValue) { // Special case for GroupByTimeseriesQueryRunnerTest to allow a floating point delta to be used @@ -179,13 +181,21 @@ public class TestHelper if (resultsIter.hasNext()) { Assert.fail( - StringUtils.format("%s: Expected resultsIter to be exhausted, next element was %s", failMsg, resultsIter.next()) + StringUtils.format( + "%s: Expected resultsIter to be exhausted, next element was %s", + failMsg, + resultsIter.next() + ) ); } if (resultsIter2.hasNext()) { Assert.fail( - StringUtils.format("%s: Expected resultsIter2 to be exhausted, next element was %s", failMsg, resultsIter.next()) + StringUtils.format( + "%s: Expected resultsIter2 to be exhausted, next element was %s", + failMsg, + resultsIter.next() + ) ); } @@ -213,12 +223,15 @@ public class TestHelper final Object next2 = resultsIter2.next(); String failMsg = msg + "-" + index++; - String failMsg2 = StringUtils.format("%s: Second iterator bad, multiple calls to iterator() should be safe", failMsg); + String failMsg2 = StringUtils.format( + "%s: Second iterator bad, multiple calls to iterator() should be safe", + failMsg + ); - if (expectedNext instanceof Row) { + if (expectedNext instanceof ResultRow) { // HACK! Special casing for groupBy - assertRow(failMsg, (Row) expectedNext, (Row) next); - assertRow(failMsg2, (Row) expectedNext, (Row) next2); + assertRow(failMsg, (ResultRow) expectedNext, (ResultRow) next); + assertRow(failMsg2, (ResultRow) expectedNext, (ResultRow) next2); } else { Assert.assertEquals(failMsg, expectedNext, next); Assert.assertEquals(failMsg2, expectedNext, next2); @@ -262,10 +275,14 @@ public class TestHelper TimeseriesResultValue expectedVal = (TimeseriesResultValue) expected.getValue(); TimeseriesResultValue actualVal = (TimeseriesResultValue) actual.getValue(); - final Map expectedMap = (Map) expectedVal.getBaseObject(); - final Map actualMap = (Map) actualVal.getBaseObject(); + final Map expectedMap = expectedVal.getBaseObject(); + final Map actualMap = actualVal.getBaseObject(); - assertRow(msg, new MapBasedRow(expected.getTimestamp(), expectedMap), new MapBasedRow(actual.getTimestamp(), actualMap)); + assertRow( + msg, + new MapBasedRow(expected.getTimestamp(), expectedMap), + new MapBasedRow(actual.getTimestamp(), actualMap) + ); } private static void assertTopNResultValue(String msg, Result expected, Result actual) @@ -332,6 +349,35 @@ public class TestHelper } } + private static void assertRow(String msg, ResultRow expected, ResultRow actual) + { + Assert.assertEquals( + StringUtils.format("%s: row length", msg), + expected.length(), + actual.length() + ); + + for (int i = 0; i < expected.length(); i++) { + final String message = StringUtils.format("%s: idx[%d]", msg, i); + final Object expectedValue = expected.get(i); + final Object actualValue = actual.get(i); + + if (expectedValue instanceof Float || expectedValue instanceof Double) { + Assert.assertEquals( + message, + ((Number) expectedValue).doubleValue(), + ((Number) actualValue).doubleValue(), + Math.abs(((Number) expectedValue).doubleValue() * 1e-6) + ); + } else { + Assert.assertEquals( + message, + expectedValue, + actualValue + ); + } + } + } public static Map createExpectedMap(Object... vals) { diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index 335ca490319..c420642ff08 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -550,7 +550,7 @@ public abstract class BaseFilterTest // Perform test final SettableSupplier rowSupplier = new SettableSupplier<>(); final ValueMatcher matcher = makeFilter(filter).makeMatcher( - VIRTUAL_COLUMNS.wrap(RowBasedColumnSelectorFactory.create(rowSupplier, rowSignature)) + VIRTUAL_COLUMNS.wrap(RowBasedColumnSelectorFactory.create(rowSupplier::get, rowSignature)) ); final List values = new ArrayList<>(); for (InputRow row : rows) { diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumnTest.java index 899554fc1d4..b189bf02c3f 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumnTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumnTest.java @@ -21,7 +21,6 @@ package org.apache.druid.segment.virtual; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -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.query.QueryRunnerTestHelper; @@ -33,6 +32,7 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTestHelper; +import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.query.topn.TopNResultValue; @@ -256,18 +256,12 @@ public class DummyStringVirtualColumnTest .addOrderByColumn(VSTRING_DIM) .build(); - List rows = groupByTestHelper.runQueryOnSegmentsObjs(segments, query).toList(); + List rows = groupByTestHelper.runQueryOnSegmentsObjs(segments, query).toList(); - List expectedRows = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2000-01-01T00:00:00.000Z", COUNT, 1674L, VSTRING_DIM, "spot"), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2000-01-01T00:00:00.000Z", - COUNT, - 372L, - VSTRING_DIM, - "total_market" - ), - GroupByQueryRunnerTestHelper.createExpectedRow("2000-01-01T00:00:00.000Z", COUNT, 372L, VSTRING_DIM, "upfront") + List expectedRows = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow(query, "2000", COUNT, 1674L, VSTRING_DIM, "spot"), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "2000", COUNT, 372L, VSTRING_DIM, "total_market"), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "2000", COUNT, 372L, VSTRING_DIM, "upfront") ); TestHelper.assertExpectedObjects(expectedRows, rows, "failed"); @@ -301,10 +295,10 @@ public class DummyStringVirtualColumnTest .setDimFilter(new SelectorDimFilter(VSTRING_DIM, "spot", null)) .build(); - List rows = groupByTestHelper.runQueryOnSegmentsObjs(segments, query).toList(); + List rows = groupByTestHelper.runQueryOnSegmentsObjs(segments, query).toList(); - List expectedRows = Collections.singletonList( - GroupByQueryRunnerTestHelper.createExpectedRow("2000-01-01T00:00:00.000Z", COUNT, 1674L, VSTRING_DIM, "spot") + List expectedRows = Collections.singletonList( + GroupByQueryRunnerTestHelper.createExpectedRow(query, "2000", COUNT, 1674L, VSTRING_DIM, "spot") ); TestHelper.assertExpectedObjects(expectedRows, rows, "failed"); @@ -338,11 +332,11 @@ public class DummyStringVirtualColumnTest .setDimFilter(new RegexDimFilter(VSTRING_DIM, "(spot)|(upfront)", null)) .build(); - List rows = groupByTestHelper.runQueryOnSegmentsObjs(segments, query).toList(); + List rows = groupByTestHelper.runQueryOnSegmentsObjs(segments, query).toList(); - List expectedRows = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2000-01-01T00:00:00.000Z", COUNT, 1674L, VSTRING_DIM, "spot"), - GroupByQueryRunnerTestHelper.createExpectedRow("2000-01-01T00:00:00.000Z", COUNT, 372L, VSTRING_DIM, "upfront") + List expectedRows = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow(query, "2000", COUNT, 1674L, VSTRING_DIM, "spot"), + GroupByQueryRunnerTestHelper.createExpectedRow(query, "2000", COUNT, 372L, VSTRING_DIM, "upfront") ); TestHelper.assertExpectedObjects(expectedRows, rows, "failed"); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java index 4e721dac20c..d53dd3fcbe2 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java @@ -176,7 +176,7 @@ public class ExpressionVirtualColumnTest private static final ThreadLocal CURRENT_ROW = new ThreadLocal<>(); private static final ColumnSelectorFactory COLUMN_SELECTOR_FACTORY = RowBasedColumnSelectorFactory.create( - CURRENT_ROW, + CURRENT_ROW::get, null ); @@ -596,7 +596,7 @@ public class ExpressionVirtualColumnTest { final ColumnValueSelector selector = ExpressionSelectors.makeExprEvalSelector( RowBasedColumnSelectorFactory.create( - CURRENT_ROW, + CURRENT_ROW::get, ImmutableMap.of("x", ValueType.LONG) ), Parser.parse(SCALE_LONG.getExpression(), TestExprMacroTable.INSTANCE) @@ -617,7 +617,7 @@ public class ExpressionVirtualColumnTest { final ColumnValueSelector selector = ExpressionSelectors.makeExprEvalSelector( RowBasedColumnSelectorFactory.create( - CURRENT_ROW, + CURRENT_ROW::get, ImmutableMap.of("x", ValueType.DOUBLE) ), Parser.parse(SCALE_FLOAT.getExpression(), TestExprMacroTable.INSTANCE) @@ -638,7 +638,7 @@ public class ExpressionVirtualColumnTest { final ColumnValueSelector selector = ExpressionSelectors.makeExprEvalSelector( RowBasedColumnSelectorFactory.create( - CURRENT_ROW, + CURRENT_ROW::get, ImmutableMap.of("x", ValueType.FLOAT) ), Parser.parse(SCALE_FLOAT.getExpression(), TestExprMacroTable.INSTANCE) diff --git a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java index 98802402ec8..ab06f54126a 100644 --- a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java +++ b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java @@ -508,7 +508,15 @@ public class DirectDruidClient implements QueryRunner @Override public JsonParserIterator make() { - return new JsonParserIterator(queryResultType, future, url, query, host, objectMapper, null); + return new JsonParserIterator( + queryResultType, + future, + url, + query, + host, + toolChest.decorateObjectMapper(objectMapper, query), + null + ); } @Override diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 69cef4c64df..79ac192229e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -38,6 +38,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.BySegmentQueryRunner; import org.apache.druid.query.CPUTimeMetricQueryRunner; +import org.apache.druid.query.FinalizeResultsQueryRunner; import org.apache.druid.query.MetricsEmittingQueryRunner; import org.apache.druid.query.NoopQueryRunner; import org.apache.druid.query.Query; @@ -176,7 +177,7 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker final boolean skipIncrementalSegment = query.getContextValue(CONTEXT_SKIP_INCREMENTAL_SEGMENT, false); final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); - return CPUTimeMetricQueryRunner.safeBuild( + final QueryRunner mergedRunner = toolChest.mergeResults( factory.mergeRunners( queryExecutorService, @@ -268,7 +269,10 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker } ) ) - ), + ); + + return CPUTimeMetricQueryRunner.safeBuild( + new FinalizeResultsQueryRunner<>(mergedRunner, toolChest), toolChest, emitter, cpuTimeAccumulator, diff --git a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java index 243036f38c5..00d982ad37c 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java +++ b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java @@ -345,6 +345,15 @@ public class QueryLifecycle return baseQuery; } + public QueryToolChest getToolChest() + { + if (state.compareTo(State.INITIALIZED) < 0) { + throw new ISE("Not yet initialized"); + } + + return toolChest; + } + private void transition(final State from, final State to) { if (state != from) { diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java index 15124864a36..f6c426d7d8b 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectWriter; import com.fasterxml.jackson.databind.module.SimpleModule; import com.fasterxml.jackson.datatype.joda.ser.DateTimeSerializer; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -42,6 +43,7 @@ import org.apache.druid.query.GenericQueryMetricsFactory; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryInterruptedException; +import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.server.metrics.QueryCountStatsProvider; import org.apache.druid.server.security.Access; @@ -51,6 +53,7 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; import org.joda.time.DateTime; +import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; @@ -71,8 +74,6 @@ import java.util.Set; import java.util.TreeSet; import java.util.concurrent.atomic.AtomicLong; -/** - */ @LazySingleton @Path("/druid/v2/") public class QueryResource implements QueryCountStatsProvider @@ -163,7 +164,9 @@ public class QueryResource implements QueryCountStatsProvider public Response doPost( final InputStream in, @QueryParam("pretty") final String pretty, - @Context final HttpServletRequest req // used to get request content-type,Accept header, remote address and auth-related headers + + // used to get request content-type,Accept header, remote address and auth-related headers + @Context final HttpServletRequest req ) throws IOException { final QueryLifecycle queryLifecycle = queryLifecycleFactory.factorize(); @@ -179,12 +182,20 @@ public class QueryResource implements QueryCountStatsProvider final String currThreadName = Thread.currentThread().getName(); try { - queryLifecycle.initialize(readQuery(req, in)); + queryLifecycle.initialize(readQuery(req, in, ioReaderWriter)); query = queryLifecycle.getQuery(); final String queryId = query.getId(); - Thread.currentThread() - .setName(StringUtils.format("%s[%s_%s_%s]", currThreadName, query.getType(), query.getDataSource().getNames(), queryId)); + final String queryThreadName = StringUtils.format( + "%s[%s_%s_%s]", + currThreadName, + query.getType(), + query.getDataSource().getNames(), + queryId + ); + + Thread.currentThread().setName(queryThreadName); + if (log.isDebugEnabled()) { log.debug("Got query [%s]", query); } @@ -212,7 +223,13 @@ public class QueryResource implements QueryCountStatsProvider boolean serializeDateTimeAsLong = QueryContexts.isSerializeDateTimeAsLong(query, false) || (!shouldFinalize && QueryContexts.isSerializeDateTimeAsLongInner(query, false)); - final ObjectWriter jsonWriter = ioReaderWriter.newOutputWriter(serializeDateTimeAsLong); + + final ObjectWriter jsonWriter = ioReaderWriter.newOutputWriter( + queryLifecycle.getToolChest(), + queryLifecycle.getQuery(), + serializeDateTimeAsLong + ); + Response.ResponseBuilder builder = Response .ok( new StreamingOutput() @@ -311,10 +328,11 @@ public class QueryResource implements QueryCountStatsProvider private Query readQuery( final HttpServletRequest req, - final InputStream in + final InputStream in, + final ResourceIOReaderWriter ioReaderWriter ) throws IOException { - Query baseQuery = getMapperForRequest(req.getContentType()).readValue(in, Query.class); + Query baseQuery = ioReaderWriter.getInputMapper().readValue(in, Query.class); String prevEtag = getPreviousEtag(req); if (prevEtag != null) { @@ -331,13 +349,6 @@ public class QueryResource implements QueryCountStatsProvider return req.getHeader(HEADER_IF_NONE_MATCH); } - protected ObjectMapper getMapperForRequest(String requestContentType) - { - boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(requestContentType) || - APPLICATION_SMILE.equals(requestContentType); - return isSmile ? smileMapper : jsonMapper; - } - protected ObjectMapper serializeDataTimeAsLong(ObjectMapper mapper) { return mapper.copy().registerModule(new SimpleModule().addSerializer(DateTime.class, new DateTimeSerializer())); @@ -386,22 +397,35 @@ public class QueryResource implements QueryCountStatsProvider return inputMapper; } - ObjectWriter newOutputWriter(boolean serializeDateTimeAsLong) + ObjectWriter newOutputWriter( + @Nullable QueryToolChest toolChest, + @Nullable Query query, + boolean serializeDateTimeAsLong + ) { - ObjectMapper mapper = serializeDateTimeAsLong ? serializeDateTimeAsLongInputMapper : inputMapper; - return isPretty ? mapper.writerWithDefaultPrettyPrinter() : mapper.writer(); + final ObjectMapper mapper = serializeDateTimeAsLong ? serializeDateTimeAsLongInputMapper : inputMapper; + final ObjectMapper decoratedMapper; + if (toolChest != null) { + decoratedMapper = toolChest.decorateObjectMapper(mapper, Preconditions.checkNotNull(query, "query")); + } else { + decoratedMapper = mapper; + } + return isPretty ? decoratedMapper.writerWithDefaultPrettyPrinter() : decoratedMapper.writer(); } Response ok(Object object) throws IOException { - return Response.ok(newOutputWriter(false).writeValueAsString(object), contentType).build(); + return Response.ok(newOutputWriter(null, null, false).writeValueAsString(object), contentType).build(); } Response gotError(Exception e) throws IOException { return Response.serverError() .type(contentType) - .entity(newOutputWriter(false).writeValueAsBytes(QueryInterruptedException.wrapIfNeeded(e))) + .entity( + newOutputWriter(null, null, false) + .writeValueAsBytes(QueryInterruptedException.wrapIfNeeded(e)) + ) .build(); } } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 6d59d7f14d3..60944fd9672 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -52,8 +52,6 @@ import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy; import org.apache.druid.client.selector.QueryableDruidServer; import org.apache.druid.client.selector.RandomServerSelectorStrategy; import org.apache.druid.client.selector.ServerSelector; -import org.apache.druid.data.input.MapBasedRow; -import org.apache.druid.data.input.Row; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.hll.HyperLogLogCollector; import org.apache.druid.java.util.common.DateTimes; @@ -100,6 +98,7 @@ import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.search.SearchHit; @@ -1463,18 +1462,21 @@ public class CachingClusteredClientTest query, Intervals.of("2011-01-01/2011-01-02"), makeGroupByResults( + query, DateTimes.of("2011-01-01"), ImmutableMap.of("a", "a", "rows", 1, "imps", 1, "impers", 1, "uniques", collector) ), Intervals.of("2011-01-02/2011-01-03"), makeGroupByResults( + query, DateTimes.of("2011-01-02"), ImmutableMap.of("a", "b", "rows", 2, "imps", 2, "impers", 2, "uniques", collector) ), Intervals.of("2011-01-05/2011-01-10"), makeGroupByResults( + query, DateTimes.of("2011-01-05"), ImmutableMap.of("a", "c", "rows", 3, "imps", 3, "impers", 3, "uniques", collector), DateTimes.of("2011-01-06"), @@ -1489,6 +1491,7 @@ public class CachingClusteredClientTest Intervals.of("2011-01-05/2011-01-10"), makeGroupByResults( + query, DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "c", "rows", 3, "imps", 3, "impers", 3, "uniques", collector), DateTimes.of("2011-01-06T01"), @@ -1508,6 +1511,7 @@ public class CachingClusteredClientTest ); TestHelper.assertExpectedObjects( makeGroupByResults( + query, DateTimes.of("2011-01-05T"), ImmutableMap.of("a", "c", "rows", 3, "imps", 3, "impers", 3, "uniques", collector), DateTimes.of("2011-01-05T01"), @@ -2070,14 +2074,14 @@ public class CachingClusteredClientTest } else if (query instanceof GroupByQuery) { List segmentIds = new ArrayList<>(); List intervals = new ArrayList<>(); - List> results = new ArrayList<>(); + List> results = new ArrayList<>(); for (ServerExpectation expectation : expectations) { segmentIds.add(expectation.getSegmentId()); intervals.add(expectation.getInterval()); results.add(expectation.getResults()); } EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(context))) - .andReturn(toQueryableGroupByResults(segmentIds, intervals, results)) + .andReturn(toQueryableGroupByResults((GroupByQuery) query, segmentIds, intervals, results)) .once(); } else if (query instanceof TimeBoundaryQuery) { List segmentIds = new ArrayList<>(); @@ -2114,8 +2118,10 @@ public class CachingClusteredClientTest { for (int i = 0; i < numTimesToQuery; ++i) { TestHelper.assertExpectedResults( - new MergeIterable<>( - Comparators.naturalNullsFirst(), + new MergeIterable( + query instanceof GroupByQuery + ? ((GroupByQuery) query).getResultOrdering() + : Comparators.naturalNullsFirst(), FunctionalIterable .create(new RangeIterable(expectedResultsRangeStart, expectedResultsRangeEnd)) .transformCat( @@ -2381,9 +2387,10 @@ public class CachingClusteredClientTest } private Sequence toQueryableGroupByResults( + GroupByQuery query, Iterable segmentIds, Iterable intervals, - Iterable> results + Iterable> results ) { return Sequences.simple( @@ -2392,14 +2399,26 @@ public class CachingClusteredClientTest .trinaryTransform( intervals, results, - new TrinaryFn, Result>() + new TrinaryFn, Result>() { @Override @SuppressWarnings("unchecked") - public Result apply(final SegmentId segmentId, final Interval interval, final Iterable results) + public Result apply( + final SegmentId segmentId, + final Interval interval, + final Iterable results + ) { + final DateTime timestamp; + + if (query.getUniversalTimestamp() != null) { + timestamp = query.getUniversalTimestamp(); + } else { + timestamp = query.getGranularity().toDateTime(results.iterator().next().getLong(0)); + } + return new Result( - results.iterator().next().getTimestamp(), + timestamp, new BySegmentResultValueClass( Lists.newArrayList(results), segmentId.toString(), @@ -2611,13 +2630,25 @@ public class CachingClusteredClientTest return retVal; } - private Iterable makeGroupByResults(Object... objects) + private Iterable makeGroupByResults(GroupByQuery query, Object... objects) { - List retVal = new ArrayList<>(); + List retVal = new ArrayList<>(); int index = 0; while (index < objects.length) { - DateTime timestamp = (DateTime) objects[index++]; - retVal.add(new MapBasedRow(timestamp, (Map) objects[index++])); + final DateTime timestamp = (DateTime) objects[index++]; + final Map rowMap = (Map) objects[index++]; + final ResultRow row = ResultRow.create(query.getResultRowSizeWithoutPostAggregators()); + + if (query.getResultRowHasTimestamp()) { + row.set(0, timestamp.getMillis()); + } + + for (Map.Entry entry : rowMap.entrySet()) { + final int position = query.getResultRowPositionLookup().getInt(entry.getKey()); + row.set(position, entry.getValue()); + } + + retVal.add(row); } return retVal; } @@ -3049,18 +3080,21 @@ public class CachingClusteredClientTest query1, Intervals.of("2011-01-01/2011-01-02"), makeGroupByResults( + query1, DateTimes.of("2011-01-01"), ImmutableMap.of("output", "a", "rows", 1, "imps", 1, "impers", 1) ), Intervals.of("2011-01-02/2011-01-03"), makeGroupByResults( + query1, DateTimes.of("2011-01-02"), ImmutableMap.of("output", "b", "rows", 2, "imps", 2, "impers", 2) ), Intervals.of("2011-01-05/2011-01-10"), makeGroupByResults( + query1, DateTimes.of("2011-01-05"), ImmutableMap.of("output", "c", "rows", 3, "imps", 3, "impers", 3), DateTimes.of("2011-01-06"), ImmutableMap.of("output", "d", "rows", 4, "imps", 4, "impers", 4), DateTimes.of("2011-01-07"), ImmutableMap.of("output", "e", "rows", 5, "imps", 5, "impers", 5), @@ -3070,6 +3104,7 @@ public class CachingClusteredClientTest Intervals.of("2011-01-05/2011-01-10"), makeGroupByResults( + query1, DateTimes.of("2011-01-05T01"), ImmutableMap.of("output", "c", "rows", 3, "imps", 3, "impers", 3), DateTimes.of("2011-01-06T01"), ImmutableMap.of("output", "d", "rows", 4, "imps", 4, "impers", 4), DateTimes.of("2011-01-07T01"), ImmutableMap.of("output", "e", "rows", 5, "imps", 5, "impers", 5), @@ -3085,6 +3120,7 @@ public class CachingClusteredClientTest ResponseContext context = ResponseContext.createEmpty(); TestHelper.assertExpectedObjects( makeGroupByResults( + query1, DateTimes.of("2011-01-05T"), ImmutableMap.of("output", "c", "rows", 3, "imps", 3, "impers", 3), DateTimes.of("2011-01-05T01"), ImmutableMap.of("output", "c", "rows", 3, "imps", 3, "impers", 3), DateTimes.of("2011-01-06T"), ImmutableMap.of("output", "d", "rows", 4, "imps", 4, "impers", 4), @@ -3106,6 +3142,7 @@ public class CachingClusteredClientTest .build(); TestHelper.assertExpectedObjects( makeGroupByResults( + query2, DateTimes.of("2011-01-05T"), ImmutableMap.of("output2", "c", "rows", 3, "imps", 3, "impers2", 3), DateTimes.of("2011-01-05T01"), ImmutableMap.of("output2", "c", "rows", 3, "imps", 3, "impers2", 3), DateTimes.of("2011-01-06T"), ImmutableMap.of("output2", "d", "rows", 4, "imps", 4, "impers2", 4), @@ -3188,16 +3225,16 @@ public class CachingClusteredClientTest timeline.add(interval, "ver", new SingleElementPartitionChunk<>(selector)); final TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder() - .dataSource(DATA_SOURCE) - .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(queryInterval))) - .context(ImmutableMap.of("If-None-Match", "aVJV29CJY93rszVW/QBy0arWZo0=")) - .build(); + .dataSource(DATA_SOURCE) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(queryInterval))) + .context(ImmutableMap.of("If-None-Match", "aVJV29CJY93rszVW/QBy0arWZo0=")) + .build(); final TimeBoundaryQuery query2 = Druids.newTimeBoundaryQueryBuilder() - .dataSource(DATA_SOURCE) - .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(queryInterval2))) - .context(ImmutableMap.of("If-None-Match", "aVJV29CJY93rszVW/QBy0arWZo0=")) - .build(); + .dataSource(DATA_SOURCE) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(queryInterval2))) + .context(ImmutableMap.of("If-None-Match", "aVJV29CJY93rszVW/QBy0arWZo0=")) + .build(); final ResponseContext responseContext = ResponseContext.createEmpty(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java index 6fca47296cb..6ba8dfa14a3 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java @@ -24,13 +24,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.primitives.Ints; +import it.unimi.dsi.fastutil.objects.Object2IntMap; import org.apache.calcite.avatica.ColumnMetaData; import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.runtime.Hook; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.NlsString; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -170,7 +170,6 @@ public class QueryMaker ); } - @SuppressWarnings("unchecked") private Sequence runQuery(Query query) { Hook.QUERY_PLAN.run(query); @@ -261,23 +260,27 @@ public class QueryMaker ) { final List fieldList = druidQuery.getOutputRowType().getFieldList(); + final Object2IntMap resultRowPositionLookup = query.getResultRowPositionLookup(); + final List sqlRowOrder = druidQuery.getOutputRowSignature().getRowOrder(); + final int[] resultRowPositions = new int[fieldList.size()]; + + for (final RelDataTypeField field : fieldList) { + final String columnName = sqlRowOrder.get(field.getIndex()); + final int resultRowPosition = resultRowPositionLookup.applyAsInt(columnName); + resultRowPositions[field.getIndex()] = resultRowPosition; + } return Sequences.map( runQuery(query), - new Function() - { - @Override - public Object[] apply(final Row row) - { - final Object[] retVal = new Object[fieldList.size()]; - for (RelDataTypeField field : fieldList) { - retVal[field.getIndex()] = coerce( - row.getRaw(druidQuery.getOutputRowSignature().getRowOrder().get(field.getIndex())), - field.getType().getSqlTypeName() - ); - } - return retVal; + resultRow -> { + final Object[] retVal = new Object[fieldList.size()]; + for (RelDataTypeField field : fieldList) { + retVal[field.getIndex()] = coerce( + resultRow.get(resultRowPositions[field.getIndex()]), + field.getType().getSqlTypeName() + ); } + return retVal; } ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index fbe2e19f5d9..583c1f5c799 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -428,25 +428,25 @@ public class CalciteTests public static final List ROWS1_WITH_FULL_TIMESTAMP = ImmutableList.of( createRow( - ImmutableMap.builder() - .put("t", "2000-01-01T10:51:45.695Z") - .put("m1", "1.0") - .put("m2", "1.0") - .put("dim1", "") - .put("dim2", ImmutableList.of("a")) - .put("dim3", ImmutableList.of("a", "b")) - .build() - ), + ImmutableMap.builder() + .put("t", "2000-01-01T10:51:45.695Z") + .put("m1", "1.0") + .put("m2", "1.0") + .put("dim1", "") + .put("dim2", ImmutableList.of("a")) + .put("dim3", ImmutableList.of("a", "b")) + .build() + ), createRow( - ImmutableMap.builder() - .put("t", "2000-01-18T10:51:45.695Z") - .put("m1", "2.0") - .put("m2", "2.0") - .put("dim1", "10.1") - .put("dim2", ImmutableList.of()) - .put("dim3", ImmutableList.of("b", "c")) - .build() - ) + ImmutableMap.builder() + .put("t", "2000-01-18T10:51:45.695Z") + .put("m1", "2.0") + .put("m2", "2.0") + .put("dim1", "10.1") + .put("dim2", ImmutableList.of()) + .put("dim3", ImmutableList.of("b", "c")) + .build() + ) ); @@ -496,6 +496,13 @@ public class CalciteTests return 10 * 1024 * 1024; } + @Override + public int getNumThreads() + { + // Only use 1 thread for tests. + return 1; + } + @Override public int getNumMergeBuffers() { @@ -673,20 +680,22 @@ public class CalciteTests .shardSpec(new LinearShardSpec(0)) .build(), forbiddenIndex - ).add(DataSegment.builder() - .dataSource(DATASOURCE3) - .interval(indexNumericDims.getDataInterval()) - .version("1") - .shardSpec(new LinearShardSpec(0)) - .build(), - indexNumericDims - ).add(DataSegment.builder() - .dataSource(DATASOURCE4) - .interval(index4.getDataInterval()) - .version("1") - .shardSpec(new LinearShardSpec(0)) - .build(), - index4 + ).add( + DataSegment.builder() + .dataSource(DATASOURCE3) + .interval(indexNumericDims.getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .build(), + indexNumericDims + ).add( + DataSegment.builder() + .dataSource(DATASOURCE4) + .interval(index4.getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .build(), + index4 ); }