From bf28d0775bbfe0ba8b1392046bc20fa88b3e7fbd Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Fri, 11 Aug 2017 03:12:38 +0300 Subject: [PATCH] Remove QueryRunner.run(Query, responseContext) and related legacy methods (#4482) * Remove QueryRunner.run(Query, responseContext) and related legacy methods * Remove local var --- .../FilteredAggregatorBenchmark.java | 3 +- .../GroupByTypeInterfaceBenchmark.java | 3 +- .../benchmark/TopNTypeInterfaceBenchmark.java | 3 +- .../benchmark/query/GroupByBenchmark.java | 9 +- .../benchmark/query/SearchBenchmark.java | 8 +- .../benchmark/query/SelectBenchmark.java | 5 +- .../druid/benchmark/query/SqlBenchmark.java | 3 +- .../benchmark/query/TimeseriesBenchmark.java | 8 +- .../druid/benchmark/query/TopNBenchmark.java | 8 +- .../query/scan/MultiSegmentScanQueryTest.java | 4 +- .../druid/query/scan/ScanQueryRunnerTest.java | 19 +- .../druid/segment/MapVirtualColumnTest.java | 3 +- .../ApproximateHistogramTopNQueryTest.java | 3 +- .../indexing/kafka/KafkaIndexTaskTest.java | 3 +- .../variance/VarianceTimeseriesQueryTest.java | 4 +- .../variance/VarianceTopNQueryTest.java | 7 +- .../common/task/RealtimeIndexTaskTest.java | 3 +- .../main/java/io/druid/query/BaseQuery.java | 11 +- .../src/main/java/io/druid/query/Query.java | 16 +- .../main/java/io/druid/query/QueryPlus.java | 7 +- .../main/java/io/druid/query/QueryRunner.java | 19 +- .../io/druid/query/AsyncQueryRunnerTest.java | 19 +- .../ChainedExecutionQueryRunnerTest.java | 32 +- .../IntervalChunkingQueryRunnerTest.java | 4 +- .../druid/query/LegacyApiQueryRunnerTest.java | 64 --- .../druid/query/MultiValuedDimensionTest.java | 2 +- .../io/druid/query/RetryQueryRunnerTest.java | 10 +- .../io/druid/query/SchemaEvolutionTest.java | 2 +- .../io/druid/query/TimewarpOperatorTest.java | 12 +- .../io/druid/query/UnionQueryRunnerTest.java | 2 +- .../aggregation/AggregationTestHelper.java | 2 +- .../DataSourceMetadataQueryTest.java | 3 +- .../groupby/GroupByMultiSegmentTest.java | 2 +- .../GroupByQueryRunnerFactoryTest.java | 2 +- .../query/groupby/GroupByQueryRunnerTest.java | 236 ++++++----- .../groupby/GroupByQueryRunnerTestHelper.java | 3 +- .../GroupByTimeseriesQueryRunnerTest.java | 28 +- .../query/metadata/SegmentAnalyzerTest.java | 3 +- .../metadata/SegmentMetadataQueryTest.java | 140 +++---- .../SegmentMetadataUnionQueryTest.java | 6 +- .../query/search/SearchQueryRunnerTest.java | 2 +- .../search/SearchQueryRunnerWithCaseTest.java | 3 +- .../select/MultiSegmentSelectQueryTest.java | 12 +- .../query/select/SelectQueryRunnerTest.java | 29 +- .../spec/SpecificSegmentQueryRunnerTest.java | 9 +- .../TimeBoundaryQueryRunnerTest.java | 11 +- .../TimeSeriesUnionQueryRunnerTest.java | 4 +- .../TimeseriesQueryRunnerBonusTest.java | 3 +- .../timeseries/TimeseriesQueryRunnerTest.java | 386 ++++++++++-------- .../topn/TopNQueryQueryToolChestTest.java | 12 +- .../query/topn/TopNQueryRunnerBenchmark.java | 9 +- .../druid/query/topn/TopNQueryRunnerTest.java | 9 +- .../druid/query/topn/TopNUnionQueryTest.java | 3 +- .../java/io/druid/segment/AppendTest.java | 31 +- .../IndexMergerV9WithSpatialIndexTest.java | 7 +- .../druid/segment/SchemalessTestFullTest.java | 15 +- .../segment/SchemalessTestSimpleTest.java | 9 +- .../segment/data/IncrementalIndexTest.java | 7 +- .../filter/SpatialFilterBonusTest.java | 7 +- .../segment/filter/SpatialFilterTest.java | 7 +- .../OnheapIncrementalIndexBenchmark.java | 5 +- .../client/CachingClusteredClientTest.java | 186 ++++----- .../druid/client/CachingQueryRunnerTest.java | 4 +- .../druid/client/DirectDruidClientTest.java | 21 +- .../appenderator/AppenderatorTest.java | 15 +- .../coordination/ServerManagerTest.java | 2 +- 66 files changed, 735 insertions(+), 794 deletions(-) delete mode 100644 processing/src/test/java/io/druid/query/LegacyApiQueryRunnerTest.java diff --git a/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java index 13c079f5b05..87fa11c3ba3 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java @@ -38,6 +38,7 @@ import io.druid.js.JavaScriptConfig; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -251,7 +252,7 @@ public class FilteredAggregatorBenchmark toolChest ); - Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); return Sequences.toList(queryResult, Lists.newArrayList()); } diff --git a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java index 2cc3ba6c736..aada2596eac 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -49,6 +49,7 @@ import io.druid.offheap.OffheapBufferGenerator; import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -472,7 +473,7 @@ public class GroupByTypeInterfaceBenchmark toolChest ); - Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); return Sequences.toList(queryResult, Lists.newArrayList()); } diff --git a/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java index f3fa2fb1751..b62ac5e28eb 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java @@ -39,6 +39,7 @@ import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -327,7 +328,7 @@ public class TopNTypeInterfaceBenchmark toolChest ); - Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); return Sequences.toList(queryResult, Lists.newArrayList()); } diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java index 01bb6c865b4..20f0e46c12b 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java @@ -50,6 +50,7 @@ import io.druid.offheap.OffheapBufferGenerator; import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -572,7 +573,7 @@ public class GroupByBenchmark toolChest ); - Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); return Sequences.toList(queryResult, Lists.newArrayList()); } @@ -625,7 +626,7 @@ public class GroupByBenchmark (QueryToolChest) toolChest ); - Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); List results = Sequences.toList(queryResult, Lists.newArrayList()); for (Row result : results) { @@ -649,7 +650,7 @@ public class GroupByBenchmark final GroupByQuery spillingQuery = query.withOverriddenContext( ImmutableMap.of("bufferGrouperMaxSize", 4000) ); - Sequence queryResult = theRunner.run(spillingQuery, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(spillingQuery), Maps.newHashMap()); List results = Sequences.toList(queryResult, Lists.newArrayList()); for (Row result : results) { @@ -676,7 +677,7 @@ public class GroupByBenchmark (QueryToolChest) toolChest ); - Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); List results = Sequences.toList(queryResult, Lists.newArrayList()); for (Row result : results) { diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java index 053d0730890..babe91f61d7 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java @@ -42,6 +42,7 @@ import io.druid.query.Druids; import io.druid.query.Druids.SearchQueryBuilder; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -400,7 +401,7 @@ public class SearchBenchmark toolChest ); - Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); return Sequences.toList(queryResult, Lists.newArrayList()); } @@ -465,7 +466,10 @@ public class SearchBenchmark ) ); - Sequence> queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence> queryResult = theRunner.run( + QueryPlus.wrap(query), + Maps.newHashMap() + ); List> results = Sequences.toList( queryResult, Lists.>newArrayList() diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java index 426f2da0f2b..39da308ee63 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java @@ -40,6 +40,7 @@ import io.druid.java.util.common.logger.Logger; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -265,7 +266,7 @@ public class SelectBenchmark toolChest ); - Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); return Sequences.toList(queryResult, Lists.newArrayList()); } @@ -374,7 +375,7 @@ public class SelectBenchmark boolean done = false; while (!done) { - Sequence> queryResult = theRunner.run(queryCopy, Maps.newHashMap()); + Sequence> queryResult = theRunner.run(QueryPlus.wrap(queryCopy), Maps.newHashMap()); List> results = Sequences.toList(queryResult, Lists.>newArrayList()); SelectResultValue result = results.get(0).getValue(); diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java index 0f831f77eee..a54cdf3d44b 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java @@ -31,6 +31,7 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -165,7 +166,7 @@ public class SqlBenchmark @OutputTimeUnit(TimeUnit.MILLISECONDS) public void queryNative(Blackhole blackhole) throws Exception { - final Sequence resultSequence = groupByQuery.run(walker, Maps.newHashMap()); + final Sequence resultSequence = QueryPlus.wrap(groupByQuery).run(walker, Maps.newHashMap()); final ArrayList resultList = Sequences.toList(resultSequence, Lists.newArrayList()); for (Row row : resultList) { diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java index 221e9381c8b..028334c5997 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java @@ -37,6 +37,7 @@ import io.druid.java.util.common.logger.Logger; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -324,7 +325,7 @@ public class TimeseriesBenchmark toolChest ); - Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); return Sequences.toList(queryResult, Lists.newArrayList()); } @@ -406,7 +407,10 @@ public class TimeseriesBenchmark ) ); - Sequence> queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence> queryResult = theRunner.run( + QueryPlus.wrap(query), + Maps.newHashMap() + ); List> results = Sequences.toList(queryResult, Lists.>newArrayList()); for (Result result : results) { diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java index 29a10ae96f1..d29c90f5717 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java @@ -38,6 +38,7 @@ import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -305,7 +306,7 @@ public class TopNBenchmark toolChest ); - Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); return Sequences.toList(queryResult, Lists.newArrayList()); } @@ -367,7 +368,10 @@ public class TopNBenchmark ) ); - Sequence> queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence> queryResult = theRunner.run( + QueryPlus.wrap(query), + Maps.newHashMap() + ); List> results = Sequences.toList(queryResult, Lists.>newArrayList()); for (Result result : results) { diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java index 1cf6830e1af..0a4a557a8c8 100644 --- a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java @@ -195,7 +195,7 @@ public class MultiSegmentScanQueryTest factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of( factory.createRunner(segment0), factory.createRunner(segment1) - )).run(query, new HashMap()), + )).run(QueryPlus.wrap(query), new HashMap()), Lists.newArrayList() ); int totalCount = 0; @@ -232,7 +232,7 @@ public class MultiSegmentScanQueryTest ); ScanQuery query = newBuilder().build(); List results = Sequences.toList( - runner.run(query, new HashMap()), + runner.run(QueryPlus.wrap(query), new HashMap()), Lists.newArrayList() ); int totalCount = 0; diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java index 601cf53e58d..c5e03915ac5 100644 --- a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java @@ -28,6 +28,7 @@ import com.google.common.collect.Sets; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequences; import io.druid.query.DefaultGenericQueryMetricsFactory; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.TableDataSource; @@ -159,7 +160,7 @@ public class ScanQueryRunnerTest HashMap context = new HashMap(); Iterable results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.newArrayList() ); @@ -202,7 +203,7 @@ public class ScanQueryRunnerTest HashMap context = new HashMap(); Iterable results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.newArrayList() ); @@ -225,7 +226,7 @@ public class ScanQueryRunnerTest HashMap context = new HashMap(); Iterable results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.newArrayList() ); @@ -263,7 +264,7 @@ public class ScanQueryRunnerTest HashMap context = new HashMap(); Iterable results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.newArrayList() ); @@ -304,7 +305,7 @@ public class ScanQueryRunnerTest HashMap context = new HashMap(); Iterable results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.newArrayList() ); @@ -366,13 +367,13 @@ public class ScanQueryRunnerTest .build(); Iterable results = Sequences.toList( - runner.run(query, Maps.newHashMap()), + runner.run(QueryPlus.wrap(query), Maps.newHashMap()), Lists.newArrayList() ); Iterable resultsOptimize = Sequences.toList( toolChest .postMergeQueryDecoration(toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner))) - .run(query, Maps.newHashMap()), Lists.newArrayList() + .run(QueryPlus.wrap(query), Maps.newHashMap()), Lists.newArrayList() ); final List>> events = toEvents( @@ -422,7 +423,7 @@ public class ScanQueryRunnerTest .build(); Iterable results = Sequences.toList( - runner.run(query, Maps.newHashMap()), + runner.run(QueryPlus.wrap(query), Maps.newHashMap()), Lists.newArrayList() ); @@ -440,7 +441,7 @@ public class ScanQueryRunnerTest .build(); Iterable results = Sequences.toList( - runner.run(query, Maps.newHashMap()), + runner.run(QueryPlus.wrap(query), Maps.newHashMap()), Lists.newArrayList() ); diff --git a/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java b/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java index 41c907f62fb..bc5a3629895 100644 --- a/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java @@ -32,6 +32,7 @@ import io.druid.data.input.impl.TimestampSpec; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -185,7 +186,7 @@ public class MapVirtualColumnTest private void checkSelectQuery(SelectQuery searchQuery, List expected) throws Exception { List> results = Sequences.toList( - runner.run(searchQuery, ImmutableMap.of()), + runner.run(QueryPlus.wrap(searchQuery), ImmutableMap.of()), Lists.>newArrayList() ); Assert.assertEquals(1, results.size()); diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java index ba5af3778f3..e42b0f4a7ac 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.collections.StupidPool; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -247,6 +248,6 @@ public class ApproximateHistogramTopNQueryTest ); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 0fe5791a60d..a6bc82afe8a 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -87,6 +87,7 @@ import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Druids; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -1728,7 +1729,7 @@ public class KafkaIndexTaskTest .build(); ArrayList> results = Sequences.toList( - task.getQueryRunner(query).run(query, ImmutableMap.of()), + task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()), Lists.>newArrayList() ); diff --git a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceTimeseriesQueryTest.java b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceTimeseriesQueryTest.java index 88b2288bc93..707e1364ce4 100644 --- a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceTimeseriesQueryTest.java +++ b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceTimeseriesQueryTest.java @@ -20,9 +20,9 @@ package io.druid.query.aggregation.variance; import com.google.common.collect.Lists; - import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; @@ -107,7 +107,7 @@ public class VarianceTimeseriesQueryTest ); Iterable> results = Sequences.toList( - runner.run(query, new HashMap()), + runner.run(QueryPlus.wrap(query), new HashMap()), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); diff --git a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceTopNQueryTest.java b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceTopNQueryTest.java index 22985f29999..170592aac50 100644 --- a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceTopNQueryTest.java +++ b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceTopNQueryTest.java @@ -22,8 +22,8 @@ package io.druid.query.aggregation.variance; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; - import io.druid.java.util.common.guava.Sequence; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -142,7 +142,10 @@ public class VarianceTopNQueryTest QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() ); final QueryRunner> mergeRunner = chest.mergeResults(runner); - final Sequence> retval = mergeRunner.run(query, ImmutableMap.of()); + final Sequence> retval = mergeRunner.run( + QueryPlus.wrap(query), + ImmutableMap.of() + ); TestHelper.assertExpectedResults(expectedResults, retval); return retval; } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index fe8c5ec33b1..ce6510fa316 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -77,6 +77,7 @@ import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Druids; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -1067,7 +1068,7 @@ public class RealtimeIndexTaskTest .build(); ArrayList> results = Sequences.toList( - task.getQueryRunner(query).run(query, ImmutableMap.of()), + task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()), Lists.>newArrayList() ); return results.isEmpty() ? 0 : results.get(0).getValue().getLongMetric(metric); diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index 4a3e4716a59..8869508807f 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -24,7 +24,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; -import io.druid.java.util.common.guava.Sequence; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Duration; import org.joda.time.Interval; @@ -87,15 +86,9 @@ public abstract class BaseQuery> implements Query } @Override - public Sequence run(QuerySegmentWalker walker, Map context) + public QueryRunner getRunner(QuerySegmentWalker walker, Map context) { - return run(querySegmentSpec.lookup(this, walker), context); - } - - @Override - public Sequence run(QueryRunner runner, Map context) - { - return runner.run(this, context); + return querySegmentSpec.lookup(this, walker); } @Override diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index 4e6745ca6d4..e8c62962f53 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -22,7 +22,6 @@ package io.druid.query; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.collect.Ordering; -import io.druid.java.util.common.guava.Sequence; import io.druid.query.datasourcemetadata.DataSourceMetadataQuery; import io.druid.query.filter.DimFilter; import io.druid.query.groupby.GroupByQuery; @@ -70,20 +69,7 @@ public interface Query String getType(); - /** - * @deprecated use {@link QueryPlus#run(QuerySegmentWalker, Map)} instead. This method is going to be removed in Druid - * 0.11. In the future, a method like getRunner(QuerySegmentWalker, Map) could be added instead of this method, so - * that {@link QueryPlus#run(QuerySegmentWalker, Map)} could be implemented as {@code - * this.query.getRunner(walker, context).run(this, context))}. - */ - @Deprecated - Sequence run(QuerySegmentWalker walker, Map context); - - /** - * @deprecated use {@link QueryRunner#run(QueryPlus, Map)} instead. This method is going to be removed in Druid 0.11. - */ - @Deprecated - Sequence run(QueryRunner runner, Map context); + QueryRunner getRunner(QuerySegmentWalker walker, Map context); List getIntervals(); diff --git a/processing/src/main/java/io/druid/query/QueryPlus.java b/processing/src/main/java/io/druid/query/QueryPlus.java index 0e155301180..c6cfe6f89f8 100644 --- a/processing/src/main/java/io/druid/query/QueryPlus.java +++ b/processing/src/main/java/io/druid/query/QueryPlus.java @@ -124,11 +124,6 @@ public final class QueryPlus public Sequence run(QuerySegmentWalker walker, Map context) { - if (query instanceof BaseQuery) { - return ((BaseQuery) query).getQuerySegmentSpec().lookup(query, walker).run(this, context); - } else { - // fallback - return query.run(walker, context); - } + return query.getRunner(walker, context).run(this, context); } } diff --git a/processing/src/main/java/io/druid/query/QueryRunner.java b/processing/src/main/java/io/druid/query/QueryRunner.java index a7e17f43d24..cf9366451e0 100644 --- a/processing/src/main/java/io/druid/query/QueryRunner.java +++ b/processing/src/main/java/io/druid/query/QueryRunner.java @@ -23,27 +23,10 @@ import io.druid.java.util.common.guava.Sequence; import java.util.Map; -/** - * This interface has two similar run() methods. {@link #run(Query, Map)} is legacy and {@link #run(QueryPlus, Map)} - * is the new one. Their default implementations delegate to each other. Every implementation of QueryRunner should - * override only one of those methods. New implementations should override the new method: {@link #run(QueryPlus, Map)}. - */ public interface QueryRunner { - /** - * @deprecated use and override {@link #run(QueryPlus, Map)} instead. This method is going to be removed in Druid 0.11 - */ - @Deprecated - default Sequence run(Query query, Map responseContext) - { - return run(QueryPlus.wrap(query), responseContext); - } - /** * Runs the given query and returns results in a time-ordered sequence. */ - default Sequence run(QueryPlus queryPlus, Map responseContext) - { - return run(queryPlus.getQuery(), responseContext); - } + Sequence run(QueryPlus queryPlus, Map responseContext); } diff --git a/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java b/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java index 83c205606b3..7d8dbdbc33d 100644 --- a/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java @@ -79,8 +79,8 @@ public class AsyncQueryRunnerTest executor, QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - - Sequence lazy = asyncRunner.run(query, Collections.EMPTY_MAP); + + Sequence lazy = asyncRunner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP); latch.countDown(); Assert.assertEquals(Lists.newArrayList(1), Sequences.toList(lazy, Lists.newArrayList())); } @@ -110,7 +110,7 @@ public class AsyncQueryRunnerTest ); Sequence lazy = asyncRunner.run( - query.withOverriddenContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 1)), + QueryPlus.wrap(query.withOverriddenContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 1))), Collections.EMPTY_MAP ); @@ -127,22 +127,15 @@ public class AsyncQueryRunnerTest @Test public void testQueryRegistration() { - QueryRunner baseRunner = new QueryRunner() - { - @Override - public Sequence run(QueryPlus queryPlus, Map responseContext) - { - return null; - } - }; + QueryRunner baseRunner = (queryPlus, responseContext) -> null; QueryWatcher mock = EasyMock.createMock(QueryWatcher.class); mock.registerQuery(EasyMock.eq(query), EasyMock.anyObject(ListenableFuture.class)); EasyMock.replay(mock); AsyncQueryRunner asyncRunner = new AsyncQueryRunner<>(baseRunner, executor, mock); - - asyncRunner.run(query, Collections.EMPTY_MAP); + + asyncRunner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP); EasyMock.verify(mock); } } diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index 935957eb079..8b0ec89c946 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -26,8 +26,8 @@ import com.google.common.util.concurrent.ListenableFuture; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.lifecycle.Lifecycle; -import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.timeseries.TimeseriesQuery; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.IAnswer; @@ -119,14 +119,12 @@ public class ChainedExecutionQueryRunnerTest ) ); Map context = ImmutableMap.of(); - final Sequence seq = chainedRunner.run( - Druids.newTimeseriesQueryBuilder() - .dataSource("test") - .intervals("2014/2015") - .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) - .build(), - context - ); + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals("2014/2015") + .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) + .build(); + final Sequence seq = chainedRunner.run(QueryPlus.wrap(query), context); Future resultFuture = Executors.newFixedThreadPool(1).submit( new Runnable() @@ -246,15 +244,13 @@ public class ChainedExecutionQueryRunnerTest ) ); HashMap context = new HashMap(); - final Sequence seq = chainedRunner.run( - Druids.newTimeseriesQueryBuilder() - .dataSource("test") - .intervals("2014/2015") - .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) - .context(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 100, "queryId", "test")) - .build(), - context - ); + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals("2014/2015") + .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) + .context(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 100, "queryId", "test")) + .build(); + final Sequence seq = chainedRunner.run(QueryPlus.wrap(query), context); Future resultFuture = Executors.newFixedThreadPool(1).submit( new Runnable() diff --git a/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java b/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java index 550f9464af5..ef389424f00 100644 --- a/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java @@ -86,7 +86,7 @@ public class IntervalChunkingQueryRunnerTest EasyMock.replay(toolChest); QueryRunner runner = decorator.decorate(baseRunner, toolChest); - runner.run(query, Collections.EMPTY_MAP); + runner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP); EasyMock.verify(executors); } @@ -103,7 +103,7 @@ public class IntervalChunkingQueryRunnerTest EasyMock.replay(toolChest); QueryRunner runner = decorator.decorate(baseRunner, toolChest); - runner.run(query, Collections.EMPTY_MAP); + runner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP); EasyMock.verify(executors); } diff --git a/processing/src/test/java/io/druid/query/LegacyApiQueryRunnerTest.java b/processing/src/test/java/io/druid/query/LegacyApiQueryRunnerTest.java deleted file mode 100644 index 31bf7825fc4..00000000000 --- a/processing/src/test/java/io/druid/query/LegacyApiQueryRunnerTest.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.query; - -import com.google.common.collect.ImmutableList; -import io.druid.java.util.common.guava.Sequence; -import io.druid.java.util.common.guava.Sequences; -import io.druid.query.spec.MultipleIntervalSegmentSpec; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Test; - -import java.util.HashMap; -import java.util.Map; - -/** - * Tests that if a QueryRunner overrides a legacy {@link QueryRunner#run(Query, Map)} method, it still works. This - * test should be removed when {@link QueryRunner#run(Query, Map)} is removed. - */ -public class LegacyApiQueryRunnerTest -{ - private static class LegacyApiQueryRunner implements QueryRunner - { - /** - * Overrides legacy API. - */ - @Override - public Sequence run(Query query, Map responseContext) - { - return Sequences.empty(); - } - } - - @Test - public void testQueryRunnerLegacyApi() - { - final Query query = new TestQuery( - new TableDataSource("test"), - new MultipleIntervalSegmentSpec(ImmutableList.of(new Interval("0/100"))), - false, - new HashMap() - ); - - Map context = new HashMap<>(); - Assert.assertEquals(Sequences.empty(), new LegacyApiQueryRunner<>().run(QueryPlus.wrap(query), context)); - } -} diff --git a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java index 97936a23b46..e17b20369f4 100644 --- a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java @@ -310,7 +310,7 @@ public class MultiValuedDimensionTest null ); Map context = Maps.newHashMap(); - Sequence> result = runner.run(query, context); + Sequence> result = runner.run(QueryPlus.wrap(query), context); List> expectedResults = Arrays.asList( new Result( new DateTime("2011-01-12T00:00:00.000Z"), diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index 62c7b69cbff..e046b60d664 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -129,7 +129,7 @@ public class RetryQueryRunnerTest ); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); @@ -189,7 +189,7 @@ public class RetryQueryRunnerTest ); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); @@ -248,7 +248,7 @@ public class RetryQueryRunnerTest ); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); @@ -292,7 +292,7 @@ public class RetryQueryRunnerTest ); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); @@ -396,7 +396,7 @@ public class RetryQueryRunnerTest ); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); diff --git a/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java b/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java index 41d0f47e9c7..71871864778 100644 --- a/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java +++ b/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java @@ -123,7 +123,7 @@ public class SchemaEvolutionTest ) ), (QueryToolChest>) factory.getToolchest() - ).run(query, Maps.newHashMap()); + ).run(QueryPlus.wrap(query), Maps.newHashMap()); return Sequences.toList(results, Lists.newArrayList()); } diff --git a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java index 133e435ed97..f0b6c490f0a 100644 --- a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java +++ b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java @@ -128,7 +128,10 @@ public class TimewarpOperatorTest new TimeseriesResultValue(ImmutableMap.of("metric", 5)) ) ), - Sequences.toList(queryRunner.run(query, CONTEXT), Lists.>newArrayList()) + Sequences.toList( + queryRunner.run(QueryPlus.wrap(query), CONTEXT), + Lists.>newArrayList() + ) ); @@ -178,7 +181,7 @@ public class TimewarpOperatorTest ) ), Sequences.toList( - timeBoundaryRunner.run(timeBoundaryQuery, CONTEXT), + timeBoundaryRunner.run(QueryPlus.wrap(timeBoundaryQuery), CONTEXT), Lists.>newArrayList() ) ); @@ -233,7 +236,10 @@ public class TimewarpOperatorTest new TimeseriesResultValue(ImmutableMap.of("metric", 3)) ) ), - Sequences.toList(queryRunner.run(query, Maps.newHashMap()), Lists.>newArrayList()) + Sequences.toList( + queryRunner.run(QueryPlus.wrap(query), Maps.newHashMap()), + Lists.>newArrayList() + ) ); } } diff --git a/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java index 2b3529ddfc0..090255692bd 100644 --- a/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java @@ -70,7 +70,7 @@ public class UnionQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonDoubleAggregators) .build(); Map responseContext = Maps.newHashMap(); - Sequence result = runner.run(q, responseContext); + Sequence result = runner.run(QueryPlus.wrap(q), responseContext); List res = Sequences.toList(result, Lists.newArrayList()); Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6), res); diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java index fe89aafa071..0c4d9b62001 100644 --- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java @@ -551,7 +551,7 @@ public class AggregationTestHelper toolChest ); - return baseRunner.run(query, Maps.newHashMap()); + return baseRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); } public QueryRunner makeStringSerdeQueryRunner(final ObjectMapper mapper, final QueryToolChest toolChest, final Query query, final QueryRunner baseRunner) diff --git a/processing/src/test/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java b/processing/src/test/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java index 15f539b439d..12dbfad4cfe 100644 --- a/processing/src/test/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java @@ -33,6 +33,7 @@ import io.druid.query.Druids; import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryContexts; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -137,7 +138,7 @@ public class DataSourceMetadataQueryTest Map context = new MapMaker().makeMap(); context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList()); Iterable> results = Sequences.toList( - runner.run(dataSourceMetadataQuery, context), + runner.run(QueryPlus.wrap(dataSourceMetadataQuery), context), Lists.>newArrayList() ); DataSourceMetadataResultValue val = results.iterator().next().getValue(); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java index cc91b7e6917..9611b0cff1e 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java @@ -335,7 +335,7 @@ public class GroupByMultiSegmentTest .setGranularity(Granularities.ALL) .build(); - Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); List results = Sequences.toList(queryResult, Lists.newArrayList()); Row expectedRow = GroupByQueryRunnerTestHelper.createExpectedRow( diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java index ed378aebbcf..1f3117ae3f1 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java @@ -117,7 +117,7 @@ public class GroupByQueryRunnerFactoryTest } ); - Sequence result = mergedRunner.run(query, Maps.newHashMap()); + Sequence result = mergedRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t1", "count", 2L), diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index ed07120a881..ddef2a474c7 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -30,7 +30,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; -import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; import io.druid.collections.BlockingPool; import io.druid.collections.DefaultBlockingPool; @@ -54,14 +53,12 @@ import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.DruidProcessingConfig; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; -import io.druid.query.Query; import io.druid.query.QueryContexts; import io.druid.query.QueryDataSource; import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryToolChest; -import io.druid.query.QueryWatcher; import io.druid.query.ResourceLimitExceededException; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; @@ -2523,7 +2520,7 @@ public class GroupByQueryRunnerTest ); Map context = Maps.newHashMap(); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged"); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged"); List allGranExpectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), @@ -2537,7 +2534,11 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) ); - TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, context), "merged"); + TestHelper.assertExpectedObjects( + allGranExpectedResults, + mergedRunner.run(QueryPlus.wrap(allGranQuery), context), + "merged" + ); } @Test @@ -2582,7 +2583,9 @@ public class GroupByQueryRunnerTest Map context = Maps.newHashMap(); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, context), StringUtils.format("limit: %d", limit) + Iterables.limit(expectedResults, limit), + mergeRunner.run(QueryPlus.wrap(fullQuery), context), + StringUtils.format("limit: %d", limit) ); } @@ -2629,7 +2632,9 @@ public class GroupByQueryRunnerTest Map context = Maps.newHashMap(); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, context), StringUtils.format("limit: %d", limit) + Iterables.limit(expectedResults, limit), + mergeRunner.run(QueryPlus.wrap(fullQuery), context), + StringUtils.format("limit: %d", limit) ); } @@ -2684,7 +2689,9 @@ public class GroupByQueryRunnerTest Map context = Maps.newHashMap(); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, context), StringUtils.format("limit: %d", limit) + Iterables.limit(expectedResults, limit), + mergeRunner.run(QueryPlus.wrap(fullQuery), context), + StringUtils.format("limit: %d", limit) ); } @@ -2812,7 +2819,7 @@ public class GroupByQueryRunnerTest ); Map context = Maps.newHashMap(); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged"); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged"); } @Test @@ -2849,10 +2856,12 @@ public class GroupByQueryRunnerTest Map context = Maps.newHashMap(); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit"); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, 5), mergeRunner.run(builder.setLimit(5).build(), context), "limited" + Iterables.limit(expectedResults, 5), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), + "limited" ); // Now try it with an expression based aggregator. @@ -2877,9 +2886,14 @@ public class GroupByQueryRunnerTest new Object[]{"2011-04-01", "mezzanine", 6L, 6635.47998046875D} ); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(builder.build(), context), "no-limit"); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, 5), mergeRunner.run(builder.setLimit(5).build(), context), "limited" + expectedResults, + mergeRunner.run(QueryPlus.wrap(builder.build()), context), + "no-limit"); + TestHelper.assertExpectedObjects( + Iterables.limit(expectedResults, 5), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), + "limited" ); // Now try it with an expression virtual column. @@ -2894,9 +2908,15 @@ public class GroupByQueryRunnerTest ) ); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(builder.build(), context), "no-limit"); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, 5), mergeRunner.run(builder.setLimit(5).build(), context), "limited" + expectedResults, + mergeRunner.run(QueryPlus.wrap(builder.build()), context), + "no-limit" + ); + TestHelper.assertExpectedObjects( + Iterables.limit(expectedResults, 5), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), + "limited" ); } @@ -2934,9 +2954,11 @@ public class GroupByQueryRunnerTest Map context = Maps.newHashMap(); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit"); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, 5), mergeRunner.run(builder.setLimit(5).build(), context), "limited" + Iterables.limit(expectedResults, 5), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), + "limited" ); } @@ -2975,9 +2997,11 @@ public class GroupByQueryRunnerTest Map context = Maps.newHashMap(); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit"); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, 5), mergeRunner.run(builder.setLimit(5).build(), context), "limited" + Iterables.limit(expectedResults, 5), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), + "limited" ); } @@ -3015,9 +3039,11 @@ public class GroupByQueryRunnerTest Map context = Maps.newHashMap(); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit"); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, 5), mergeRunner.run(builder.setLimit(5).build(), context), "limited" + Iterables.limit(expectedResults, 5), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context), + "limited" ); } @@ -3600,7 +3626,7 @@ public class GroupByQueryRunnerTest ); Map context = Maps.newHashMap(); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged"); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged"); } @Test @@ -3933,7 +3959,7 @@ public class GroupByQueryRunnerTest ); Map context = Maps.newHashMap(); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged"); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged"); } @Test @@ -4049,7 +4075,7 @@ public class GroupByQueryRunnerTest factory.getToolchest().mergeResults( factory.getToolchest().preMergeQueryDecoration(mergedRunner) ) - ).run(fullQuery, context), + ).run(QueryPlus.wrap(fullQuery), context), "merged" ); @@ -4065,7 +4091,7 @@ public class GroupByQueryRunnerTest factory.getToolchest().mergeResults( factory.getToolchest().preMergeQueryDecoration(mergedRunner) ) - ).run(fullQuery, context), + ).run(QueryPlus.wrap(fullQuery), context), "merged" ); } @@ -4094,7 +4120,7 @@ public class GroupByQueryRunnerTest QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); Map context = Maps.newHashMap(); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit"); } @Test @@ -4153,7 +4179,7 @@ public class GroupByQueryRunnerTest Map context = Maps.newHashMap(); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit"); } // A subquery identical to the query should yield identical results @@ -6623,7 +6649,11 @@ public class GroupByQueryRunnerTest ) ); - TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(fullQuery, Maps.newHashMap()), ""); + TestHelper.assertExpectedObjects( + bySegmentResults, + theRunner.run(QueryPlus.wrap(fullQuery), Maps.newHashMap()), + "" + ); exec.shutdownNow(); } @@ -6698,7 +6728,7 @@ public class GroupByQueryRunnerTest ) ); - TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(fullQuery, Maps.newHashMap()), ""); + TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(QueryPlus.wrap(fullQuery), Maps.newHashMap()), ""); exec.shutdownNow(); } @@ -6772,7 +6802,7 @@ public class GroupByQueryRunnerTest ) ); - TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(fullQuery, Maps.newHashMap()), ""); + TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(QueryPlus.wrap(fullQuery), Maps.newHashMap()), ""); exec.shutdownNow(); } @@ -7202,7 +7232,7 @@ public class GroupByQueryRunnerTest ) ); - TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(fullQuery, Maps.newHashMap()), ""); + TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(QueryPlus.wrap(fullQuery), Maps.newHashMap()), ""); exec.shutdownNow(); } @@ -8687,33 +8717,27 @@ public class GroupByQueryRunnerTest new QueryRunner() { @Override - public Sequence run( - Query query, Map responseContext - ) + public Sequence run(QueryPlus queryPlus, Map responseContext) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return factory.getToolchest().mergeResults( - new QueryRunner() - { - @Override - public Sequence run(Query query, Map responseContext) - { - return new MergeSequence( - query.getResultOrdering(), - Sequences.simple( - Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + (queryPlus3, responseContext1) -> new MergeSequence<>( + queryPlus3.getQuery().getResultOrdering(), + Sequences.simple( + Arrays.asList( + runner.run(queryPlus1, responseContext1), + runner.run(queryPlus2, responseContext1) ) - ); - } - } - ).run(query, responseContext); + ) + ) + ).run(queryPlus, responseContext); } } ); @@ -8726,7 +8750,11 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L) ); - TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, context), "merged"); + TestHelper.assertExpectedObjects( + allGranExpectedResults, + mergedRunner.run(QueryPlus.wrap(allGranQuery), context), + "merged" + ); } @Test @@ -8772,32 +8800,28 @@ public class GroupByQueryRunnerTest { @Override public Sequence run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return factory.getToolchest().mergeResults( - new QueryRunner() - { - @Override - public Sequence run(Query query, Map responseContext) - { - return new MergeSequence( - query.getResultOrdering(), - Sequences.simple( - Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + (queryPlus3, responseContext1) -> new MergeSequence<>( + queryPlus3.getQuery().getResultOrdering(), + Sequences.simple( + Arrays.asList( + runner.run(queryPlus1, responseContext1), + runner.run(queryPlus2, responseContext1) ) - ); - } - } - ).run(query, responseContext); + ) + ) + ).run(queryPlus, responseContext); } } ); @@ -8811,7 +8835,10 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) ); - Iterable results = Sequences.toList(mergedRunner.run(allGranQuery, context), Lists.newArrayList()); + Iterable results = Sequences.toList( + mergedRunner.run(QueryPlus.wrap(allGranQuery), context), + new ArrayList() + ); TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); } @@ -8866,32 +8893,28 @@ public class GroupByQueryRunnerTest { @Override public Sequence run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return factory.getToolchest().mergeResults( - new QueryRunner() - { - @Override - public Sequence run(Query query, Map responseContext) - { - return new MergeSequence( - query.getResultOrdering(), - Sequences.simple( - Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + (queryPlus3, responseContext1) -> new MergeSequence<>( + queryPlus3.getQuery().getResultOrdering(), + Sequences.simple( + Arrays.asList( + runner.run(queryPlus1, responseContext1), + runner.run(queryPlus2, responseContext1) ) - ); - } - } - ).run(query, responseContext); + ) + ) + ).run(queryPlus, responseContext); } } ); @@ -8905,7 +8928,10 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "spot", "rows", 2L, "idx", 257L) ); - Iterable results = Sequences.toList(mergedRunner.run(allGranQuery, context), Lists.newArrayList()); + Iterable results = Sequences.toList( + mergedRunner.run(QueryPlus.wrap(allGranQuery), context), + Lists.newArrayList() + ); TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); } @@ -8964,32 +8990,28 @@ public class GroupByQueryRunnerTest { @Override public Sequence run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return factory.getToolchest().mergeResults( - new QueryRunner() - { - @Override - public Sequence run(Query query, Map responseContext) - { - return new MergeSequence( - query.getResultOrdering(), - Sequences.simple( - Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + (queryPlus3, responseContext1) -> new MergeSequence<>( + queryPlus3.getQuery().getResultOrdering(), + Sequences.simple( + Arrays.asList( + runner.run(queryPlus1, responseContext1), + runner.run(queryPlus2, responseContext1) ) - ); - } - } - ).run(query, responseContext); + ) + ) + ).run(queryPlus, responseContext); } } ); @@ -9003,7 +9025,10 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "spot", "rows", 2L, "idx", 257L) ); - Iterable results = Sequences.toList(mergedRunner.run(allGranQuery, context), Lists.newArrayList()); + Iterable results = Sequences.toList( + mergedRunner.run(QueryPlus.wrap(allGranQuery), context), + Lists.newArrayList() + ); TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); } @@ -9281,13 +9306,8 @@ public class GroupByQueryRunnerTest ChainedExecutionQueryRunner ceqr = new ChainedExecutionQueryRunner( MoreExecutors.sameThreadExecutor(), - new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - return; - } + (query1, future) -> { + return; }, ImmutableList.>of(runner, runner) ); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java index e6a7dc39d7b..f8f98e1280a 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java @@ -28,6 +28,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -51,7 +52,7 @@ public class GroupByQueryRunnerTestHelper toolChest ); - Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap()); return Sequences.toList(queryResult, Lists.newArrayList()); } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index 4c7097fd4e2..21e956594c8 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -90,20 +90,20 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest toolChest ); + GroupByQuery newQuery = GroupByQuery + .builder() + .setDataSource(tsQuery.getDataSource()) + .setQuerySegmentSpec(tsQuery.getQuerySegmentSpec()) + .setGranularity(tsQuery.getGranularity()) + .setDimFilter(tsQuery.getDimensionsFilter()) + .setAggregatorSpecs(tsQuery.getAggregatorSpecs()) + .setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs()) + .setVirtualColumns(tsQuery.getVirtualColumns()) + .setContext(tsQuery.getContext()) + .build(); + return Sequences.map( - newRunner.run( - GroupByQuery.builder() - .setDataSource(tsQuery.getDataSource()) - .setQuerySegmentSpec(tsQuery.getQuerySegmentSpec()) - .setGranularity(tsQuery.getGranularity()) - .setDimFilter(tsQuery.getDimensionsFilter()) - .setAggregatorSpecs(tsQuery.getAggregatorSpecs()) - .setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs()) - .setVirtualColumns(tsQuery.getVirtualColumns()) - .setContext(tsQuery.getContext()) - .build(), - responseContext - ), + newRunner.run(queryPlus.withQuery(newQuery), responseContext), new Function>() { @Override @@ -158,7 +158,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest DateTime expectedLast = new DateTime("2011-04-15"); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); Result result = results.iterator().next(); diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java index 1d55b8a0719..bda2351c91f 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.Lists; import io.druid.data.input.impl.DimensionSchema; import io.druid.java.util.common.guava.Sequences; import io.druid.query.LegacyDataSource; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -194,6 +195,6 @@ public class SegmentAnalyzerTest new LegacyDataSource("test"), QuerySegmentSpecs.create("2011/2012"), null, null, null, analyses, false, false ); HashMap context = new HashMap(); - return Sequences.toList(query.run(runner, context), Lists.newArrayList()); + return Sequences.toList(runner.run(QueryPlus.wrap(query), context), Lists.newArrayList()); } } diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java index d8d2d3ca425..899f65ad2b5 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -36,6 +36,7 @@ import io.druid.query.BySegmentResultValueClass; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -261,7 +262,7 @@ public class SegmentMetadataQueryTest public void testSegmentMetadataQuery() { List results = Sequences.toList( - runner1.run(testQuery, Maps.newHashMap()), + runner1.run(QueryPlus.wrap(testQuery), Maps.newHashMap()), Lists.newArrayList() ); @@ -320,18 +321,17 @@ public class SegmentMetadataQueryTest toolChest ); + SegmentMetadataQuery query = Druids + .newSegmentMetadataQueryBuilder() + .dataSource("testing") + .intervals("2013/2014") + .toInclude(new ListColumnIncluderator(Arrays.asList("placement", "placementish"))) + .analysisTypes(SegmentMetadataQuery.AnalysisType.ROLLUP) + .merge(true) + .build(); TestHelper.assertExpectedObjects( ImmutableList.of(mergedSegmentAnalysis), - myRunner.run( - Druids.newSegmentMetadataQueryBuilder() - .dataSource("testing") - .intervals("2013/2014") - .toInclude(new ListColumnIncluderator(Arrays.asList("placement", "placementish"))) - .analysisTypes(SegmentMetadataQuery.AnalysisType.ROLLUP) - .merge(true) - .build(), - Maps.newHashMap() - ), + myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()), "failed SegmentMetadata merging query" ); exec.shutdownNow(); @@ -389,18 +389,17 @@ public class SegmentMetadataQueryTest toolChest ); + SegmentMetadataQuery query = Druids + .newSegmentMetadataQueryBuilder() + .dataSource("testing") + .intervals("2013/2014") + .toInclude(new ListColumnIncluderator(Arrays.asList("placement", "placementish"))) + .analysisTypes(SegmentMetadataQuery.AnalysisType.CARDINALITY) + .merge(true) + .build(); TestHelper.assertExpectedObjects( ImmutableList.of(mergedSegmentAnalysis), - myRunner.run( - Druids.newSegmentMetadataQueryBuilder() - .dataSource("testing") - .intervals("2013/2014") - .toInclude(new ListColumnIncluderator(Arrays.asList("placement", "placementish"))) - .analysisTypes(SegmentMetadataQuery.AnalysisType.CARDINALITY) - .merge(true) - .build(), - Maps.newHashMap() - ), + myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()), "failed SegmentMetadata merging query" ); exec.shutdownNow(); @@ -458,18 +457,17 @@ public class SegmentMetadataQueryTest toolChest ); + SegmentMetadataQuery query = Druids + .newSegmentMetadataQueryBuilder() + .dataSource("testing") + .intervals("2013/2014") + .toInclude(new ListColumnIncluderator(Arrays.asList("placement", "quality_uniques"))) + .analysisTypes(SegmentMetadataQuery.AnalysisType.CARDINALITY) + .merge(true) + .build(); TestHelper.assertExpectedObjects( ImmutableList.of(mergedSegmentAnalysis), - myRunner.run( - Druids.newSegmentMetadataQueryBuilder() - .dataSource("testing") - .intervals("2013/2014") - .toInclude(new ListColumnIncluderator(Arrays.asList("placement", "quality_uniques"))) - .analysisTypes(SegmentMetadataQuery.AnalysisType.CARDINALITY) - .merge(true) - .build(), - Maps.newHashMap() - ), + myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()), "failed SegmentMetadata merging query" ); exec.shutdownNow(); @@ -580,7 +578,7 @@ public class SegmentMetadataQueryTest TestHelper.assertExpectedObjects( ImmutableList.of(mergedSegmentAnalysis), - myRunner.run(query, Maps.newHashMap()), + myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()), "failed SegmentMetadata merging query" ); exec.shutdownNow(); @@ -628,18 +626,17 @@ public class SegmentMetadataQueryTest toolChest ); + SegmentMetadataQuery query = Druids + .newSegmentMetadataQueryBuilder() + .dataSource("testing") + .intervals("2013/2014") + .toInclude(new ListColumnIncluderator(Arrays.asList("placement"))) + .analysisTypes() + .merge(true) + .build(); TestHelper.assertExpectedObjects( ImmutableList.of(mergedSegmentAnalysis), - myRunner.run( - Druids.newSegmentMetadataQueryBuilder() - .dataSource("testing") - .intervals("2013/2014") - .toInclude(new ListColumnIncluderator(Arrays.asList("placement"))) - .analysisTypes() - .merge(true) - .build(), - Maps.newHashMap() - ), + myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()), "failed SegmentMetadata merging query" ); exec.shutdownNow(); @@ -691,18 +688,17 @@ public class SegmentMetadataQueryTest toolChest ); + SegmentMetadataQuery query = Druids + .newSegmentMetadataQueryBuilder() + .dataSource("testing") + .intervals("2013/2014") + .toInclude(new ListColumnIncluderator(Arrays.asList("placement"))) + .analysisTypes(SegmentMetadataQuery.AnalysisType.AGGREGATORS) + .merge(true) + .build(); TestHelper.assertExpectedObjects( ImmutableList.of(mergedSegmentAnalysis), - myRunner.run( - Druids.newSegmentMetadataQueryBuilder() - .dataSource("testing") - .intervals("2013/2014") - .toInclude(new ListColumnIncluderator(Arrays.asList("placement"))) - .analysisTypes(SegmentMetadataQuery.AnalysisType.AGGREGATORS) - .merge(true) - .build(), - Maps.newHashMap() - ), + myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()), "failed SegmentMetadata merging query" ); exec.shutdownNow(); @@ -750,18 +746,17 @@ public class SegmentMetadataQueryTest toolChest ); + SegmentMetadataQuery query = Druids + .newSegmentMetadataQueryBuilder() + .dataSource("testing") + .intervals("2013/2014") + .toInclude(new ListColumnIncluderator(Arrays.asList("placement"))) + .analysisTypes(SegmentMetadataQuery.AnalysisType.TIMESTAMPSPEC) + .merge(true) + .build(); TestHelper.assertExpectedObjects( ImmutableList.of(mergedSegmentAnalysis), - myRunner.run( - Druids.newSegmentMetadataQueryBuilder() - .dataSource("testing") - .intervals("2013/2014") - .toInclude(new ListColumnIncluderator(Arrays.asList("placement"))) - .analysisTypes(SegmentMetadataQuery.AnalysisType.TIMESTAMPSPEC) - .merge(true) - .build(), - Maps.newHashMap() - ), + myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()), "failed SegmentMetadata merging query" ); exec.shutdownNow(); @@ -809,18 +804,17 @@ public class SegmentMetadataQueryTest toolChest ); + SegmentMetadataQuery query = Druids + .newSegmentMetadataQueryBuilder() + .dataSource("testing") + .intervals("2013/2014") + .toInclude(new ListColumnIncluderator(Arrays.asList("placement"))) + .analysisTypes(SegmentMetadataQuery.AnalysisType.QUERYGRANULARITY) + .merge(true) + .build(); TestHelper.assertExpectedObjects( ImmutableList.of(mergedSegmentAnalysis), - myRunner.run( - Druids.newSegmentMetadataQueryBuilder() - .dataSource("testing") - .intervals("2013/2014") - .toInclude(new ListColumnIncluderator(Arrays.asList("placement"))) - .analysisTypes(SegmentMetadataQuery.AnalysisType.QUERYGRANULARITY) - .merge(true) - .build(), - Maps.newHashMap() - ), + myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()), "failed SegmentMetadata merging query" ); exec.shutdownNow(); @@ -858,7 +852,7 @@ public class SegmentMetadataQueryTest TestHelper.assertExpectedObjects( ImmutableList.of(bySegmentResult, bySegmentResult), myRunner.run( - testQuery.withOverriddenContext(ImmutableMap.of("bySegment", true)), + QueryPlus.wrap(testQuery.withOverriddenContext(ImmutableMap.of("bySegment", true))), Maps.newHashMap() ), "failed SegmentMetadata bySegment query" diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java index 8e3ad079db4..d2811e0bbd3 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -128,7 +129,10 @@ public class SegmentMetadataUnionQueryTest SegmentMetadataQuery.AnalysisType.MINMAX ) .build(); - List result = Sequences.toList(runner.run(query, Maps.newHashMap()), Lists.newArrayList()); + List result = Sequences.toList( + runner.run(QueryPlus.wrap(query), Maps.newHashMap()), + Lists.newArrayList() + ); TestHelper.assertExpectedObjects(ImmutableList.of(expected), result, "failed SegmentMetadata union query"); } diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java index b1b9254f3dd..1cd5b15374b 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -815,7 +815,7 @@ public class SearchQueryRunnerTest private void checkSearchQuery(Query searchQuery, QueryRunner runner, List expectedResults) { Iterable> results = Sequences.toList( - runner.run(searchQuery, ImmutableMap.of()), + runner.run(QueryPlus.wrap(searchQuery), ImmutableMap.of()), Lists.>newArrayList() ); List copy = Lists.newLinkedList(expectedResults); diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerWithCaseTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerWithCaseTest.java index 59bae93554c..231d9103c27 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerWithCaseTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerWithCaseTest.java @@ -27,6 +27,7 @@ import com.google.common.io.CharSource; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.Result; import io.druid.query.search.search.AutoStrategy; @@ -243,7 +244,7 @@ public class SearchQueryRunnerWithCaseTest { HashMap context = new HashMap<>(); Iterable> results = Sequences.toList( - runner.run(searchQuery, context), + runner.run(QueryPlus.>wrap(searchQuery), context), Lists.>newArrayList() ); diff --git a/processing/src/test/java/io/druid/query/select/MultiSegmentSelectQueryTest.java b/processing/src/test/java/io/druid/query/select/MultiSegmentSelectQueryTest.java index c29ea33dfc7..9930097e2d1 100644 --- a/processing/src/test/java/io/druid/query/select/MultiSegmentSelectQueryTest.java +++ b/processing/src/test/java/io/druid/query/select/MultiSegmentSelectQueryTest.java @@ -29,6 +29,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -248,7 +249,7 @@ public class MultiSegmentSelectQueryTest { for (int[] expected : expectedOffsets) { List> results = Sequences.toList( - runner.run(query, ImmutableMap.of()), + runner.run(QueryPlus.wrap(query), ImmutableMap.of()), Lists.>newArrayList() ); Assert.assertEquals(1, results.size()); @@ -295,7 +296,7 @@ public class MultiSegmentSelectQueryTest { for (int[] expected : expectedOffsets) { List> results = Sequences.toList( - runner.run(query, ImmutableMap.of()), + runner.run(QueryPlus.wrap(query), ImmutableMap.of()), Lists.>newArrayList() ); Assert.assertEquals(2, results.size()); @@ -340,14 +341,17 @@ public class MultiSegmentSelectQueryTest QueryRunner unionQueryRunner = new UnionQueryRunner(runner); List> results = Sequences.toList( - unionQueryRunner.run(query, ImmutableMap.of()), + unionQueryRunner.run(QueryPlus.wrap(query), ImmutableMap.of()), Lists.>newArrayList() ); Map pagingIdentifiers = results.get(0).getValue().getPagingIdentifiers(); query = query.withPagingSpec(toNextCursor(PagingSpec.merge(Arrays.asList(pagingIdentifiers)), query, 3)); - Sequences.toList(unionQueryRunner.run(query, ImmutableMap.of()), Lists.>newArrayList()); + Sequences.toList( + unionQueryRunner.run(QueryPlus.wrap(query), ImmutableMap.of()), + Lists.>newArrayList() + ); } private PagingSpec toNextCursor(Map merged, SelectQuery query, int threshold) diff --git a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java index 4c6a4622261..0fdeb9e9d65 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -33,6 +33,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequences; import io.druid.js.JavaScriptConfig; import io.druid.query.Druids; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -169,7 +170,7 @@ public class SelectQueryRunnerTest HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); @@ -213,7 +214,7 @@ public class SelectQueryRunnerTest SelectQuery query = newTestQuery().intervals(I_0112_0114).build(); for (int offset : expected) { List> results = Sequences.toList( - runner.run(query, ImmutableMap.of()), + runner.run(QueryPlus.wrap(query), ImmutableMap.of()), Lists.>newArrayList() ); @@ -230,7 +231,7 @@ public class SelectQueryRunnerTest query = newTestQuery().intervals(I_0112_0114).build(); for (int offset : expected) { List> results = Sequences.toList( - runner.run(query, ImmutableMap.of()), + runner.run(QueryPlus.wrap(query), ImmutableMap.of()), Lists.>newArrayList() ); @@ -275,7 +276,7 @@ public class SelectQueryRunnerTest HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); @@ -385,7 +386,7 @@ public class SelectQueryRunnerTest HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); @@ -424,7 +425,7 @@ public class SelectQueryRunnerTest .build(); Iterable> results = Sequences.toList( - runner.run(query, Maps.newHashMap()), + runner.run(QueryPlus.wrap(query), Maps.newHashMap()), Lists.>newArrayList() ); @@ -462,7 +463,7 @@ public class SelectQueryRunnerTest HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); @@ -536,7 +537,7 @@ public class SelectQueryRunnerTest HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); @@ -584,13 +585,13 @@ public class SelectQueryRunnerTest .build(); Iterable> results = Sequences.toList( - runner.run(query, Maps.newHashMap()), + runner.run(QueryPlus.wrap(query), Maps.newHashMap()), Lists.>newArrayList() ); Iterable> resultsOptimize = Sequences.toList( toolChest .postMergeQueryDecoration(toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner))) - .run(query, Maps.newHashMap()), Lists.>newArrayList() + .run(QueryPlus.wrap(query), Maps.newHashMap()), Lists.>newArrayList() ); final List>> events = toEvents( @@ -642,7 +643,7 @@ public class SelectQueryRunnerTest .build(); Iterable> results = Sequences.toList( - runner.run(query, Maps.newHashMap()), + runner.run(QueryPlus.wrap(query), Maps.newHashMap()), Lists.>newArrayList() ); @@ -690,7 +691,7 @@ public class SelectQueryRunnerTest .build(); Iterable> results = Sequences.toList( - runner.run(query, Maps.newHashMap()), + runner.run(QueryPlus.wrap(query), Maps.newHashMap()), Lists.>newArrayList() ); @@ -730,7 +731,7 @@ public class SelectQueryRunnerTest HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); @@ -848,7 +849,7 @@ public class SelectQueryRunnerTest HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); diff --git a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java index 76564445abc..76591d64405 100644 --- a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java @@ -107,13 +107,13 @@ public class SpecificSegmentQueryRunnerTest ) ) .build(); - Sequence results = queryRunner.run(query, responseContext); + Sequence results = queryRunner.run(QueryPlus.wrap(query), responseContext); Sequences.toList(results, Lists.newArrayList()); validate(mapper, descriptor, responseContext); // from toYielder responseContext = Maps.newHashMap(); - results = queryRunner.run(query, responseContext); + results = queryRunner.run(QueryPlus.wrap(query), responseContext); results.toYielder( null, new YieldingAccumulator() { @@ -184,10 +184,7 @@ public class SpecificSegmentQueryRunnerTest ) ) .build(); - Sequence results = queryRunner.run( - query, - responseContext - ); + Sequence results = queryRunner.run(QueryPlus.wrap(query), responseContext); List> res = Sequences.toList( results, Lists.>newArrayList() diff --git a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index bffcb69c19b..7e7273ab375 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -27,6 +27,7 @@ import com.google.common.io.CharSource; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -175,7 +176,7 @@ public class TimeBoundaryQueryRunnerTest Assert.assertTrue(timeBoundaryQuery.hasFilters()); HashMap context = new HashMap(); Iterable> results = Sequences.toList( - customRunner.run(timeBoundaryQuery, context), + customRunner.run(QueryPlus.wrap(timeBoundaryQuery), context), Lists.>newArrayList() ); @@ -201,7 +202,7 @@ public class TimeBoundaryQueryRunnerTest Assert.assertTrue(timeBoundaryQuery.hasFilters()); HashMap context = new HashMap(); Iterable> results = Sequences.toList( - customRunner.run(timeBoundaryQuery, context), + customRunner.run(QueryPlus.wrap(timeBoundaryQuery), context), Lists.>newArrayList() ); @@ -218,7 +219,7 @@ public class TimeBoundaryQueryRunnerTest Assert.assertFalse(timeBoundaryQuery.hasFilters()); HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(timeBoundaryQuery, context), + runner.run(QueryPlus.wrap(timeBoundaryQuery), context), Lists.>newArrayList() ); TimeBoundaryResultValue val = results.iterator().next().getValue(); @@ -240,7 +241,7 @@ public class TimeBoundaryQueryRunnerTest Map context = new MapMaker().makeMap(); context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList()); Iterable> results = Sequences.toList( - runner.run(timeBoundaryQuery, context), + runner.run(QueryPlus.wrap(timeBoundaryQuery), context), Lists.>newArrayList() ); TimeBoundaryResultValue val = results.iterator().next().getValue(); @@ -262,7 +263,7 @@ public class TimeBoundaryQueryRunnerTest Map context = new MapMaker().makeMap(); context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList()); Iterable> results = Sequences.toList( - runner.run(timeBoundaryQuery, context), + runner.run(QueryPlus.wrap(timeBoundaryQuery), context), Lists.>newArrayList() ); TimeBoundaryResultValue val = results.iterator().next().getValue(); diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java index 69f4f635582..80464169b61 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -122,7 +122,7 @@ public class TimeSeriesUnionQueryRunnerTest ); HashMap context = new HashMap<>(); Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); @@ -228,7 +228,7 @@ public class TimeSeriesUnionQueryRunnerTest ); Iterable> results = Sequences.toList( - mergingrunner.run(query, Maps.newHashMap()), + mergingrunner.run(QueryPlus.wrap(query), Maps.newHashMap()), Lists.>newArrayList() ); diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java index a4f86fb516e..526751ba929 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -28,6 +28,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -138,7 +139,7 @@ public class TimeseriesQueryRunnerBonusTest .build(); HashMap context = new HashMap(); return Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); } diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java index a2bf79fd167..2c98d74facf 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -31,6 +31,7 @@ import io.druid.java.util.common.granularity.PeriodGranularity; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryToolChest; @@ -70,6 +71,7 @@ import org.junit.runners.Parameterized; import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -156,7 +158,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -177,12 +179,13 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.qualityUniques ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); @@ -248,7 +251,7 @@ public class TimeseriesQueryRunnerTest .build(); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); @@ -290,7 +293,7 @@ public class TimeseriesQueryRunnerTest DateTime expectedLast = new DateTime("2011-04-15"); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); Result result = results.iterator().next(); @@ -338,7 +341,7 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.last; Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); @@ -403,7 +406,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); @@ -451,7 +454,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); @@ -499,7 +502,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); @@ -513,7 +516,7 @@ public class TimeseriesQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(new PeriodGranularity(new Period("P1M"), null, null)) .intervals( - Arrays.asList( + Collections.singletonList( new Interval( "2011-04-02T00:00:00.000Z/2011-04-03T00:00:00.000Z" ) @@ -532,7 +535,7 @@ public class TimeseriesQueryRunnerTest .descending(descending) .build(); - List> expectedResults1 = Arrays.asList( + List> expectedResults1 = Collections.singletonList( new Result<>( new DateTime("2011-04-01"), new TimeseriesResultValue( @@ -542,7 +545,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results1 = Sequences.toList( - runner.run(query1, CONTEXT), + runner.run(QueryPlus.wrap(query1), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults1, results1); @@ -551,7 +554,7 @@ public class TimeseriesQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity("DAY") .intervals( - Arrays.asList( + Collections.singletonList( new Interval( "2011-04-02T00:00:00.000Z/2011-04-03T00:00:00.000Z" ) @@ -569,7 +572,7 @@ public class TimeseriesQueryRunnerTest ) .build(); - List> expectedResults2 = Arrays.asList( + List> expectedResults2 = Collections.singletonList( new Result<>( new DateTime("2011-04-02"), new TimeseriesResultValue( @@ -579,7 +582,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results2 = Sequences.toList( - runner.run(query2, CONTEXT), + runner.run(QueryPlus.wrap(query2), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults2, results2); @@ -599,7 +602,7 @@ public class TimeseriesQueryRunnerTest ) ) .intervals( - Arrays.asList( + Collections.singletonList( new Interval( "2011-01-12T00:00:00.000-08:00/2011-01-20T00:00:00.000-08:00" ) @@ -633,7 +636,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results1 = Sequences.toList( - runner.run(query1, CONTEXT), + runner.run(QueryPlus.wrap(query1), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults1, results1); @@ -647,7 +650,7 @@ public class TimeseriesQueryRunnerTest .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity(Granularities.HOUR) .intervals( - Arrays.asList( + Collections.singletonList( new Interval( "2011-04-14T00:00:00.000Z/2011-05-01T00:00:00.000Z" ) @@ -680,7 +683,7 @@ public class TimeseriesQueryRunnerTest List> expectedResults1 = Lists.newArrayList( Iterables.concat( - Arrays.asList( + Collections.singletonList( new Result<>( new DateTime("2011-04-14T00"), new TimeseriesResultValue( @@ -689,7 +692,7 @@ public class TimeseriesQueryRunnerTest ) ), lotsOfZeroes, - Arrays.asList( + Collections.singletonList( new Result<>( new DateTime("2011-04-15T00"), new TimeseriesResultValue( @@ -701,7 +704,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results1 = Sequences.toList( - runner.run(query1, CONTEXT), + runner.run(QueryPlus.wrap(query1), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults1, results1); @@ -721,7 +724,7 @@ public class TimeseriesQueryRunnerTest ) ) .intervals( - Arrays.asList( + Collections.singletonList( new Interval( "2011-04-15T00:00:00.000Z/2012" ) @@ -739,7 +742,7 @@ public class TimeseriesQueryRunnerTest .descending(descending) .build(); - List> expectedResults1 = Arrays.asList( + List> expectedResults1 = Collections.singletonList( new Result<>( new DateTime("2011-04-14T23:01Z"), new TimeseriesResultValue( @@ -749,7 +752,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results1 = Sequences.toList( - runner.run(query1, CONTEXT), + runner.run(QueryPlus.wrap(query1), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults1, results1); @@ -763,7 +766,7 @@ public class TimeseriesQueryRunnerTest .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity(new PeriodGranularity(new Period("P1M"), null, null)) .intervals( - Arrays.asList( + Collections.singletonList( new Interval( "2011-04-02T00:00:00.000Z/2011-04-03T00:00:00.000Z" ) @@ -782,7 +785,7 @@ public class TimeseriesQueryRunnerTest .descending(descending) .build(); - List> expectedResults1 = Arrays.asList( + List> expectedResults1 = Collections.singletonList( new Result<>( new DateTime("2011-04-01"), new TimeseriesResultValue( @@ -791,7 +794,7 @@ public class TimeseriesQueryRunnerTest ) ); Iterable> results1 = Sequences.toList( - runner.run(query1, CONTEXT), + runner.run(QueryPlus.wrap(query1), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults1, results1); @@ -801,7 +804,7 @@ public class TimeseriesQueryRunnerTest .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity("DAY") .intervals( - Arrays.asList( + Collections.singletonList( new Interval( "2011-04-02T00:00:00.000Z/2011-04-03T00:00:00.000Z" ) @@ -819,7 +822,7 @@ public class TimeseriesQueryRunnerTest ) .build(); - List> expectedResults2 = Arrays.asList( + List> expectedResults2 = Collections.singletonList( new Result<>( new DateTime("2011-04-02"), new TimeseriesResultValue( @@ -829,7 +832,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results2 = Sequences.toList( - runner.run(query2, CONTEXT), + runner.run(QueryPlus.wrap(query2), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults2, results2); @@ -843,7 +846,7 @@ public class TimeseriesQueryRunnerTest .granularity(QueryRunnerTestHelper.dayGran) .intervals( new MultipleIntervalSegmentSpec( - Arrays.asList( + Collections.singletonList( new Interval( "2015-01-01/2015-01-10" ) @@ -862,10 +865,10 @@ public class TimeseriesQueryRunnerTest .descending(descending) .build(); - List> expectedResults = Arrays.asList(); + List> expectedResults = Collections.emptyList(); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -886,7 +889,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.qualityUniques ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -916,7 +920,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -937,7 +941,7 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.qualityUniques ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) .descending(descending) .build(); @@ -967,7 +971,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -988,7 +992,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.qualityUniques ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1018,7 +1023,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1039,7 +1044,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.qualityUniques ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1069,7 +1075,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1090,7 +1096,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.qualityUniques ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1120,7 +1127,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1149,7 +1156,8 @@ public class TimeseriesQueryRunnerTest .filters(andDimFilter) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1179,7 +1187,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1208,7 +1216,8 @@ public class TimeseriesQueryRunnerTest .filters(andDimFilter) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1238,7 +1247,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1267,7 +1276,8 @@ public class TimeseriesQueryRunnerTest .filters(andDimFilter) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonDoubleAggregators) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1297,7 +1307,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1324,7 +1334,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.qualityUniques ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1354,7 +1365,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1387,7 +1398,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.qualityUniques ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1417,7 +1429,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1450,7 +1462,8 @@ public class TimeseriesQueryRunnerTest .filters(andDimFilter) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1480,7 +1493,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1495,7 +1508,8 @@ public class TimeseriesQueryRunnerTest .filters("bobby", "billy") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1525,7 +1539,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1540,15 +1554,16 @@ public class TimeseriesQueryRunnerTest .filters("bobby", "billy") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .context(ImmutableMap.of("skipEmptyBuckets", "true")) .descending(descending) .build(); - List> expectedResults = Arrays.asList(); + List> expectedResults = Collections.emptyList(); Iterable> results = Sequences.toList( - runner.run(query, new HashMap()), + runner.run(QueryPlus.wrap(query), new HashMap()), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1563,7 +1578,8 @@ public class TimeseriesQueryRunnerTest .filters("bobby", null) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1593,7 +1609,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, new HashMap()), + runner.run(QueryPlus.wrap(query), new HashMap()), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1608,7 +1624,8 @@ public class TimeseriesQueryRunnerTest .filters(new NotDimFilter(new SelectorDimFilter("bobby", "sally", null))) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1638,7 +1655,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, new HashMap()), + runner.run(QueryPlus.wrap(query), new HashMap()), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1653,7 +1670,8 @@ public class TimeseriesQueryRunnerTest .filters(QueryRunnerTestHelper.marketDimension, "billy") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1683,7 +1701,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1712,7 +1730,8 @@ public class TimeseriesQueryRunnerTest .filters(andDimFilter) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); @@ -1742,7 +1761,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, results); @@ -1783,7 +1802,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, actualResults); @@ -1825,7 +1844,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); assertExpectedResults(expectedResults, actualResults); @@ -1929,7 +1948,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); if (descending) { @@ -1942,32 +1961,32 @@ public class TimeseriesQueryRunnerTest @Test public void testTimeseriesWithMultiValueDimFilter1() { - TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() - .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.placementishDimension, "preferred") - .intervals(QueryRunnerTestHelper.firstToThird) - .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .descending(descending) - .build(); + TimeseriesQuery query = Druids + .newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .filters(QueryRunnerTestHelper.placementishDimension, "preferred") + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(aggregatorFactoryList) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .descending(descending) + .build(); + TimeseriesQuery query1 = Druids + .newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(aggregatorFactoryList) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .descending(descending) + .build(); Iterable> expectedResults = Sequences.toList( - runner.run( - Druids.newTimeseriesQueryBuilder() - .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryRunnerTestHelper.dayGran) - .intervals(QueryRunnerTestHelper.firstToThird) - .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .descending(descending) - .build(), - CONTEXT - ), + runner.run(QueryPlus.wrap(query1), CONTEXT), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1976,33 +1995,33 @@ public class TimeseriesQueryRunnerTest @Test public void testTimeseriesWithMultiValueDimFilter2() { - TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() - .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.placementishDimension, "a") - .intervals(QueryRunnerTestHelper.firstToThird) - .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .descending(descending) - .build(); + TimeseriesQuery query = Druids + .newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .filters(QueryRunnerTestHelper.placementishDimension, "a") + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(aggregatorFactoryList) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .descending(descending) + .build(); + TimeseriesQuery query1 = Druids + .newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .filters(QueryRunnerTestHelper.qualityDimension, "automotive") + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(aggregatorFactoryList) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .descending(descending) + .build(); Iterable> expectedResults = Sequences.toList( - runner.run( - Druids.newTimeseriesQueryBuilder() - .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.qualityDimension, "automotive") - .intervals(QueryRunnerTestHelper.firstToThird) - .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .descending(descending) - .build(), - CONTEXT - ), + runner.run(QueryPlus.wrap(query1), CONTEXT), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -2025,15 +2044,16 @@ public class TimeseriesQueryRunnerTest ) ) .build(); - TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() - .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryRunnerTestHelper.dayGran) - .filters(andDimFilter) - .intervals(QueryRunnerTestHelper.firstToThird) - .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .descending(descending) - .build(); + TimeseriesQuery query = Druids + .newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .filters(andDimFilter) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(aggregatorFactoryList) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .descending(descending) + .build(); AndDimFilter andDimFilter2 = Druids.newAndDimFilterBuilder() .fields( @@ -2050,23 +2070,22 @@ public class TimeseriesQueryRunnerTest ) .build(); + TimeseriesQuery query2 = Druids + .newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .filters(andDimFilter2) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(aggregatorFactoryList) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .descending(descending) + .build(); Iterable> expectedResults = Sequences.toList( - runner.run( - Druids.newTimeseriesQueryBuilder() - .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryRunnerTestHelper.dayGran) - .filters(andDimFilter2) - .intervals(QueryRunnerTestHelper.firstToThird) - .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .descending(descending) - .build(), - CONTEXT - ), + runner.run(QueryPlus.wrap(query2), CONTEXT), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -2088,15 +2107,16 @@ public class TimeseriesQueryRunnerTest ) ) .build(); - TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() - .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryRunnerTestHelper.dayGran) - .filters(andDimFilter) - .intervals(QueryRunnerTestHelper.firstToThird) - .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .descending(descending) - .build(); + TimeseriesQuery query = Druids + .newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .filters(andDimFilter) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(aggregatorFactoryList) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .descending(descending) + .build(); AndDimFilter andDimFilter2 = Druids.newAndDimFilterBuilder() .fields( @@ -2116,23 +2136,22 @@ public class TimeseriesQueryRunnerTest ) .build(); + TimeseriesQuery query2 = Druids + .newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .filters(andDimFilter2) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(aggregatorFactoryList) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .descending(descending) + .build(); Iterable> expectedResults = Sequences.toList( - runner.run( - Druids.newTimeseriesQueryBuilder() - .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryRunnerTestHelper.dayGran) - .filters(andDimFilter2) - .intervals(QueryRunnerTestHelper.firstToThird) - .aggregators(aggregatorFactoryList) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .descending(descending) - .build(), - CONTEXT - ), + runner.run(QueryPlus.wrap(query2), CONTEXT), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -2161,15 +2180,16 @@ public class TimeseriesQueryRunnerTest ) ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); - List> expectedResults = Arrays.asList( + List> expectedResults = Collections.singletonList( new Result<>( new DateTime("2011-04-01"), new TimeseriesResultValue( @@ -2210,16 +2230,17 @@ public class TimeseriesQueryRunnerTest ) ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); - List> expectedResults = Arrays.asList( + List> expectedResults = Collections.singletonList( new Result<>( new DateTime("2011-04-01"), new TimeseriesResultValue( @@ -2260,16 +2281,17 @@ public class TimeseriesQueryRunnerTest ) ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); - List> expectedResults = Arrays.asList( + List> expectedResults = Collections.singletonList( new Result<>( new DateTime("2011-04-01"), new TimeseriesResultValue( @@ -2312,15 +2334,16 @@ public class TimeseriesQueryRunnerTest ) ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); - List> expectedResults = Arrays.asList( + List> expectedResults = Collections.singletonList( new Result<>( new DateTime("2011-04-01"), new TimeseriesResultValue( @@ -2363,15 +2386,16 @@ public class TimeseriesQueryRunnerTest ) ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .descending(descending) .build(); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); - List> expectedResults = Arrays.asList( + List> expectedResults = Collections.singletonList( new Result<>( new DateTime("2011-04-01"), new TimeseriesResultValue( @@ -2406,7 +2430,7 @@ public class TimeseriesQueryRunnerTest .descending(descending) .build(); - List> expectedResults = Arrays.asList( + List> expectedResults = Collections.singletonList( new Result<>( new DateTime("2011-04-01"), new TimeseriesResultValue( @@ -2423,7 +2447,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> actualResults = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); @@ -2480,7 +2504,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.qualityUniques ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -2509,7 +2534,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -2537,7 +2562,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.qualityUniques ) ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper + .addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -2566,7 +2592,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query, CONTEXT), + runner.run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -2577,7 +2603,7 @@ public class TimeseriesQueryRunnerTest QueryRunner> optimizedRunner = toolChest.postMergeQueryDecoration( toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner))); Iterable> results2 = Sequences.toList( - new FinalizeResultsQueryRunner(optimizedRunner, toolChest).run(query, CONTEXT), + new FinalizeResultsQueryRunner(optimizedRunner, toolChest).run(QueryPlus.wrap(query), CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results2); diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java index 3bfafeb82a5..43a8dace166 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java @@ -215,19 +215,21 @@ public class TopNQueryQueryToolChestTest TopNQuery query1 = builder.threshold(10).context(null).build(); MockQueryRunner mockRunner = new MockQueryRunner(runner); - new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config) - .run(query1, ImmutableMap.of()); + new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config).run( + QueryPlus.wrap(query1), + ImmutableMap.of() + ); Assert.assertEquals(1000, mockRunner.query.getThreshold()); TopNQuery query2 = builder.threshold(10).context(context).build(); new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config) - .run(query2, ImmutableMap.of()); + .run(QueryPlus.wrap(query2), ImmutableMap.of()); Assert.assertEquals(500, mockRunner.query.getThreshold()); TopNQuery query3 = builder.threshold(2000).context(context).build(); new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config) - .run(query3, ImmutableMap.of()); + .run(QueryPlus.wrap(query3), ImmutableMap.of()); Assert.assertEquals(2000, mockRunner.query.getThreshold()); } @@ -248,7 +250,7 @@ public class TopNQueryQueryToolChestTest ) { this.query = (TopNQuery) queryPlus.getQuery(); - return query.run(runner, responseContext); + return runner.run(queryPlus, responseContext); } } } diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerBenchmark.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerBenchmark.java index db6305dd584..a1933764990 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerBenchmark.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerBenchmark.java @@ -26,6 +26,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import io.druid.collections.StupidPool; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -134,7 +135,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark @Test public void testmMapped() { - testCaseMap.get(TestCases.mMappedTestIndex).run(query, context); + testCaseMap.get(TestCases.mMappedTestIndex).run(QueryPlus.wrap(query), context); } @Ignore @@ -142,7 +143,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark @Test public void testrtIndex() { - testCaseMap.get(TestCases.rtIndex).run(query, context); + testCaseMap.get(TestCases.rtIndex).run(QueryPlus.wrap(query), context); } @Ignore @@ -150,7 +151,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark @Test public void testMerged() { - testCaseMap.get(TestCases.mergedRealtimeIndex).run(query, context); + testCaseMap.get(TestCases.mergedRealtimeIndex).run(QueryPlus.wrap(query), context); } @Ignore @@ -158,6 +159,6 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark @Test public void testOffHeap() { - testCaseMap.get(TestCases.rtIndexOffheap).run(query, context); + testCaseMap.get(TestCases.rtIndexOffheap).run(QueryPlus.wrap(query), context); } } diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 593108eef54..112522f8895 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -44,6 +44,7 @@ import io.druid.query.BySegmentResultValue; import io.druid.query.BySegmentResultValueClass; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -267,7 +268,7 @@ public class TopNQueryRunnerTest chest.mergeResults(runner), chest ); - return mergeRunner.run(query, context); + return mergeRunner.run(QueryPlus.wrap(query), context); } @Test @@ -4048,7 +4049,7 @@ public class TopNQueryRunnerTest ) ) ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, new HashMap())); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), new HashMap())); } @Test @@ -4087,7 +4088,7 @@ public class TopNQueryRunnerTest ) ) ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, new HashMap())); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), new HashMap())); } @@ -4210,7 +4211,7 @@ public class TopNQueryRunnerTest chest.mergeResults(chest.preMergeQueryDecoration(runner)), chest ); - return Runner.run(query, context); + return Runner.run(QueryPlus.wrap(query), context); } @Test diff --git a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java index c4f3b00e695..6758d2d9ecc 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.collections.StupidPool; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -180,7 +181,7 @@ public class TopNUnionQueryTest ) ); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } diff --git a/processing/src/test/java/io/druid/segment/AppendTest.java b/processing/src/test/java/io/druid/segment/AppendTest.java index f814d494dd9..b1365002263 100644 --- a/processing/src/test/java/io/druid/segment/AppendTest.java +++ b/processing/src/test/java/io/druid/segment/AppendTest.java @@ -26,6 +26,7 @@ import io.druid.java.util.common.Pair; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; import io.druid.query.Druids; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -182,7 +183,7 @@ public class AppendTest .build(); QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -207,7 +208,7 @@ public class AppendTest .build(); QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment2); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -232,7 +233,7 @@ public class AppendTest TimeseriesQuery query = makeTimeseriesQuery(); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -257,7 +258,7 @@ public class AppendTest TimeseriesQuery query = makeTimeseriesQuery(); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -282,7 +283,7 @@ public class AppendTest TimeseriesQuery query = makeFilteredTimeseriesQuery(); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -307,7 +308,7 @@ public class AppendTest TimeseriesQuery query = makeFilteredTimeseriesQuery(); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -353,7 +354,7 @@ public class AppendTest TopNQuery query = makeTopNQuery(); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -399,7 +400,7 @@ public class AppendTest TopNQuery query = makeTopNQuery(); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -427,7 +428,7 @@ public class AppendTest TopNQuery query = makeFilteredTopNQuery(); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -445,7 +446,7 @@ public class AppendTest TopNQuery query = makeFilteredTopNQuery(); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -468,7 +469,7 @@ public class AppendTest SearchQuery query = makeSearchQuery(); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -490,7 +491,7 @@ public class AppendTest SearchQuery query = makeSearchQuery(); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -511,7 +512,7 @@ public class AppendTest SearchQuery query = makeFilteredSearchQuery(); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -533,7 +534,7 @@ public class AppendTest SearchQuery query = makeFilteredSearchQuery(); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -575,7 +576,7 @@ public class AppendTest .build(); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment3); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } private TimeseriesQuery makeTimeseriesQuery() diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java index bc0b2ecba6f..e4218468ed3 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java @@ -31,6 +31,7 @@ import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -578,7 +579,7 @@ public class IndexMergerV9WithSpatialIndexTest factory.getToolchest() ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap())); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), Maps.newHashMap())); } catch (Exception e) { throw Throwables.propagate(e); @@ -632,7 +633,7 @@ public class IndexMergerV9WithSpatialIndexTest factory.getToolchest() ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap())); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), Maps.newHashMap())); } catch (Exception e) { throw Throwables.propagate(e); @@ -721,7 +722,7 @@ public class IndexMergerV9WithSpatialIndexTest factory.getToolchest() ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap())); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), Maps.newHashMap())); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java b/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java index 608966f37dd..9a8fe1ba56f 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java @@ -28,6 +28,7 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -1452,7 +1453,7 @@ public class SchemalessTestFullTest failMsg += " timeseries "; HashMap context = new HashMap<>(); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); @@ -1486,7 +1487,7 @@ public class SchemalessTestFullTest failMsg += " filtered timeseries "; HashMap context = new HashMap<>(); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); @@ -1519,7 +1520,7 @@ public class SchemalessTestFullTest failMsg += " topN "; HashMap context = new HashMap<>(); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); @@ -1553,7 +1554,7 @@ public class SchemalessTestFullTest failMsg += " filtered topN "; HashMap context = new HashMap<>(); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); @@ -1571,7 +1572,7 @@ public class SchemalessTestFullTest failMsg += " search "; HashMap context = new HashMap<>(); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); @@ -1590,7 +1591,7 @@ public class SchemalessTestFullTest failMsg += " filtered search "; HashMap context = new HashMap<>(); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); @@ -1609,7 +1610,7 @@ public class SchemalessTestFullTest failMsg += " timeBoundary "; HashMap context = new HashMap<>(); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java b/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java index f6e8c3377b2..2a8489f0c47 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.Lists; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; import io.druid.query.Druids; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.Result; import io.druid.query.TestQueryRunners; @@ -167,7 +168,7 @@ public class SchemalessTestSimpleTest ); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @@ -241,7 +242,7 @@ public class SchemalessTestSimpleTest QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -270,7 +271,7 @@ public class SchemalessTestSimpleTest QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } @Test @@ -296,6 +297,6 @@ public class SchemalessTestSimpleTest QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } } diff --git a/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java index d90871e1c47..e6849544a7a 100644 --- a/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java @@ -41,6 +41,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -484,7 +485,7 @@ public class IncrementalIndexTest List> results = Sequences.toList( - runner.run(query, new HashMap()), + runner.run(QueryPlus.wrap(query), new HashMap()), new LinkedList>() ); Result result = Iterables.getOnlyElement(results); @@ -642,7 +643,7 @@ public class IncrementalIndexTest factory.getToolchest() ); Map context = new HashMap(); - Sequence> sequence = runner.run(query, context); + Sequence> sequence = runner.run(QueryPlus.wrap(query), context); for (Double result : sequence.accumulate( @@ -701,7 +702,7 @@ public class IncrementalIndexTest .build(); Map context = new HashMap(); List> results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), new LinkedList>() ); boolean isRollup = index.isRollup(); diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java index ce0ce6dedca..2279836aba7 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -32,6 +32,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -512,7 +513,7 @@ public class SpatialFilterBonusTest factory.getToolchest() ); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } catch (Exception e) { throw Throwables.propagate(e); @@ -600,7 +601,7 @@ public class SpatialFilterBonusTest factory.getToolchest() ); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } catch (Exception e) { throw Throwables.propagate(e); @@ -694,7 +695,7 @@ public class SpatialFilterBonusTest factory.getToolchest() ); HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context)); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java index 38511774645..365d7be7d1a 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -31,6 +31,7 @@ import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -568,7 +569,7 @@ public class SpatialFilterTest factory.getToolchest() ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap())); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), Maps.newHashMap())); } catch (Exception e) { throw Throwables.propagate(e); @@ -621,7 +622,7 @@ public class SpatialFilterTest factory.getToolchest() ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap())); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), Maps.newHashMap())); } catch (Exception e) { throw Throwables.propagate(e); @@ -709,7 +710,7 @@ public class SpatialFilterTest factory.getToolchest() ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap())); + TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), Maps.newHashMap())); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index 712dfcd7fa6..446d5a2089f 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -40,6 +40,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.parsers.ParseException; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -393,7 +394,7 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark Map context = new HashMap(); for (Result result : Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), new LinkedList>() ) ) { @@ -429,7 +430,7 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark .build(); Map context = new HashMap(); List> results = Sequences.toList( - runner.run(query, context), + runner.run(QueryPlus.wrap(query), context), new LinkedList>() ); final int expectedVal = elementsPerThread * taskCount; diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 9fb01f7cfdf..b764421048d 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -529,6 +529,10 @@ public class CachingClusteredClientTest HashMap context = new HashMap(); + TimeseriesQuery query = builder.intervals("2011-01-01/2011-01-10") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build(); TestHelper.assertExpectedResults( makeRenamedTimeResults( new DateTime("2011-01-01"), 50, 5000, @@ -545,13 +549,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09"), 18, 521, new DateTime("2011-01-09T01"), 181, 52 ), - runner.run( - builder.intervals("2011-01-01/2011-01-10") - .aggregators(RENAMED_AGGS) - .postAggregators(RENAMED_POST_AGGS) - .build(), - context - ) + runner.run(QueryPlus.wrap(query), context) ); } @@ -591,7 +589,7 @@ public class CachingClusteredClientTest selector.addServerAndUpdateSegment(new QueryableDruidServer(lastServer, null), dataSegment); timeline.add(interval, "v", new SingleElementPartitionChunk<>(selector)); - getDefaultQueryRunner().run(query, context); + getDefaultQueryRunner().run(QueryPlus.wrap(query), context); Assert.assertTrue("Capture cache keys", cacheKeyCapture.hasCaptured()); Assert.assertTrue("Cache key below limit", ImmutableList.copyOf(cacheKeyCapture.getValue()).size() <= limit); @@ -605,7 +603,7 @@ public class CachingClusteredClientTest .once(); EasyMock.replay(cache); client = makeClient(MoreExecutors.sameThreadExecutor(), cache, 0); - getDefaultQueryRunner().run(query, context); + getDefaultQueryRunner().run(QueryPlus.wrap(query), context); EasyMock.verify(cache); EasyMock.verify(dataSegment); Assert.assertTrue("Capture cache keys", cacheKeyCapture.hasCaptured()); @@ -651,6 +649,11 @@ public class CachingClusteredClientTest ) ); + TimeseriesQuery query = builder + .intervals("2011-01-05/2011-01-10") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build(); TestHelper.assertExpectedResults( makeRenamedTimeResults( new DateTime("2011-01-05T00"), 85, 102, @@ -664,13 +667,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09T00"), 18, 521, new DateTime("2011-01-09T02"), 181, 52 ), - runner.run( - builder.intervals("2011-01-05/2011-01-10") - .aggregators(RENAMED_AGGS) - .postAggregators(RENAMED_POST_AGGS) - .build(), - Maps.newHashMap() - ) + runner.run(QueryPlus.wrap(query), Maps.newHashMap()) ); } @@ -705,6 +702,11 @@ public class CachingClusteredClientTest ) ); HashMap context = new HashMap(); + TimeseriesQuery query = builder + .intervals("2011-11-04/2011-11-08") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build(); TestHelper.assertExpectedResults( makeRenamedTimeResults( new DateTime("2011-11-04", TIMEZONE), 50, 5000, @@ -712,13 +714,7 @@ public class CachingClusteredClientTest new DateTime("2011-11-06", TIMEZONE), 23, 85312, new DateTime("2011-11-07", TIMEZONE), 85, 102 ), - runner.run( - builder.intervals("2011-11-04/2011-11-08") - .aggregators(RENAMED_AGGS) - .postAggregators(RENAMED_POST_AGGS) - .build(), - context - ) + runner.run(QueryPlus.wrap(query), context) ); } @@ -844,6 +840,12 @@ public class CachingClusteredClientTest ) ); HashMap context = new HashMap(); + TopNQuery query = builder + .intervals("2011-01-01/2011-01-10") + .metric("imps") + .aggregators(RENAMED_AGGS) + .postAggregators(DIFF_ORDER_POST_AGGS) + .build(); TestHelper.assertExpectedResults( makeRenamedTopNResults( new DateTime("2011-01-01"), "a", 50, 5000, "b", 50, 4999, "c", 50, 4998, @@ -859,14 +861,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09"), "c1", 50, 4985, "b", 50, 4984, "c", 50, 4983, new DateTime("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983 ), - runner.run( - builder.intervals("2011-01-01/2011-01-10") - .metric("imps") - .aggregators(RENAMED_AGGS) - .postAggregators(DIFF_ORDER_POST_AGGS) - .build(), - context - ) + runner.run(QueryPlus.wrap(query), context) ); } @@ -905,6 +900,12 @@ public class CachingClusteredClientTest ) ); HashMap context = new HashMap(); + TopNQuery query = builder + .intervals("2011-11-04/2011-11-08") + .metric("imps") + .aggregators(RENAMED_AGGS) + .postAggregators(DIFF_ORDER_POST_AGGS) + .build(); TestHelper.assertExpectedResults( makeRenamedTopNResults( @@ -913,14 +914,7 @@ public class CachingClusteredClientTest new DateTime("2011-11-06", TIMEZONE), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, new DateTime("2011-11-07", TIMEZONE), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986 ), - runner.run( - builder.intervals("2011-11-04/2011-11-08") - .metric("imps") - .aggregators(RENAMED_AGGS) - .postAggregators(DIFF_ORDER_POST_AGGS) - .build(), - context - ) + runner.run(QueryPlus.wrap(query), context) ); } @@ -1027,6 +1021,12 @@ public class CachingClusteredClientTest ); HashMap context = new HashMap(); + TopNQuery query = builder + .intervals("2011-01-01/2011-01-10") + .metric("imps") + .aggregators(RENAMED_AGGS) + .postAggregators(DIFF_ORDER_POST_AGGS) + .build(); TestHelper.assertExpectedResults( makeRenamedTopNResults( new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, @@ -1040,14 +1040,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983, new DateTime("2011-01-09T01"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 ), - runner.run( - builder.intervals("2011-01-01/2011-01-10") - .metric("imps") - .aggregators(RENAMED_AGGS) - .postAggregators(DIFF_ORDER_POST_AGGS) - .build(), - context - ) + runner.run(QueryPlus.wrap(query), context) ); } @@ -1101,6 +1094,12 @@ public class CachingClusteredClientTest ); HashMap context = new HashMap(); + TopNQuery query = builder + .intervals("2011-01-01/2011-01-10") + .metric("avg_imps_per_row_double") + .aggregators(AGGS) + .postAggregators(DIFF_ORDER_POST_AGGS) + .build(); TestHelper.assertExpectedResults( makeTopNResultsWithoutRename( new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, @@ -1114,14 +1113,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09"), "c1", 50, 4985, "b", 50, 4984, "c", 50, 4983, new DateTime("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983 ), - runner.run( - builder.intervals("2011-01-01/2011-01-10") - .metric("avg_imps_per_row_double") - .aggregators(AGGS) - .postAggregators(DIFF_ORDER_POST_AGGS) - .build(), - context - ) + runner.run(QueryPlus.wrap(query), context) ); } @@ -1191,11 +1183,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4, new DateTime("2011-01-09T01"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4 ), - runner.run( - builder.intervals("2011-01-01/2011-01-10") - .build(), - context - ) + runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context) ); } @@ -1265,12 +1253,12 @@ public class CachingClusteredClientTest new DateTime("2011-01-09"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4, new DateTime("2011-01-09T01"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4 ), - runner.run( - builder.intervals("2011-01-01/2011-01-10") - .build(), - context - ) + runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context) ); + SearchQuery query = builder + .intervals("2011-01-01/2011-01-10") + .dimensions(new DefaultDimensionSpec(TOP_DIM, "new_dim")) + .build(); TestHelper.assertExpectedResults( makeSearchResults( "new_dim", @@ -1287,15 +1275,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4, new DateTime("2011-01-09T01"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4 ), - runner.run( - builder.intervals("2011-01-01/2011-01-10") - .dimensions(new DefaultDimensionSpec( - TOP_DIM, - "new_dim" - )) - .build(), - context - ) + runner.run(QueryPlus.wrap(query), context) ); } @@ -1364,11 +1344,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9), new DateTime("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9) ), - runner.run( - builder.intervals("2011-01-01/2011-01-10") - .build(), - context - ) + runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context) ); } @@ -1443,13 +1419,13 @@ public class CachingClusteredClientTest new DateTime("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9), new DateTime("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9) ), - runner.run( - builder.intervals("2011-01-01/2011-01-10") - .build(), - context - ) + runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context) ); + SelectQuery query = builder + .intervals("2011-01-01/2011-01-10") + .dimensionSpecs(Lists.newArrayList(new DefaultDimensionSpec("a", "a2"))) + .build(); TestHelper.assertExpectedResults( makeSelectResults( dimensions, metrics, @@ -1466,12 +1442,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09"), ImmutableMap.of("a2", "h", "rows", 9), new DateTime("2011-01-09T01"), ImmutableMap.of("a2", "h", "rows", 9) ), - runner.run( - builder.intervals("2011-01-01/2011-01-10") - .dimensionSpecs(Lists.newArrayList(new DefaultDimensionSpec("a", "a2"))) - .build(), - context - ) + runner.run(QueryPlus.wrap(query), context) ); } @@ -1571,11 +1542,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09T01"), ImmutableMap.of("a", "g", "rows", 7, "imps", 7, "impers", 7, "uniques", collector) ), - runner.run( - builder.setInterval("2011-01-05/2011-01-10") - .build(), - context - ), + runner.run(QueryPlus.wrap(builder.setInterval("2011-01-05/2011-01-10").build()), context), "" ); } @@ -2198,11 +2165,9 @@ public class CachingClusteredClientTest ) ), runner.run( - query.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec( - ImmutableList.of( - actualQueryInterval - ) + QueryPlus.wrap( + query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(ImmutableList.of(actualQueryInterval)) ) ), context @@ -3138,14 +3103,15 @@ public class CachingClusteredClientTest new DateTime("2011-01-09T"), ImmutableMap.of("output", "g", "rows", 7, "imps", 7, "impers", 7), new DateTime("2011-01-09T01"), ImmutableMap.of("output", "g", "rows", 7, "imps", 7, "impers", 7) ), - runner.run( - builder.setInterval("2011-01-05/2011-01-10") - .build(), - context - ), + runner.run(QueryPlus.wrap(builder.setInterval("2011-01-05/2011-01-10").build()), context), "" ); + GroupByQuery query = builder + .setInterval("2011-01-05/2011-01-10") + .setDimensions(Collections.singletonList(new DefaultDimensionSpec("a", "output2"))) + .setAggregatorSpecs(RENAMED_AGGS) + .build(); TestHelper.assertExpectedObjects( makeGroupByResults( new DateTime("2011-01-05T"), ImmutableMap.of("output2", "c", "rows", 3, "imps", 3, "impers2", 3), @@ -3159,13 +3125,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09T"), ImmutableMap.of("output2", "g", "rows", 7, "imps", 7, "impers2", 7), new DateTime("2011-01-09T01"), ImmutableMap.of("output2", "g", "rows", 7, "imps", 7, "impers2", 7) ), - runner.run( - builder.setInterval("2011-01-05/2011-01-10") - .setDimensions(Arrays.asList(new DefaultDimensionSpec("a", "output2"))) - .setAggregatorSpecs(RENAMED_AGGS) - .build(), - context - ), + runner.run(QueryPlus.wrap(query), context), "renamed aggregators test" ); } @@ -3204,7 +3164,7 @@ public class CachingClusteredClientTest Map responseContext = new HashMap<>(); - getDefaultQueryRunner().run(query, responseContext); + getDefaultQueryRunner().run(QueryPlus.wrap(query), responseContext); Assert.assertEquals("Z/eS4rQz5v477iq7Aashr6JPZa0=", responseContext.get("ETag")); } diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index d2e2d365c91..3ee8c9ec37a 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -300,7 +300,7 @@ public class CachingQueryRunnerTest ); HashMap context = new HashMap(); - Sequence res = runner.run(query, context); + Sequence res = runner.run(QueryPlus.wrap(query), context); // base sequence is not closed yet Assert.assertFalse("sequence must not be closed", closable.isClosed()); Assert.assertNull("cache must be empty", cache.get(cacheKey)); @@ -386,7 +386,7 @@ public class CachingQueryRunnerTest ); HashMap context = new HashMap(); - List results = Sequences.toList(runner.run(query, context), new ArrayList()); + List results = Sequences.toList(runner.run(QueryPlus.wrap(query), context), new ArrayList()); Assert.assertEquals(expectedResults.toString(), results.toString()); } diff --git a/server/src/test/java/io/druid/client/DirectDruidClientTest.java b/server/src/test/java/io/druid/client/DirectDruidClientTest.java index 2f0729bba9b..425b96fbd7e 100644 --- a/server/src/test/java/io/druid/client/DirectDruidClientTest.java +++ b/server/src/test/java/io/druid/client/DirectDruidClientTest.java @@ -33,12 +33,13 @@ import io.druid.client.selector.ConnectionCountServerSelectorStrategy; import io.druid.client.selector.HighestPriorityTierSelectorStrategy; import io.druid.client.selector.QueryableDruidServer; import io.druid.client.selector.ServerSelector; -import io.druid.java.util.common.StringUtils; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; import io.druid.query.QueryInterruptedException; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.ReflectionQueryToolChestWarehouse; import io.druid.query.Result; @@ -165,22 +166,22 @@ public class DirectDruidClientTest TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build(); - Sequence s1 = client1.run(query, defaultContext); + Sequence s1 = client1.run(QueryPlus.wrap(query), defaultContext); Assert.assertTrue(capturedRequest.hasCaptured()); Assert.assertEquals(url, capturedRequest.getValue().getUrl()); Assert.assertEquals(HttpMethod.POST, capturedRequest.getValue().getMethod()); Assert.assertEquals(1, client1.getNumOpenConnections()); // simulate read timeout - Sequence s2 = client1.run(query, defaultContext); + Sequence s2 = client1.run(QueryPlus.wrap(query), defaultContext); Assert.assertEquals(2, client1.getNumOpenConnections()); futureException.setException(new ReadTimeoutException()); Assert.assertEquals(1, client1.getNumOpenConnections()); // subsequent connections should work - Sequence s3 = client1.run(query, defaultContext); - Sequence s4 = client1.run(query, defaultContext); - Sequence s5 = client1.run(query, defaultContext); + Sequence s3 = client1.run(QueryPlus.wrap(query), defaultContext); + Sequence s4 = client1.run(QueryPlus.wrap(query), defaultContext); + Sequence s5 = client1.run(QueryPlus.wrap(query), defaultContext); Assert.assertTrue(client1.getNumOpenConnections() == 4); @@ -195,8 +196,8 @@ public class DirectDruidClientTest Assert.assertEquals(new DateTime("2014-01-01T01:02:03Z"), results.get(0).getTimestamp()); Assert.assertEquals(3, client1.getNumOpenConnections()); - client2.run(query, defaultContext); - client2.run(query, defaultContext); + client2.run(QueryPlus.wrap(query), defaultContext); + client2.run(QueryPlus.wrap(query), defaultContext); Assert.assertTrue(client2.getNumOpenConnections() == 2); @@ -269,7 +270,7 @@ public class DirectDruidClientTest TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build(); cancellationFuture.set(new StatusResponseHolder(HttpResponseStatus.OK, new StringBuilder("cancelled"))); - Sequence results = client1.run(query, defaultContext); + Sequence results = client1.run(QueryPlus.wrap(query), defaultContext); Assert.assertEquals(HttpMethod.DELETE, capturedRequest.getValue().getMethod()); Assert.assertEquals(0, client1.getNumOpenConnections()); @@ -344,7 +345,7 @@ public class DirectDruidClientTest StringUtils.toUtf8("{\"error\":\"testing1\",\"errorMessage\":\"testing2\"}") ) ); - Sequence results = client1.run(query, defaultContext); + Sequence results = client1.run(QueryPlus.wrap(query), defaultContext); QueryInterruptedException actualException = null; try { diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java index 650069f4703..f678f64b955 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java @@ -31,6 +31,7 @@ import io.druid.data.input.MapBasedInputRow; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; +import io.druid.query.QueryPlus; import io.druid.query.Result; import io.druid.query.SegmentDescriptor; import io.druid.query.aggregation.AggregatorFactory; @@ -313,7 +314,7 @@ public class AppenderatorTest .build(); final List> results1 = Lists.newArrayList(); - Sequences.toList(query1.run(appenderator, ImmutableMap.of()), results1); + Sequences.toList(QueryPlus.wrap(query1).run(appenderator, ImmutableMap.of()), results1); Assert.assertEquals( "query1", ImmutableList.of( @@ -339,7 +340,7 @@ public class AppenderatorTest .build(); final List> results2 = Lists.newArrayList(); - Sequences.toList(query2.run(appenderator, ImmutableMap.of()), results2); + Sequences.toList(QueryPlus.wrap(query2).run(appenderator, ImmutableMap.of()), results2); Assert.assertEquals( "query2", ImmutableList.of( @@ -369,7 +370,7 @@ public class AppenderatorTest .build(); final List> results3 = Lists.newArrayList(); - Sequences.toList(query3.run(appenderator, ImmutableMap.of()), results3); + Sequences.toList(QueryPlus.wrap(query3).run(appenderator, ImmutableMap.of()), results3); Assert.assertEquals( ImmutableList.of( new Result<>( @@ -403,7 +404,7 @@ public class AppenderatorTest .build(); final List> results4 = Lists.newArrayList(); - Sequences.toList(query4.run(appenderator, ImmutableMap.of()), results4); + Sequences.toList(QueryPlus.wrap(query4).run(appenderator, ImmutableMap.of()), results4); Assert.assertEquals( ImmutableList.of( new Result<>( @@ -459,7 +460,7 @@ public class AppenderatorTest .build(); final List> results1 = Lists.newArrayList(); - Sequences.toList(query1.run(appenderator, ImmutableMap.of()), results1); + Sequences.toList(QueryPlus.wrap(query1).run(appenderator, ImmutableMap.of()), results1); Assert.assertEquals( "query1", ImmutableList.of( @@ -495,7 +496,7 @@ public class AppenderatorTest .build(); final List> results2 = Lists.newArrayList(); - Sequences.toList(query2.run(appenderator, ImmutableMap.of()), results2); + Sequences.toList(QueryPlus.wrap(query2).run(appenderator, ImmutableMap.of()), results2); Assert.assertEquals( "query2", ImmutableList.of( @@ -536,7 +537,7 @@ public class AppenderatorTest .build(); final List> results3 = Lists.newArrayList(); - Sequences.toList(query3.run(appenderator, ImmutableMap.of()), results3); + Sequences.toList(QueryPlus.wrap(query3).run(appenderator, ImmutableMap.of()), results3); Assert.assertEquals( "query2", ImmutableList.of( diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 2deda1ea5f2..9412232afcc 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -442,7 +442,7 @@ public class ServerManagerTest public void run() { Map context = new HashMap(); - Sequence> seq = runner.run(query, context); + Sequence> seq = runner.run(QueryPlus.wrap(query), context); Sequences.toList(seq, Lists.>newArrayList()); Iterator adaptersIter = factory.getAdapters().iterator();