You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by le...@apache.org on 2019/07/24 15:29:14 UTC
[incubator-druid] branch master updated: Response context
refactoring (#8110)
This is an automated email from the ASF dual-hosted git repository.
leventov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git
The following commit(s) were added to refs/heads/master by this push:
new 799d202 Response context refactoring (#8110)
799d202 is described below
commit 799d20249fe6333ea86b020f6d09c91fa4d3f998
Author: Eugene Sevastianov <se...@gmail.com>
AuthorDate: Wed Jul 24 18:29:03 2019 +0300
Response context refactoring (#8110)
* Response context refactoring
* Serialization/Deserialization of ResponseContext
* Added java doc comments
* Renamed vars related to ResponseContext
* Renamed empty() methods to createEmpty()
* Fixed ResponseContext usage
* Renamed multiple ResponseContext static fields
* Added PublicApi annotations
* Renamed QueryResponseContext class to ResourceIOReaderWriter
* Moved the protected method below public static constants
* Added createEmpty method to ResponseContext with DefaultResponseContext creation
* Fixed inspection error
* Added comments to the ResponseContext length limit and ResponseContext
http header name
* Added a comment of possible future refactoring
* Removed .gitignore file of indexing-service
* Removed a never-used method
* VisibleForTesting method reducing boilerplate
Co-Authored-By: Clint Wylie <cj...@gmail.com>
* Reduced boilerplate
* Renamed the method serialize to serializeWith
* Removed unused import
* Fixed incorrectly refactored test method
* Added comments for ResponseContext keys
* Fixed incorrectly refactored test method
* Fixed IntervalChunkingQueryRunnerTest mocks
---
.../benchmark/FilteredAggregatorBenchmark.java | 4 +-
.../benchmark/GroupByTypeInterfaceBenchmark.java | 4 +-
.../benchmark/TopNTypeInterfaceBenchmark.java | 4 +-
.../query/CachingClusteredClientBenchmark.java | 5 +-
.../druid/benchmark/query/GroupByBenchmark.java | 10 +-
.../druid/benchmark/query/QueryBenchmarkUtil.java | 5 +-
.../druid/benchmark/query/ScanBenchmark.java | 6 +-
.../druid/benchmark/query/SearchBenchmark.java | 6 +-
.../druid/benchmark/query/SelectBenchmark.java | 5 +-
.../benchmark/query/SerializingQueryRunner.java | 5 +-
.../benchmark/query/SqlVsNativeBenchmark.java | 4 +-
.../druid/benchmark/query/TimeseriesBenchmark.java | 6 +-
.../druid/benchmark/query/TopNBenchmark.java | 6 +-
.../query/timecompare/TimeCompareBenchmark.java | 5 +-
.../MaterializedViewQueryQueryToolChest.java | 7 +-
.../MaterializedViewQueryRunner.java | 5 +-
.../movingaverage/MovingAverageQueryRunner.java | 20 ++--
.../movingaverage/MovingAverageQueryTest.java | 4 +-
.../druid/segment/MapVirtualColumnGroupByTest.java | 5 +-
.../druid/segment/MapVirtualColumnSelectTest.java | 3 +-
.../druid/segment/MapVirtualColumnTopNTest.java | 5 +-
.../ApproximateHistogramTopNQueryTest.java | 5 +-
.../FixedBucketsHistogramTopNQueryTest.java | 4 +-
.../druid/indexing/kafka/KafkaIndexTaskTest.java | 5 +-
.../indexing/kinesis/KinesisIndexTaskTest.java | 3 +-
.../variance/VarianceTimeseriesQueryTest.java | 3 +-
.../variance/VarianceTopNQueryTest.java | 5 +-
.../AppenderatorDriverRealtimeIndexTaskTest.java | 2 +-
.../common/task/RealtimeIndexTaskTest.java | 3 +-
.../org/apache/druid/query/AsyncQueryRunner.java | 4 +-
.../apache/druid/query/BySegmentQueryRunner.java | 4 +-
.../druid/query/BySegmentSkippingQueryRunner.java | 7 +-
.../druid/query/CPUTimeMetricQueryRunner.java | 4 +-
.../druid/query/ChainedExecutionQueryRunner.java | 4 +-
.../org/apache/druid/query/ConcatQueryRunner.java | 5 +-
.../druid/query/FinalizeResultsQueryRunner.java | 5 +-
.../druid/query/FluentQueryRunnerBuilder.java | 4 +-
.../druid/query/GroupByMergedQueryRunner.java | 4 +-
.../druid/query/IntervalChunkingQueryRunner.java | 4 +-
.../druid/query/MetricsEmittingQueryRunner.java | 4 +-
.../org/apache/druid/query/NoopQueryRunner.java | 5 +-
.../query/PerSegmentOptimizingQueryRunner.java | 5 +-
.../java/org/apache/druid/query/QueryPlus.java | 4 +-
.../java/org/apache/druid/query/QueryRunner.java | 12 +-
.../org/apache/druid/query/QueryRunnerHelper.java | 4 +-
.../query/ReferenceCountingSegmentQueryRunner.java | 5 +-
.../ReportTimelineMissingSegmentQueryRunner.java | 8 +-
.../main/java/org/apache/druid/query/Result.java | 3 -
.../apache/druid/query/ResultMergeQueryRunner.java | 4 +-
.../org/apache/druid/query/RetryQueryRunner.java | 10 +-
.../apache/druid/query/SubqueryQueryRunner.java | 5 +-
.../org/apache/druid/query/TimewarpOperator.java | 4 +-
.../org/apache/druid/query/UnionQueryRunner.java | 5 +-
.../ConcurrentResponseContext.java} | 23 ++--
.../DefaultResponseContext.java} | 23 ++--
.../druid/query/context/ResponseContext.java | 131 +++++++++++++++++++++
.../DataSourceMetadataQueryRunnerFactory.java | 4 +-
.../DataSourceQueryQueryToolChest.java | 4 +-
.../query/groupby/GroupByQueryQueryToolChest.java | 11 +-
.../query/groupby/GroupByQueryRunnerFactory.java | 6 +-
.../epinephelinae/GroupByMergingQueryRunnerV2.java | 4 +-
.../query/groupby/strategy/GroupByStrategy.java | 4 +-
.../query/groupby/strategy/GroupByStrategyV1.java | 4 +-
.../query/groupby/strategy/GroupByStrategyV2.java | 7 +-
.../SegmentMetadataQueryQueryToolChest.java | 3 +-
.../SegmentMetadataQueryRunnerFactory.java | 5 +-
.../apache/druid/query/scan/ScanQueryEngine.java | 23 ++--
.../query/scan/ScanQueryLimitRowIterator.java | 4 +-
.../druid/query/scan/ScanQueryRunnerFactory.java | 16 +--
.../query/search/SearchQueryQueryToolChest.java | 5 +-
.../druid/query/search/SearchQueryRunner.java | 4 +-
.../query/select/SelectQueryQueryToolChest.java | 3 +-
.../query/select/SelectQueryRunnerFactory.java | 4 +-
.../query/spec/SpecificSegmentQueryRunner.java | 12 +-
.../TimeBoundaryQueryQueryToolChest.java | 4 +-
.../TimeBoundaryQueryRunnerFactory.java | 4 +-
.../timeseries/TimeseriesQueryQueryToolChest.java | 3 +-
.../timeseries/TimeseriesQueryRunnerFactory.java | 4 +-
.../druid/query/topn/TopNQueryQueryToolChest.java | 7 +-
.../druid/query/topn/TopNQueryRunnerFactory.java | 4 +-
.../apache/druid/query/AsyncQueryRunnerTest.java | 16 ++-
.../query/ChainedExecutionQueryRunnerTest.java | 11 +-
.../org/apache/druid/query/DoubleStorageTest.java | 6 +-
.../query/IntervalChunkingQueryRunnerTest.java | 10 +-
.../druid/query/MultiValuedDimensionTest.java | 10 +-
.../apache/druid/query/QueryRunnerTestHelper.java | 7 +-
.../apache/druid/query/RetryQueryRunnerTest.java | 59 +++++-----
.../apache/druid/query/SchemaEvolutionTest.java | 2 +-
.../apache/druid/query/TimewarpOperatorTest.java | 25 ++--
.../apache/druid/query/UnionQueryRunnerTest.java | 7 +-
.../query/aggregation/AggregationTestHelper.java | 9 +-
.../DataSourceMetadataQueryTest.java | 8 +-
...GroupByLimitPushDownInsufficientBufferTest.java | 11 +-
.../GroupByLimitPushDownMultiNodeMergeTest.java | 11 +-
.../query/groupby/GroupByMultiSegmentTest.java | 5 +-
.../groupby/GroupByQueryRunnerFactoryTest.java | 9 +-
.../query/groupby/GroupByQueryRunnerTest.java | 75 +++++-------
.../groupby/GroupByQueryRunnerTestHelper.java | 2 +-
.../groupby/GroupByTimeseriesQueryRunnerTest.java | 6 +-
.../query/groupby/NestedQueryPushDownTest.java | 17 +--
.../druid/query/metadata/SegmentAnalyzerTest.java | 4 +-
.../query/metadata/SegmentMetadataQueryTest.java | 23 ++--
.../metadata/SegmentMetadataUnionQueryTest.java | 3 +-
.../query/scan/MultiSegmentScanQueryTest.java | 13 +-
.../query/scan/ScanQueryLimitRowIteratorTest.java | 7 +-
.../query/scan/ScanQueryRunnerFactoryTest.java | 4 +-
.../druid/query/scan/ScanQueryRunnerTest.java | 38 +++---
.../druid/query/search/SearchQueryRunnerTest.java | 6 +-
.../search/SearchQueryRunnerWithCaseTest.java | 5 +-
.../query/select/MultiSegmentSelectQueryTest.java | 9 +-
.../druid/query/select/SelectQueryRunnerTest.java | 35 +++---
.../query/spec/SpecificSegmentQueryRunnerTest.java | 17 ++-
.../timeboundary/TimeBoundaryQueryRunnerTest.java | 22 ++--
.../timeseries/TimeSeriesUnionQueryRunnerTest.java | 11 +-
.../timeseries/TimeseriesQueryRunnerBonusTest.java | 4 +-
.../timeseries/TimeseriesQueryRunnerTest.java | 114 +++++++++---------
.../query/topn/TopNQueryQueryToolChestTest.java | 14 +--
.../druid/query/topn/TopNQueryRunnerBenchmark.java | 10 +-
.../druid/query/topn/TopNQueryRunnerTest.java | 25 ++--
.../druid/query/topn/TopNUnionQueryTest.java | 4 +-
.../java/org/apache/druid/segment/AppendTest.java | 46 +++-----
.../segment/IndexMergerV9WithSpatialIndexTest.java | 7 +-
.../druid/segment/SchemalessTestFullTest.java | 22 ++--
.../druid/segment/SchemalessTestSimpleTest.java | 13 +-
.../druid/segment/data/IncrementalIndexTest.java | 10 +-
.../segment/filter/SpatialFilterBonusTest.java | 10 +-
.../druid/segment/filter/SpatialFilterTest.java | 7 +-
.../OnheapIncrementalIndexBenchmark.java | 8 +-
.../druid/client/CachingClusteredClient.java | 17 +--
.../apache/druid/client/CachingQueryRunner.java | 4 +-
.../org/apache/druid/client/DirectDruidClient.java | 31 ++---
.../druid/query/ResultLevelCachingQueryRunner.java | 6 +-
.../realtime/appenderator/AppenderatorPlumber.java | 3 +-
.../apache/druid/server/BrokerQueryResource.java | 9 +-
.../org/apache/druid/server/QueryLifecycle.java | 10 +-
.../org/apache/druid/server/QueryResource.java | 48 ++++----
.../server/SetAndVerifyContextQueryRunner.java | 5 +-
.../CachingClusteredClientFunctionalityTest.java | 25 ++--
.../druid/client/CachingClusteredClientTest.java | 64 ++++------
.../druid/client/CachingQueryRunnerTest.java | 12 +-
.../apache/druid/client/DirectDruidClientTest.java | 29 ++---
.../realtime/appenderator/AppenderatorTest.java | 18 +--
.../org/apache/druid/server/QueryResourceTest.java | 4 +-
.../server/coordination/ServerManagerTest.java | 8 +-
.../java/org/apache/druid/cli/DumpSegment.java | 4 +-
145 files changed, 843 insertions(+), 854 deletions(-)
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
index d7a98e8..a194822 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
@@ -44,6 +44,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.filter.BoundDimFilter;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.filter.InDimFilter;
@@ -89,7 +90,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.concurrent.TimeUnit;
@@ -240,7 +240,7 @@ public class FilteredAggregatorBenchmark
final QueryPlus<T> queryToRun = QueryPlus.wrap(
query.withOverriddenContext(ImmutableMap.of("vectorize", vectorize))
);
- Sequence<T> queryResult = theRunner.run(queryToRun, new HashMap<>());
+ Sequence<T> queryResult = theRunner.run(queryToRun, ResponseContext.createEmpty());
return queryResult.toList();
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
index 20c5a10..e4cf5ac 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
@@ -51,6 +51,7 @@ import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
@@ -92,7 +93,6 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@@ -450,7 +450,7 @@ public class GroupByTypeInterfaceBenchmark
toolChest
);
- Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
return queryResult.toList();
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java
index cb79811..f24a9f8 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java
@@ -46,6 +46,7 @@ import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.ExtractionDimensionSpec;
import org.apache.druid.query.extraction.IdentityExtractionFn;
import org.apache.druid.query.ordering.StringComparators;
@@ -85,7 +86,6 @@ import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@@ -322,7 +322,7 @@ public class TopNTypeInterfaceBenchmark
toolChest
);
- Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
return queryResult.toList();
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java
index b6807d0..3ddd824 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java
@@ -74,6 +74,7 @@ import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.expression.TestExprMacroTable;
import org.apache.druid.query.groupby.GroupByQuery;
@@ -432,7 +433,7 @@ public class CachingClusteredClientBenchmark
.applyPostMergeDecoration();
//noinspection unchecked
- Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
return queryResult.toList();
}
@@ -528,7 +529,7 @@ public class CachingClusteredClientBenchmark
}
@Override
- public Sequence<Object> run(QueryPlus<Object> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Object> run(QueryPlus<Object> queryPlus, ResponseContext responseContext)
{
final QueryRunnerFactory factory = conglomerate.findFactory(queryPlus.getQuery());
//noinspection unchecked
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
index 2eb7426..9076ff9 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
@@ -56,6 +56,7 @@ import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.filter.BoundDimFilter;
import org.apache.druid.query.groupby.GroupByQuery;
@@ -101,7 +102,6 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@@ -554,7 +554,7 @@ public class GroupByBenchmark
toolChest
);
- return theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ return theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
}
@Benchmark
@@ -610,7 +610,7 @@ public class GroupByBenchmark
(QueryToolChest) toolChest
);
- Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
List<Row> results = queryResult.toList();
blackhole.consume(results);
}
@@ -631,7 +631,7 @@ public class GroupByBenchmark
final GroupByQuery spillingQuery = query.withOverriddenContext(
ImmutableMap.of("bufferGrouperMaxSize", 4000)
);
- Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(spillingQuery), new HashMap<>());
+ Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(spillingQuery), ResponseContext.createEmpty());
List<Row> results = queryResult.toList();
blackhole.consume(results);
}
@@ -655,7 +655,7 @@ public class GroupByBenchmark
(QueryToolChest) toolChest
);
- Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
List<Row> results = queryResult.toList();
blackhole.consume(results);
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java
index 6e6ae2a..6589124 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java
@@ -30,11 +30,10 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.Segment;
import org.apache.druid.timeline.SegmentId;
-import java.util.Map;
-
public class QueryBenchmarkUtil
{
public static <T, QueryType extends Query<T>> QueryRunner<T> makeQueryRunner(
@@ -57,7 +56,7 @@ public class QueryBenchmarkUtil
{
return new QueryRunner<T>() {
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
return delegate.run(queryPlus, responseContext);
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java
index 60d47e3..5afcab6 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java
@@ -46,6 +46,7 @@ import org.apache.druid.query.Result;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.extraction.StrlenExtractionFn;
import org.apache.druid.query.filter.BoundDimFilter;
import org.apache.druid.query.filter.DimFilter;
@@ -88,7 +89,6 @@ import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@@ -330,7 +330,7 @@ public class ScanBenchmark
toolChest
);
- Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
return queryResult.toList();
}
@@ -441,7 +441,7 @@ public class ScanBenchmark
Sequence<Result<ScanResultValue>> queryResult = theRunner.run(
QueryPlus.wrap(effectiveQuery),
- new HashMap<>()
+ ResponseContext.createEmpty()
);
List<Result<ScanResultValue>> results = queryResult.toList();
blackhole.consume(results);
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java
index 04f1ca7..59f1fec 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java
@@ -46,6 +46,7 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.extraction.DimExtractionFn;
import org.apache.druid.query.extraction.IdentityExtractionFn;
import org.apache.druid.query.extraction.LowerExtractionFn;
@@ -95,7 +96,6 @@ import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@@ -400,7 +400,7 @@ public class SearchBenchmark
toolChest
);
- Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
return queryResult.toList();
}
@@ -461,7 +461,7 @@ public class SearchBenchmark
Sequence<Result<SearchResultValue>> queryResult = theRunner.run(
QueryPlus.wrap(query),
- new HashMap<>()
+ ResponseContext.createEmpty()
);
List<Result<SearchResultValue>> results = queryResult.toList();
blackhole.consume(results);
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java
index 647160d..439e984 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java
@@ -44,6 +44,7 @@ import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.Result;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.select.EventHolder;
import org.apache.druid.query.select.PagingSpec;
@@ -261,7 +262,7 @@ public class SelectBenchmark
toolChest
);
- Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
return queryResult.toList();
}
@@ -368,7 +369,7 @@ public class SelectBenchmark
boolean done = false;
while (!done) {
- Sequence<Result<SelectResultValue>> queryResult = theRunner.run(QueryPlus.wrap(queryCopy), new HashMap<>());
+ Sequence<Result<SelectResultValue>> queryResult = theRunner.run(QueryPlus.wrap(queryCopy), ResponseContext.createEmpty());
List<Result<SelectResultValue>> results = queryResult.toList();
SelectResultValue result = results.get(0).getValue();
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SerializingQueryRunner.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SerializingQueryRunner.java
index 2f61a90..1f6153d 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SerializingQueryRunner.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SerializingQueryRunner.java
@@ -25,8 +25,7 @@ import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
-
-import java.util.Map;
+import org.apache.druid.query.context.ResponseContext;
public class SerializingQueryRunner<T> implements QueryRunner<T>
{
@@ -48,7 +47,7 @@ public class SerializingQueryRunner<T> implements QueryRunner<T>
@Override
public Sequence<T> run(
final QueryPlus<T> queryPlus,
- final Map<String, Object> responseContext
+ final ResponseContext responseContext
)
{
return Sequences.map(
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java
index fdac5ff..47ae6ca 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java
@@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.segment.QueryableIndex;
@@ -63,7 +64,6 @@ import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
-import java.util.HashMap;
import java.util.concurrent.TimeUnit;
/**
@@ -151,7 +151,7 @@ public class SqlVsNativeBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryNative(Blackhole blackhole)
{
- final Sequence<Row> resultSequence = QueryPlus.wrap(groupByQuery).run(walker, new HashMap<>());
+ final Sequence<Row> resultSequence = QueryPlus.wrap(groupByQuery).run(walker, ResponseContext.createEmpty());
final Row lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);
blackhole.consume(lastRow);
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java
index eb7600b..ba2484a 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java
@@ -48,6 +48,7 @@ import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.filter.BoundDimFilter;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.filter.SelectorDimFilter;
@@ -89,7 +90,6 @@ import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@@ -323,7 +323,7 @@ public class TimeseriesBenchmark
toolChest
);
- Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
return queryResult.toList();
}
@@ -401,7 +401,7 @@ public class TimeseriesBenchmark
Sequence<Result<TimeseriesResultValue>> queryResult = theRunner.run(
QueryPlus.wrap(query),
- new HashMap<>()
+ ResponseContext.createEmpty()
);
List<Result<TimeseriesResultValue>> results = queryResult.toList();
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java
index 7a9e48e..904cdfe 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java
@@ -47,6 +47,7 @@ import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
@@ -86,7 +87,6 @@ import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@@ -307,7 +307,7 @@ public class TopNBenchmark
toolChest
);
- Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
return queryResult.toList();
}
@@ -367,7 +367,7 @@ public class TopNBenchmark
Sequence<Result<TopNResultValue>> queryResult = theRunner.run(
QueryPlus.wrap(query),
- new HashMap<>()
+ ResponseContext.createEmpty()
);
List<Result<TopNResultValue>> results = queryResult.toList();
blackhole.consume(results);
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java
index e5ea310..6348d18 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java
@@ -52,6 +52,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.filter.IntervalDimFilter;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
@@ -417,7 +418,7 @@ public class TimeCompareBenchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void queryMultiQueryableIndexTopN(Blackhole blackhole)
{
- Sequence<Result<TopNResultValue>> queryResult = topNRunner.run(QueryPlus.wrap(topNQuery), new HashMap<>());
+ Sequence<Result<TopNResultValue>> queryResult = topNRunner.run(QueryPlus.wrap(topNQuery), ResponseContext.createEmpty());
List<Result<TopNResultValue>> results = queryResult.toList();
blackhole.consume(results);
}
@@ -430,7 +431,7 @@ public class TimeCompareBenchmark
{
Sequence<Result<TimeseriesResultValue>> queryResult = timeseriesRunner.run(
QueryPlus.wrap(timeseriesQuery),
- new HashMap<>()
+ ResponseContext.createEmpty()
);
List<Result<TimeseriesResultValue>> results = queryResult.toList();
blackhole.consume(results);
diff --git a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChest.java b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChest.java
index 39e2f19..6ec3108 100644
--- a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChest.java
+++ b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChest.java
@@ -30,8 +30,7 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.aggregation.MetricManipulationFn;
-
-import java.util.Map;
+import org.apache.druid.query.context.ResponseContext;
public class MaterializedViewQueryQueryToolChest extends QueryToolChest
{
@@ -51,7 +50,7 @@ public class MaterializedViewQueryQueryToolChest extends QueryToolChest
{
return new QueryRunner() {
@Override
- public Sequence run(QueryPlus queryPlus, Map responseContext)
+ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext)
{
Query realQuery = getRealQuery(queryPlus.getQuery());
return warehouse.getToolChest(realQuery).mergeResults(runner).run(queryPlus.withQuery(realQuery), responseContext);
@@ -91,7 +90,7 @@ public class MaterializedViewQueryQueryToolChest extends QueryToolChest
{
return new QueryRunner() {
@Override
- public Sequence run(QueryPlus queryPlus, Map responseContext)
+ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext)
{
Query realQuery = getRealQuery(queryPlus.getQuery());
QueryToolChest realQueryToolChest = warehouse.getToolChest(realQuery);
diff --git a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryRunner.java b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryRunner.java
index 985688b..9c72808 100644
--- a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryRunner.java
+++ b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryRunner.java
@@ -27,8 +27,7 @@ import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
-
-import java.util.Map;
+import org.apache.druid.query.context.ResponseContext;
public class MaterializedViewQueryRunner<T> implements QueryRunner<T>
@@ -43,7 +42,7 @@ public class MaterializedViewQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
Query query = queryPlus.getQuery();
return new MergeSequence<>(
diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java
index 8834d0d..53c4422 100644
--- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java
+++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java
@@ -38,6 +38,7 @@ import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.Result;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.UnionDataSource;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.movingaverage.averagers.AveragerFactory;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
@@ -50,7 +51,6 @@ import org.joda.time.Interval;
import org.joda.time.Period;
import javax.annotation.Nullable;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
@@ -84,7 +84,7 @@ public class MovingAverageQueryRunner implements QueryRunner<Row>
}
@Override
- public Sequence<Row> run(QueryPlus<Row> query, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> query, ResponseContext responseContext)
{
MovingAverageQuery maq = (MovingAverageQuery) query.getQuery();
@@ -125,11 +125,11 @@ public class MovingAverageQueryRunner implements QueryRunner<Row>
.setContext(maq.getContext());
GroupByQuery gbq = builder.build();
- HashMap<String, Object> gbqResponse = new HashMap<>();
- gbqResponse.put(QUERY_FAIL_TIME, System.currentTimeMillis() + QueryContexts.getTimeout(gbq));
- gbqResponse.put(QUERY_TOTAL_BYTES_GATHERED, new AtomicLong());
+ ResponseContext gbqResponseContext = ResponseContext.createEmpty();
+ gbqResponseContext.put(QUERY_FAIL_TIME, System.currentTimeMillis() + QueryContexts.getTimeout(gbq));
+ gbqResponseContext.put(QUERY_TOTAL_BYTES_GATHERED, new AtomicLong());
- Sequence<Row> results = gbq.getRunner(walker).run(QueryPlus.wrap(gbq), gbqResponse);
+ Sequence<Row> results = gbq.getRunner(walker).run(QueryPlus.wrap(gbq), gbqResponseContext);
try {
// use localhost for remote address
requestLogger.logNativeQuery(RequestLogLine.forNative(
@@ -163,11 +163,11 @@ public class MovingAverageQueryRunner implements QueryRunner<Row>
0,
maq.getContext()
);
- HashMap<String, Object> tsqResponse = new HashMap<>();
- tsqResponse.put(QUERY_FAIL_TIME, System.currentTimeMillis() + QueryContexts.getTimeout(tsq));
- tsqResponse.put(QUERY_TOTAL_BYTES_GATHERED, new AtomicLong());
+ ResponseContext tsqResponseContext = ResponseContext.createEmpty();
+ tsqResponseContext.put(QUERY_FAIL_TIME, System.currentTimeMillis() + QueryContexts.getTimeout(tsq));
+ tsqResponseContext.put(QUERY_TOTAL_BYTES_GATHERED, new AtomicLong());
- Sequence<Result<TimeseriesResultValue>> results = tsq.getRunner(walker).run(QueryPlus.wrap(tsq), tsqResponse);
+ Sequence<Result<TimeseriesResultValue>> results = tsq.getRunner(walker).run(QueryPlus.wrap(tsq), tsqResponseContext);
try {
// use localhost for remote address
requestLogger.logNativeQuery(RequestLogLine.forNative(
diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java
index b5a8dbf..625e01f 100644
--- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java
+++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java
@@ -80,7 +80,6 @@ import java.io.InputStream;
import java.io.InputStreamReader;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Executor;
@@ -364,12 +363,11 @@ public class MovingAverageQueryTest
},
baseClient, warehouse, retryConfig, jsonMapper, serverConfig, null, new CacheConfig()
);
- final Map<String, Object> responseContext = new HashMap<>();
defineMocks();
QueryPlus queryPlus = QueryPlus.wrap(query);
- final Sequence<?> res = query.getRunner(walker).run(queryPlus, responseContext);
+ final Sequence<?> res = query.getRunner(walker).run(queryPlus);
List actualResults = new ArrayList();
actualResults = (List<MapBasedRow>) res.accumulate(actualResults, Accumulators.list());
diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java
index 5a28fbf..320f7d3 100644
--- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java
+++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java
@@ -52,7 +52,6 @@ import org.junit.rules.ExpectedException;
import java.io.IOException;
import java.nio.ByteBuffer;
-import java.util.HashMap;
import java.util.List;
public class MapVirtualColumnGroupByTest
@@ -141,7 +140,7 @@ public class MapVirtualColumnGroupByTest
expectedException.expect(UnsupportedOperationException.class);
expectedException.expectMessage("Map column doesn't support getRow()");
- runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ runner.run(QueryPlus.wrap(query)).toList();
}
@Test
@@ -162,7 +161,7 @@ public class MapVirtualColumnGroupByTest
null
);
- final List<Row> result = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ final List<Row> result = runner.run(QueryPlus.wrap(query)).toList();
final List<Row> expected = ImmutableList.of(
new MapBasedRow(
DateTimes.of("2011-01-12T00:00:00.000Z"),
diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java
index 4dd24c4..9139dfa 100644
--- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java
+++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java
@@ -22,7 +22,6 @@ package org.apache.druid.segment;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
-import com.google.common.collect.ImmutableMap;
import com.google.common.io.CharSource;
import org.apache.druid.data.input.impl.DelimitedParseSpec;
import org.apache.druid.data.input.impl.DimensionsSpec;
@@ -190,7 +189,7 @@ public class MapVirtualColumnSelectTest
private void checkSelectQuery(SelectQuery searchQuery, List<Map> expected)
{
- List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(searchQuery), ImmutableMap.of()).toList();
+ List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(searchQuery)).toList();
Assert.assertEquals(1, results.size());
List<EventHolder> events = results.get(0).getValue().getEvents();
diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java
index 761230d..af2368b 100644
--- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java
+++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java
@@ -50,7 +50,6 @@ import org.junit.rules.ExpectedException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
public class MapVirtualColumnTopNTest
@@ -105,7 +104,7 @@ public class MapVirtualColumnTopNTest
expectedException.expect(UnsupportedOperationException.class);
expectedException.expectMessage("Map column doesn't support getRow()");
- runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ runner.run(QueryPlus.wrap(query)).toList();
}
@Test
@@ -129,7 +128,7 @@ public class MapVirtualColumnTopNTest
null
);
- final List<Result<TopNResultValue>> result = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ final List<Result<TopNResultValue>> result = runner.run(QueryPlus.wrap(query)).toList();
final List<Result<TopNResultValue>> expected = Collections.singletonList(
new Result<>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java
index 23c843f..28806af 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java
@@ -48,7 +48,6 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -250,8 +249,6 @@ public class ApproximateHistogramTopNQueryTest
)
)
);
- HashMap<String, Object> context = new HashMap<String, Object>();
-
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
}
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java
index 3ed0a6e..7e12ee6 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java
@@ -48,7 +48,6 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -238,9 +237,8 @@ public class FixedBucketsHistogramTopNQueryTest
)
)
);
- HashMap<String, Object> context = new HashMap<String, Object>();
- List<Result<TopNResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ List<Result<TopNResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, results);
}
}
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
index 3fd6511..d906029 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
@@ -2359,7 +2359,7 @@ public class KafkaIndexTaskTest
{
ScanQuery query = new Druids.ScanQueryBuilder().dataSource(
DATA_SCHEMA.getDataSource()).intervals(spec).build();
- return task.getQueryRunner(query).run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ return task.getQueryRunner(query).run(QueryPlus.wrap(query)).toList();
}
private void insertData() throws ExecutionException, InterruptedException
@@ -2772,8 +2772,7 @@ public class KafkaIndexTaskTest
.intervals("0000/3000")
.build();
- List<Result<TimeseriesResultValue>> results =
- task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
+ List<Result<TimeseriesResultValue>> results = task.getQueryRunner(query).run(QueryPlus.wrap(query)).toList();
return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows"));
}
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
index 4fc7ba1..8eb4c14 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
@@ -2960,8 +2960,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
.intervals("0000/3000")
.build();
- List<Result<TimeseriesResultValue>> results =
- task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
+ List<Result<TimeseriesResultValue>> results = task.getQueryRunner(query).run(QueryPlus.wrap(query)).toList();
return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows"));
}
diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTimeseriesQueryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTimeseriesQueryTest.java
index 7e0e059..4656f00 100644
--- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTimeseriesQueryTest.java
+++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTimeseriesQueryTest.java
@@ -35,7 +35,6 @@ import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.util.Arrays;
-import java.util.HashMap;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
@@ -112,7 +111,7 @@ public class VarianceTimeseriesQueryTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java
index 3a93a65..0a493c1 100644
--- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java
+++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java
@@ -140,10 +140,7 @@ public class VarianceTopNQueryTest
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
final QueryRunner<Result<TopNResultValue>> mergeRunner = chest.mergeResults(runner);
- final Sequence<Result<TopNResultValue>> retval = mergeRunner.run(
- QueryPlus.wrap(query),
- ImmutableMap.of()
- );
+ final Sequence<Result<TopNResultValue>> retval = mergeRunner.run(QueryPlus.wrap(query));
TestHelper.assertExpectedResults(expectedResults, retval);
return retval;
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
index 7628dea..aee84c1 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
@@ -1641,7 +1641,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest
.build();
List<Result<TimeseriesResultValue>> results =
- task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
+ task.getQueryRunner(query).run(QueryPlus.wrap(query)).toList();
if (results.isEmpty()) {
return 0L;
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
index 316327b..4cddf34 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
@@ -1020,8 +1020,7 @@ public class RealtimeIndexTaskTest
.intervals("2000/3000")
.build();
- List<Result<TimeseriesResultValue>> results =
- task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
+ List<Result<TimeseriesResultValue>> results = task.getQueryRunner(query).run(QueryPlus.wrap(query)).toList();
if (results.isEmpty()) {
return 0L;
} else {
diff --git a/processing/src/main/java/org/apache/druid/query/AsyncQueryRunner.java b/processing/src/main/java/org/apache/druid/query/AsyncQueryRunner.java
index f714eb2..7b1aee0 100644
--- a/processing/src/main/java/org/apache/druid/query/AsyncQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/AsyncQueryRunner.java
@@ -25,8 +25,8 @@ import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import org.apache.druid.java.util.common.guava.LazySequence;
import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.context.ResponseContext;
-import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
@@ -47,7 +47,7 @@ public class AsyncQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
final Query<T> query = queryPlus.getQuery();
final int priority = QueryContexts.getPriority(query);
diff --git a/processing/src/main/java/org/apache/druid/query/BySegmentQueryRunner.java b/processing/src/main/java/org/apache/druid/query/BySegmentQueryRunner.java
index b6a74ca..eced1f3 100644
--- a/processing/src/main/java/org/apache/druid/query/BySegmentQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/BySegmentQueryRunner.java
@@ -21,12 +21,12 @@ package org.apache.druid.query;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTime;
import java.util.Collections;
import java.util.List;
-import java.util.Map;
/**
*/
@@ -45,7 +45,7 @@ public class BySegmentQueryRunner<T> implements QueryRunner<T>
@Override
@SuppressWarnings("unchecked")
- public Sequence<T> run(final QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, ResponseContext responseContext)
{
if (QueryContexts.isBySegment(queryPlus.getQuery())) {
final Sequence<T> baseSequence = base.run(queryPlus, responseContext);
diff --git a/processing/src/main/java/org/apache/druid/query/BySegmentSkippingQueryRunner.java b/processing/src/main/java/org/apache/druid/query/BySegmentSkippingQueryRunner.java
index 0a5d5de..7061d47 100644
--- a/processing/src/main/java/org/apache/druid/query/BySegmentSkippingQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/BySegmentSkippingQueryRunner.java
@@ -21,8 +21,7 @@ package org.apache.druid.query;
import org.apache.druid.java.util.common.guava.Sequence;
-
-import java.util.Map;
+import org.apache.druid.query.context.ResponseContext;
/**
*/
@@ -38,7 +37,7 @@ public abstract class BySegmentSkippingQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
if (QueryContexts.isBySegment(queryPlus.getQuery())) {
return baseRunner.run(queryPlus, responseContext);
@@ -47,5 +46,5 @@ public abstract class BySegmentSkippingQueryRunner<T> implements QueryRunner<T>
return doRun(baseRunner, queryPlus, responseContext);
}
- protected abstract Sequence<T> doRun(QueryRunner<T> baseRunner, QueryPlus<T> queryPlus, Map<String, Object> context);
+ protected abstract Sequence<T> doRun(QueryRunner<T> baseRunner, QueryPlus<T> queryPlus, ResponseContext context);
}
diff --git a/processing/src/main/java/org/apache/druid/query/CPUTimeMetricQueryRunner.java b/processing/src/main/java/org/apache/druid/query/CPUTimeMetricQueryRunner.java
index 5bbe941..594a327 100644
--- a/processing/src/main/java/org/apache/druid/query/CPUTimeMetricQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/CPUTimeMetricQueryRunner.java
@@ -25,9 +25,9 @@ import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.SequenceWrapper;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.utils.JvmUtils;
-import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
public class CPUTimeMetricQueryRunner<T> implements QueryRunner<T>
@@ -58,7 +58,7 @@ public class CPUTimeMetricQueryRunner<T> implements QueryRunner<T>
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
final QueryPlus<T> queryWithMetrics = queryPlus.withQueryMetrics(queryToolChest);
final Sequence<T> baseSequence = delegate.run(queryWithMetrics, responseContext);
diff --git a/processing/src/main/java/org/apache/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/org/apache/druid/query/ChainedExecutionQueryRunner.java
index c74c4c3..08a0e4d 100644
--- a/processing/src/main/java/org/apache/druid/query/ChainedExecutionQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/ChainedExecutionQueryRunner.java
@@ -32,11 +32,11 @@ import org.apache.druid.java.util.common.guava.BaseSequence;
import org.apache.druid.java.util.common.guava.MergeIterable;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
-import java.util.Map;
import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
@@ -87,7 +87,7 @@ public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
Query<T> query = queryPlus.getQuery();
final int priority = QueryContexts.getPriority(query);
diff --git a/processing/src/main/java/org/apache/druid/query/ConcatQueryRunner.java b/processing/src/main/java/org/apache/druid/query/ConcatQueryRunner.java
index 9392233..1a22b3e 100644
--- a/processing/src/main/java/org/apache/druid/query/ConcatQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/ConcatQueryRunner.java
@@ -22,8 +22,7 @@ package org.apache.druid.query;
import com.google.common.base.Function;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
-
-import java.util.Map;
+import org.apache.druid.query.context.ResponseContext;
/**
*/
@@ -37,7 +36,7 @@ public class ConcatQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
return Sequences.concat(
Sequences.map(
diff --git a/processing/src/main/java/org/apache/druid/query/FinalizeResultsQueryRunner.java b/processing/src/main/java/org/apache/druid/query/FinalizeResultsQueryRunner.java
index db00640..ebb18f2 100644
--- a/processing/src/main/java/org/apache/druid/query/FinalizeResultsQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/FinalizeResultsQueryRunner.java
@@ -27,8 +27,7 @@ import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.aggregation.MetricManipulationFn;
import org.apache.druid.query.aggregation.MetricManipulatorFns;
-
-import java.util.Map;
+import org.apache.druid.query.context.ResponseContext;
/**
*/
@@ -47,7 +46,7 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, ResponseContext responseContext)
{
final Query<T> query = queryPlus.getQuery();
final boolean isBySegment = QueryContexts.isBySegment(query);
diff --git a/processing/src/main/java/org/apache/druid/query/FluentQueryRunnerBuilder.java b/processing/src/main/java/org/apache/druid/query/FluentQueryRunnerBuilder.java
index 5990058..1535869 100644
--- a/processing/src/main/java/org/apache/druid/query/FluentQueryRunnerBuilder.java
+++ b/processing/src/main/java/org/apache/druid/query/FluentQueryRunnerBuilder.java
@@ -21,8 +21,8 @@ package org.apache.druid.query;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.query.context.ResponseContext;
-import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
public class FluentQueryRunnerBuilder<T>
@@ -49,7 +49,7 @@ public class FluentQueryRunnerBuilder<T>
}
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
return baseRunner.run(queryPlus, responseContext);
}
diff --git a/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java b/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java
index 04c6f6c..9c46d60 100644
--- a/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java
@@ -37,6 +37,7 @@ import org.apache.druid.java.util.common.guava.Accumulator;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.GroupByQueryHelper;
@@ -44,7 +45,6 @@ import org.apache.druid.segment.incremental.IncrementalIndex;
import java.nio.ByteBuffer;
import java.util.List;
-import java.util.Map;
import java.util.Queue;
import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException;
@@ -78,7 +78,7 @@ public class GroupByMergedQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
final GroupByQuery query = (GroupByQuery) queryPlus.getQuery();
final GroupByQueryConfig querySpecificConfig = configSupplier.get().withOverrides(query);
diff --git a/processing/src/main/java/org/apache/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/org/apache/druid/query/IntervalChunkingQueryRunner.java
index 5bc76d0..848d0fc 100644
--- a/processing/src/main/java/org/apache/druid/query/IntervalChunkingQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/IntervalChunkingQueryRunner.java
@@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.guava.FunctionalIterable;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@@ -36,7 +37,6 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
-import java.util.Map;
import java.util.concurrent.ExecutorService;
/**
@@ -69,7 +69,7 @@ public class IntervalChunkingQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
final Period chunkPeriod = getChunkPeriod(queryPlus.getQuery());
diff --git a/processing/src/main/java/org/apache/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/org/apache/druid/query/MetricsEmittingQueryRunner.java
index aae187d..4baef60 100644
--- a/processing/src/main/java/org/apache/druid/query/MetricsEmittingQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/MetricsEmittingQueryRunner.java
@@ -25,8 +25,8 @@ import org.apache.druid.java.util.common.guava.SequenceWrapper;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.query.context.ResponseContext;
-import java.util.Map;
import java.util.function.Consumer;
import java.util.function.ObjLongConsumer;
@@ -83,7 +83,7 @@ public class MetricsEmittingQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
QueryPlus<T> queryWithMetrics = queryPlus.withQueryMetrics(queryToolChest);
final QueryMetrics<?> queryMetrics = queryWithMetrics.getQueryMetrics();
diff --git a/processing/src/main/java/org/apache/druid/query/NoopQueryRunner.java b/processing/src/main/java/org/apache/druid/query/NoopQueryRunner.java
index ab43dcb..f450431 100644
--- a/processing/src/main/java/org/apache/druid/query/NoopQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/NoopQueryRunner.java
@@ -21,15 +21,14 @@ package org.apache.druid.query;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
-
-import java.util.Map;
+import org.apache.druid.query.context.ResponseContext;
/**
*/
public class NoopQueryRunner<T> implements QueryRunner<T>
{
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
return Sequences.empty();
}
diff --git a/processing/src/main/java/org/apache/druid/query/PerSegmentOptimizingQueryRunner.java b/processing/src/main/java/org/apache/druid/query/PerSegmentOptimizingQueryRunner.java
index 0e51294..48f4d28 100644
--- a/processing/src/main/java/org/apache/druid/query/PerSegmentOptimizingQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/PerSegmentOptimizingQueryRunner.java
@@ -20,8 +20,7 @@
package org.apache.druid.query;
import org.apache.druid.java.util.common.guava.Sequence;
-
-import java.util.Map;
+import org.apache.druid.query.context.ResponseContext;
/**
* This runner optimizes queries made on a single segment, using per-segment information,
@@ -49,7 +48,7 @@ public class PerSegmentOptimizingQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> input, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> input, final ResponseContext responseContext)
{
return base.run(
input.optimizeForSegment(optimizationContext),
diff --git a/processing/src/main/java/org/apache/druid/query/QueryPlus.java b/processing/src/main/java/org/apache/druid/query/QueryPlus.java
index caa2350..f1884d3 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryPlus.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryPlus.java
@@ -23,10 +23,10 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.spec.QuerySegmentSpec;
import javax.annotation.Nullable;
-import java.util.Map;
/**
* An immutable composite object of {@link Query} + extra stuff needed in {@link QueryRunner}s.
@@ -153,7 +153,7 @@ public final class QueryPlus<T>
return new QueryPlus<>(replacementQuery, queryMetrics, identity);
}
- public Sequence<T> run(QuerySegmentWalker walker, Map<String, Object> context)
+ public Sequence<T> run(QuerySegmentWalker walker, ResponseContext context)
{
return query.getRunner(walker).run(this, context);
}
diff --git a/processing/src/main/java/org/apache/druid/query/QueryRunner.java b/processing/src/main/java/org/apache/druid/query/QueryRunner.java
index a7d62d4..3cd4f2f 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryRunner.java
@@ -19,10 +19,10 @@
package org.apache.druid.query;
+import com.google.common.annotations.VisibleForTesting;
import org.apache.druid.guice.annotations.ExtensionPoint;
import org.apache.druid.java.util.common.guava.Sequence;
-
-import java.util.Map;
+import org.apache.druid.query.context.ResponseContext;
@ExtensionPoint
public interface QueryRunner<T>
@@ -30,5 +30,11 @@ public interface QueryRunner<T>
/**
* Runs the given query and returns results in a time-ordered sequence.
*/
- Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext);
+ Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext);
+
+ @VisibleForTesting
+ default Sequence<T> run(QueryPlus<T> queryPlus)
+ {
+ return this.run(queryPlus, ResponseContext.createEmpty());
+ }
}
diff --git a/processing/src/main/java/org/apache/druid/query/QueryRunnerHelper.java b/processing/src/main/java/org/apache/druid/query/QueryRunnerHelper.java
index c99b6ae..8788778 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryRunnerHelper.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryRunnerHelper.java
@@ -24,6 +24,7 @@ import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.StorageAdapter;
@@ -32,7 +33,6 @@ import org.joda.time.Interval;
import java.io.Closeable;
import java.util.List;
-import java.util.Map;
import java.util.Objects;
/**
@@ -67,7 +67,7 @@ public class QueryRunnerHelper
return new QueryRunner<T>()
{
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
return Sequences.withBaggage(runner.run(queryPlus, responseContext), closeable);
}
diff --git a/processing/src/main/java/org/apache/druid/query/ReferenceCountingSegmentQueryRunner.java b/processing/src/main/java/org/apache/druid/query/ReferenceCountingSegmentQueryRunner.java
index 0aaa61f..ade3575 100644
--- a/processing/src/main/java/org/apache/druid/query/ReferenceCountingSegmentQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/ReferenceCountingSegmentQueryRunner.java
@@ -21,10 +21,9 @@ package org.apache.druid.query;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.ReferenceCountingSegment;
-import java.util.Map;
-
/**
*/
public class ReferenceCountingSegmentQueryRunner<T> implements QueryRunner<T>
@@ -45,7 +44,7 @@ public class ReferenceCountingSegmentQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, ResponseContext responseContext)
{
if (adapter.increment()) {
try {
diff --git a/processing/src/main/java/org/apache/druid/query/ReportTimelineMissingSegmentQueryRunner.java b/processing/src/main/java/org/apache/druid/query/ReportTimelineMissingSegmentQueryRunner.java
index 471b4cd..97b6aa2 100644
--- a/processing/src/main/java/org/apache/druid/query/ReportTimelineMissingSegmentQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/ReportTimelineMissingSegmentQueryRunner.java
@@ -21,10 +21,10 @@ package org.apache.druid.query;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.context.ResponseContext;
import java.util.ArrayList;
import java.util.List;
-import java.util.Map;
/**
*/
@@ -38,13 +38,13 @@ public class ReportTimelineMissingSegmentQueryRunner<T> implements QueryRunner<T
}
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
List<SegmentDescriptor> missingSegments =
- (List<SegmentDescriptor>) responseContext.get(Result.MISSING_SEGMENTS_KEY);
+ (List<SegmentDescriptor>) responseContext.get(ResponseContext.CTX_MISSING_SEGMENTS);
if (missingSegments == null) {
missingSegments = new ArrayList<>();
- responseContext.put(Result.MISSING_SEGMENTS_KEY, missingSegments);
+ responseContext.put(ResponseContext.CTX_MISSING_SEGMENTS, missingSegments);
}
missingSegments.add(descriptor);
return Sequences.empty();
diff --git a/processing/src/main/java/org/apache/druid/query/Result.java b/processing/src/main/java/org/apache/druid/query/Result.java
index 728a193..fe0ed50 100644
--- a/processing/src/main/java/org/apache/druid/query/Result.java
+++ b/processing/src/main/java/org/apache/druid/query/Result.java
@@ -34,9 +34,6 @@ import java.util.function.Function;
@PublicApi
public class Result<T> implements Comparable<Result<T>>
{
- public static final String MISSING_SEGMENTS_KEY = "missingSegments";
-
- @Nullable
private final DateTime timestamp;
private final T value;
diff --git a/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java b/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java
index f3f34c6..6361479 100644
--- a/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java
@@ -22,9 +22,9 @@ package org.apache.druid.query;
import org.apache.druid.common.guava.CombiningSequence;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.context.ResponseContext;
import java.util.Comparator;
-import java.util.Map;
import java.util.function.BinaryOperator;
import java.util.function.Function;
@@ -48,7 +48,7 @@ public class ResultMergeQueryRunner<T> extends BySegmentSkippingQueryRunner<T>
}
@Override
- public Sequence<T> doRun(QueryRunner<T> baseRunner, QueryPlus<T> queryPlus, Map<String, Object> context)
+ public Sequence<T> doRun(QueryRunner<T> baseRunner, QueryPlus<T> queryPlus, ResponseContext context)
{
Query<T> query = queryPlus.getQuery();
return CombiningSequence.create(
diff --git a/processing/src/main/java/org/apache/druid/query/RetryQueryRunner.java b/processing/src/main/java/org/apache/druid/query/RetryQueryRunner.java
index 5410603..28bcf0b 100644
--- a/processing/src/main/java/org/apache/druid/query/RetryQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/RetryQueryRunner.java
@@ -29,12 +29,12 @@ import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.java.util.common.guava.YieldingAccumulator;
import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
import org.apache.druid.segment.SegmentMissingException;
import java.util.ArrayList;
import java.util.List;
-import java.util.Map;
public class RetryQueryRunner<T> implements QueryRunner<T>
{
@@ -56,7 +56,7 @@ public class RetryQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> context)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
{
final List<Sequence<T>> listOfSequences = new ArrayList<>();
listOfSequences.add(baseRunner.run(queryPlus, context));
@@ -72,7 +72,7 @@ public class RetryQueryRunner<T> implements QueryRunner<T>
for (int i = 0; i < config.getNumTries(); i++) {
log.info("[%,d] missing segments found. Retry attempt [%,d]", missingSegments.size(), i);
- context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
+ context.put(ResponseContext.CTX_MISSING_SEGMENTS, new ArrayList<>());
final QueryPlus<T> retryQueryPlus = queryPlus.withQuerySegmentSpec(
new MultipleSpecificSegmentSpec(
missingSegments
@@ -100,9 +100,9 @@ public class RetryQueryRunner<T> implements QueryRunner<T>
};
}
- private List<SegmentDescriptor> getMissingSegments(final Map<String, Object> context)
+ private List<SegmentDescriptor> getMissingSegments(final ResponseContext context)
{
- final Object maybeMissingSegments = context.get(Result.MISSING_SEGMENTS_KEY);
+ final Object maybeMissingSegments = context.get(ResponseContext.CTX_MISSING_SEGMENTS);
if (maybeMissingSegments == null) {
return new ArrayList<>();
}
diff --git a/processing/src/main/java/org/apache/druid/query/SubqueryQueryRunner.java b/processing/src/main/java/org/apache/druid/query/SubqueryQueryRunner.java
index 25eb1f6..a6bcbb8 100644
--- a/processing/src/main/java/org/apache/druid/query/SubqueryQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/SubqueryQueryRunner.java
@@ -21,10 +21,9 @@ package org.apache.druid.query;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.groupby.GroupByQueryConfig;
-import java.util.Map;
-
/**
* If there's a subquery, run it instead of the outer query
*/
@@ -38,7 +37,7 @@ public class SubqueryQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, ResponseContext responseContext)
{
DataSource dataSource = queryPlus.getQuery().getDataSource();
boolean forcePushDownNestedQuery = queryPlus.getQuery()
diff --git a/processing/src/main/java/org/apache/druid/query/TimewarpOperator.java b/processing/src/main/java/org/apache/druid/query/TimewarpOperator.java
index 827e780..0beca68 100644
--- a/processing/src/main/java/org/apache/druid/query/TimewarpOperator.java
+++ b/processing/src/main/java/org/apache/druid/query/TimewarpOperator.java
@@ -26,6 +26,7 @@ import org.apache.druid.data.input.MapBasedRow;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.timeboundary.TimeBoundaryQuery;
import org.apache.druid.query.timeboundary.TimeBoundaryResultValue;
@@ -35,7 +36,6 @@ import org.joda.time.Interval;
import org.joda.time.Period;
import java.util.Collections;
-import java.util.Map;
/**
* TimewarpOperator is an example post-processing operator that maps current time
@@ -79,7 +79,7 @@ public class TimewarpOperator<T> implements PostProcessingOperator<T>
return new QueryRunner<T>()
{
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
final DateTimeZone tz = queryPlus.getQuery().getTimezone();
final long offset = computeOffset(now, tz);
diff --git a/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java b/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java
index f5105ab..f0b2cb5 100644
--- a/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java
@@ -24,8 +24,7 @@ import com.google.common.collect.Lists;
import org.apache.druid.java.util.common.guava.MergeSequence;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
-
-import java.util.Map;
+import org.apache.druid.query.context.ResponseContext;
public class UnionQueryRunner<T> implements QueryRunner<T>
{
@@ -39,7 +38,7 @@ public class UnionQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
Query<T> query = queryPlus.getQuery();
DataSource dataSource = query.getDataSource();
diff --git a/processing/src/main/java/org/apache/druid/query/NoopQueryRunner.java b/processing/src/main/java/org/apache/druid/query/context/ConcurrentResponseContext.java
similarity index 59%
copy from processing/src/main/java/org/apache/druid/query/NoopQueryRunner.java
copy to processing/src/main/java/org/apache/druid/query/context/ConcurrentResponseContext.java
index ab43dcb..48838f1 100644
--- a/processing/src/main/java/org/apache/druid/query/NoopQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/context/ConcurrentResponseContext.java
@@ -17,20 +17,29 @@
* under the License.
*/
-package org.apache.druid.query;
+package org.apache.druid.query.context;
-import org.apache.druid.java.util.common.guava.Sequence;
-import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.guice.annotations.PublicApi;
import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
/**
-*/
-public class NoopQueryRunner<T> implements QueryRunner<T>
+ * The implementation of {@link ResponseContext} with a {@link ConcurrentHashMap} as a delegate
+ */
+@PublicApi
+public class ConcurrentResponseContext extends ResponseContext
{
+ public static ConcurrentResponseContext createEmpty()
+ {
+ return new ConcurrentResponseContext();
+ }
+
+ private final ConcurrentHashMap<String, Object> delegate = new ConcurrentHashMap<>();
+
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ protected Map<String, Object> getDelegate()
{
- return Sequences.empty();
+ return delegate;
}
}
diff --git a/processing/src/main/java/org/apache/druid/query/NoopQueryRunner.java b/processing/src/main/java/org/apache/druid/query/context/DefaultResponseContext.java
similarity index 62%
copy from processing/src/main/java/org/apache/druid/query/NoopQueryRunner.java
copy to processing/src/main/java/org/apache/druid/query/context/DefaultResponseContext.java
index ab43dcb..adff1ff 100644
--- a/processing/src/main/java/org/apache/druid/query/NoopQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/context/DefaultResponseContext.java
@@ -17,20 +17,29 @@
* under the License.
*/
-package org.apache.druid.query;
+package org.apache.druid.query.context;
-import org.apache.druid.java.util.common.guava.Sequence;
-import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.guice.annotations.PublicApi;
+import java.util.HashMap;
import java.util.Map;
/**
-*/
-public class NoopQueryRunner<T> implements QueryRunner<T>
+ * The implementation of {@link ResponseContext} with a HashMap as a delegate
+ */
+@PublicApi
+public class DefaultResponseContext extends ResponseContext
{
+ public static DefaultResponseContext createEmpty()
+ {
+ return new DefaultResponseContext();
+ }
+
+ private final HashMap<String, Object> delegate = new HashMap<>();
+
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ protected Map<String, Object> getDelegate()
{
- return Sequences.empty();
+ return delegate;
}
}
diff --git a/processing/src/main/java/org/apache/druid/query/context/ResponseContext.java b/processing/src/main/java/org/apache/druid/query/context/ResponseContext.java
new file mode 100644
index 0000000..93841f4
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/context/ResponseContext.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.context;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.guice.annotations.PublicApi;
+import org.apache.druid.java.util.common.jackson.JacksonUtils;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * The context for storing and passing data between chains of {@link org.apache.druid.query.QueryRunner}s.
+ * The context is also transferred between Druid nodes with all the data it contains.
+ * All the keys associated with data inside the context should be stored here.
+ * CTX_* keys might be aggregated into an enum. Consider refactoring that.
+ */
+@PublicApi
+public abstract class ResponseContext
+{
+ /**
+ * Lists intervals for which NO segment is present.
+ */
+ public static final String CTX_UNCOVERED_INTERVALS = "uncoveredIntervals";
+ /**
+ * Indicates if the number of uncovered intervals exceeded the limit (true/false).
+ */
+ public static final String CTX_UNCOVERED_INTERVALS_OVERFLOWED = "uncoveredIntervalsOverflowed";
+ /**
+ * Lists missing segments.
+ */
+ public static final String CTX_MISSING_SEGMENTS = "missingSegments";
+ /**
+ * Entity tag. A part of HTTP cache validation mechanism.
+ * Is being removed from the context before sending and used as a separate HTTP header.
+ */
+ public static final String CTX_ETAG = "ETag";
+ /**
+ * Query total bytes gathered.
+ */
+ public static final String CTX_QUERY_TOTAL_BYTES_GATHERED = "queryTotalBytesGathered";
+ /**
+ * This variable indicates when a running query should be expired,
+ * and is effective only when 'timeout' of queryContext has a positive value.
+ */
+ public static final String CTX_TIMEOUT_AT = "timeoutAt";
+ /**
+ * The number of scanned rows.
+ */
+ public static final String CTX_COUNT = "count";
+
+ /**
+ * Create an empty DefaultResponseContext instance
+ * @return empty DefaultResponseContext instance
+ */
+ public static ResponseContext createEmpty()
+ {
+ return DefaultResponseContext.createEmpty();
+ }
+
+ protected abstract Map<String, Object> getDelegate();
+
+ public Object put(String key, Object value)
+ {
+ return getDelegate().put(key, value);
+ }
+
+ public Object get(String key)
+ {
+ return getDelegate().get(key);
+ }
+
+ public Object remove(String key)
+ {
+ return getDelegate().remove(key);
+ }
+
+ public void putAll(Map<? extends String, ?> m)
+ {
+ getDelegate().putAll(m);
+ }
+
+ public void putAll(ResponseContext responseContext)
+ {
+ getDelegate().putAll(responseContext.getDelegate());
+ }
+
+ public int size()
+ {
+ return getDelegate().size();
+ }
+
+ public String serializeWith(ObjectMapper objectMapper) throws JsonProcessingException
+ {
+ return objectMapper.writeValueAsString(getDelegate());
+ }
+
+ public static ResponseContext deserialize(String responseContext, ObjectMapper objectMapper) throws IOException
+ {
+ final Map<String, Object> delegate = objectMapper.readValue(
+ responseContext,
+ JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
+ );
+ return new ResponseContext()
+ {
+ @Override
+ protected Map<String, Object> getDelegate()
+ {
+ return delegate;
+ }
+ };
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java
index 9f4c271..73e0fab 100644
--- a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java
@@ -31,11 +31,11 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.Result;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.StorageAdapter;
import java.util.Iterator;
-import java.util.Map;
import java.util.concurrent.ExecutorService;
/**
@@ -89,7 +89,7 @@ public class DataSourceMetadataQueryRunnerFactory
@Override
public Sequence<Result<DataSourceMetadataResultValue>> run(
QueryPlus<Result<DataSourceMetadataResultValue>> input,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
Query<Result<DataSourceMetadataResultValue>> query = input.getQuery();
diff --git a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java
index 7b951c8..dbe8922 100644
--- a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java
@@ -35,10 +35,10 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.MetricManipulationFn;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.timeline.LogicalSegment;
import java.util.List;
-import java.util.Map;
import java.util.stream.Collectors;
/**
@@ -82,7 +82,7 @@ public class DataSourceQueryQueryToolChest
protected Sequence<Result<DataSourceMetadataResultValue>> doRun(
QueryRunner<Result<DataSourceMetadataResultValue>> baseRunner,
QueryPlus<Result<DataSourceMetadataResultValue>> input,
- Map<String, Object> context
+ ResponseContext context
)
{
DataSourceMetadataQuery query = (DataSourceMetadataQuery) input.getQuery();
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
index b81b255..1b58b20 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
@@ -53,6 +53,7 @@ import org.apache.druid.query.aggregation.MetricManipulationFn;
import org.apache.druid.query.aggregation.MetricManipulatorFns;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.extraction.ExtractionFn;
@@ -145,7 +146,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
private Sequence<Row> initAndMergeGroupByResults(
final GroupByQuery query,
QueryRunner<Row> runner,
- Map<String, Object> context
+ ResponseContext context
)
{
final GroupByStrategy groupByStrategy = strategySelector.strategize(query);
@@ -159,7 +160,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
final GroupByQuery query,
GroupByQueryResource resource,
QueryRunner<Row> runner,
- Map<String, Object> context
+ ResponseContext context
)
{
if (isNestedQueryPushDown(query, groupByStrategy)) {
@@ -173,7 +174,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
GroupByQuery query,
GroupByQueryResource resource,
QueryRunner<Row> runner,
- Map<String, Object> context
+ ResponseContext context
)
{
// If there's a subquery, merge subquery results and then apply the aggregator
@@ -254,7 +255,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
GroupByQuery query,
GroupByQueryResource resource,
QueryRunner<Row> runner,
- Map<String, Object> context
+ ResponseContext context
)
{
Sequence<Row> pushDownQueryResults = groupByStrategy.mergeResults(runner, query, context);
@@ -417,7 +418,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
GroupByQuery groupByQuery = (GroupByQuery) queryPlus.getQuery();
if (groupByQuery.getDimFilter() != null) {
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
index 91bd4d3..6d7c5ef 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
@@ -31,11 +31,11 @@ import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.StorageAdapter;
-import java.util.Map;
import java.util.concurrent.ExecutorService;
/**
@@ -70,7 +70,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
return new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
QueryRunner<Row> rowQueryRunner = strategySelector.strategize((GroupByQuery) queryPlus.getQuery()).mergeRunners(
queryExecutor,
@@ -99,7 +99,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
}
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
Query<Row> query = queryPlus.getQuery();
if (!(query instanceof GroupByQuery)) {
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java
index ef8c304..cccb3e5 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java
@@ -52,6 +52,7 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.ResourceLimitExceededException;
import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper.RowBasedKey;
@@ -61,7 +62,6 @@ import java.io.File;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
-import java.util.Map;
import java.util.UUID;
import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException;
@@ -108,7 +108,7 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<Row>
}
@Override
- public Sequence<Row> run(final QueryPlus<Row> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<Row> run(final QueryPlus<Row> queryPlus, final ResponseContext responseContext)
{
final GroupByQuery query = (GroupByQuery) queryPlus.getQuery();
final GroupByQueryConfig querySpecificConfig = config.withOverrides(query);
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java
index 56035a4..b8dace9 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java
@@ -27,6 +27,7 @@ import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
import org.apache.druid.query.groupby.resource.GroupByQueryResource;
@@ -34,7 +35,6 @@ import org.apache.druid.segment.StorageAdapter;
import javax.annotation.Nullable;
import java.util.Comparator;
-import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.function.BinaryOperator;
@@ -72,7 +72,7 @@ public interface GroupByStrategy
GroupByQueryQueryToolChest toolChest
);
- Sequence<Row> mergeResults(QueryRunner<Row> baseRunner, GroupByQuery query, Map<String, Object> responseContext);
+ Sequence<Row> mergeResults(QueryRunner<Row> baseRunner, GroupByQuery query, ResponseContext responseContext);
/**
* See {@link org.apache.druid.query.QueryToolChest#createMergeFn(Query)} for details, allows
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java
index ee6f473..69f8790 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java
@@ -39,6 +39,7 @@ import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
@@ -56,7 +57,6 @@ import org.joda.time.Interval;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashSet;
-import java.util.Map;
import java.util.Set;
public class GroupByStrategyV1 implements GroupByStrategy
@@ -112,7 +112,7 @@ public class GroupByStrategyV1 implements GroupByStrategy
public Sequence<Row> mergeResults(
final QueryRunner<Row> baseRunner,
final GroupByQuery query,
- final Map<String, Object> responseContext
+ final ResponseContext responseContext
)
{
final IncrementalIndex index = GroupByQueryHelper.makeIncrementalIndex(
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java
index 3167f5f..72801a0 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java
@@ -55,6 +55,7 @@ import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.ResourceLimitExceededException;
import org.apache.druid.query.ResultMergeQueryRunner;
import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.groupby.GroupByQuery;
@@ -229,7 +230,7 @@ public class GroupByStrategyV2 implements GroupByStrategy
public Sequence<Row> mergeResults(
final QueryRunner<Row> baseRunner,
final GroupByQuery query,
- final Map<String, Object> responseContext
+ final ResponseContext responseContext
)
{
// Merge streams using ResultMergeQueryRunner, then apply postaggregators, then apply limit (which may
@@ -363,7 +364,7 @@ public class GroupByStrategyV2 implements GroupByStrategy
mergeResults(new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
return GroupByRowProcessor.getRowsFromGrouper(
query,
@@ -440,7 +441,7 @@ public class GroupByStrategyV2 implements GroupByStrategy
mergeResults(new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
return GroupByRowProcessor.getRowsFromGrouper(
queryWithoutSubtotalsSpec,
diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java
index b2d1a2a..989abc2 100644
--- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java
@@ -50,6 +50,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
import org.apache.druid.query.aggregation.MetricManipulationFn;
import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery;
@@ -112,7 +113,7 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
public Sequence<SegmentAnalysis> doRun(
QueryRunner<SegmentAnalysis> baseRunner,
QueryPlus<SegmentAnalysis> queryPlus,
- Map<String, Object> context
+ ResponseContext context
)
{
SegmentMetadataQuery updatedQuery = ((SegmentMetadataQuery) queryPlus.getQuery()).withFinalizedAnalysisTypes(config);
diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java
index 3d85572..ba5c582 100644
--- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java
@@ -40,6 +40,7 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
import org.apache.druid.query.metadata.metadata.ColumnIncluderator;
import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
@@ -83,7 +84,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
return new QueryRunner<SegmentAnalysis>()
{
@Override
- public Sequence<SegmentAnalysis> run(QueryPlus<SegmentAnalysis> inQ, Map<String, Object> responseContext)
+ public Sequence<SegmentAnalysis> run(QueryPlus<SegmentAnalysis> inQ, ResponseContext responseContext)
{
SegmentMetadataQuery updatedQuery = ((SegmentMetadataQuery) inQ.getQuery())
.withFinalizedAnalysisTypes(toolChest.getConfig());
@@ -201,7 +202,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
@Override
public Sequence<SegmentAnalysis> run(
final QueryPlus<SegmentAnalysis> queryPlus,
- final Map<String, Object> responseContext
+ final ResponseContext responseContext
)
{
final Query<SegmentAnalysis> query = queryPlus.getQuery();
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
index 2a5edbd..8d0bf51 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
@@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryInterruptedException;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.Segment;
@@ -60,20 +61,20 @@ public class ScanQueryEngine
public Sequence<ScanResultValue> process(
final ScanQuery query,
final Segment segment,
- final Map<String, Object> responseContext
+ final ResponseContext responseContext
)
{
// "legacy" should be non-null due to toolChest.mergeResults
final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "WTF?! Expected non-null legacy");
- if (responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) != null) {
- long count = (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT);
+ if (responseContext.get(ResponseContext.CTX_COUNT) != null) {
+ long count = (long) responseContext.get(ResponseContext.CTX_COUNT);
if (count >= query.getLimit() && query.getOrder().equals(ScanQuery.Order.NONE)) {
return Sequences.empty();
}
}
final boolean hasTimeout = QueryContexts.hasTimeout(query);
- final long timeoutAt = (long) responseContext.get(ScanQueryRunnerFactory.CTX_TIMEOUT_AT);
+ final long timeoutAt = (long) responseContext.get(ResponseContext.CTX_TIMEOUT_AT);
final long start = System.currentTimeMillis();
final StorageAdapter adapter = segment.asStorageAdapter();
@@ -120,8 +121,8 @@ public class ScanQueryEngine
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()));
- if (responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) == null) {
- responseContext.put(ScanQueryRunnerFactory.CTX_COUNT, 0L);
+ if (responseContext.get(ResponseContext.CTX_COUNT) == null) {
+ responseContext.put(ResponseContext.CTX_COUNT, 0L);
}
final long limit = calculateLimit(query, responseContext);
return Sequences.concat(
@@ -187,12 +188,12 @@ public class ScanQueryEngine
throw new UOE("resultFormat[%s] is not supported", resultFormat.toString());
}
responseContext.put(
- ScanQueryRunnerFactory.CTX_COUNT,
- (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) + (offset - lastOffset)
+ ResponseContext.CTX_COUNT,
+ (long) responseContext.get(ResponseContext.CTX_COUNT) + (offset - lastOffset)
);
if (hasTimeout) {
responseContext.put(
- ScanQueryRunnerFactory.CTX_TIMEOUT_AT,
+ ResponseContext.CTX_TIMEOUT_AT,
timeoutAt - (System.currentTimeMillis() - start)
);
}
@@ -262,10 +263,10 @@ public class ScanQueryEngine
* If we're performing time-ordering, we want to scan through the first `limit` rows in each segment ignoring the number
* of rows already counted on other segments.
*/
- private long calculateLimit(ScanQuery query, Map<String, Object> responseContext)
+ private long calculateLimit(ScanQuery query, ResponseContext responseContext)
{
if (query.getOrder().equals(ScanQuery.Order.NONE)) {
- return query.getLimit() - (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT);
+ return query.getLimit() - (long) responseContext.get(ResponseContext.CTX_COUNT);
}
return query.getLimit();
}
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java
index f246d7c..4e30e86 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java
@@ -29,11 +29,11 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
+import org.apache.druid.query.context.ResponseContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
-import java.util.Map;
/**
* This iterator supports iteration through a Sequence returned by a ScanResultValue QueryRunner. Its behaviour
@@ -60,7 +60,7 @@ public class ScanQueryLimitRowIterator implements CloseableIterator<ScanResultVa
public ScanQueryLimitRowIterator(
QueryRunner<ScanResultValue> baseRunner,
QueryPlus<ScanResultValue> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
this.query = (ScanQuery) queryPlus.getQuery();
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java
index c34d58c..570819a 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java
@@ -39,6 +39,7 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.SinkQueryRunners;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.query.spec.SpecificSegmentSpec;
@@ -52,17 +53,12 @@ import java.util.Comparator;
import java.util.Deque;
import java.util.LinkedHashMap;
import java.util.List;
-import java.util.Map;
import java.util.PriorityQueue;
import java.util.concurrent.ExecutorService;
import java.util.stream.Collectors;
public class ScanQueryRunnerFactory implements QueryRunnerFactory<ScanResultValue, ScanQuery>
{
- // This variable indicates when a running query should be expired,
- // and is effective only when 'timeout' of queryContext has a positive value.
- public static final String CTX_TIMEOUT_AT = "timeoutAt";
- public static final String CTX_COUNT = "count";
private final ScanQueryQueryToolChest toolChest;
private final ScanQueryEngine engine;
private final ScanQueryConfig scanQueryConfig;
@@ -98,7 +94,7 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory<ScanResultValu
// Note: this variable is effective only when queryContext has a timeout.
// See the comment of CTX_TIMEOUT_AT.
final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery());
- responseContext.put(CTX_TIMEOUT_AT, timeoutAt);
+ responseContext.put(ResponseContext.CTX_TIMEOUT_AT, timeoutAt);
if (query.getOrder().equals(ScanQuery.Order.NONE)) {
// Use normal strategy
@@ -311,7 +307,7 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory<ScanResultValu
Sequence<ScanResultValue> nWayMergeAndLimit(
List<List<QueryRunner<ScanResultValue>>> groupedRunners,
QueryPlus<ScanResultValue> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
// Starting from the innermost Sequences.map:
@@ -366,7 +362,7 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory<ScanResultValu
}
@Override
- public Sequence<ScanResultValue> run(QueryPlus<ScanResultValue> queryPlus, Map<String, Object> responseContext)
+ public Sequence<ScanResultValue> run(QueryPlus<ScanResultValue> queryPlus, ResponseContext responseContext)
{
Query<ScanResultValue> query = queryPlus.getQuery();
if (!(query instanceof ScanQuery)) {
@@ -374,9 +370,9 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory<ScanResultValu
}
// it happens in unit tests
- final Number timeoutAt = (Number) responseContext.get(CTX_TIMEOUT_AT);
+ final Number timeoutAt = (Number) responseContext.get(ResponseContext.CTX_TIMEOUT_AT);
if (timeoutAt == null || timeoutAt.longValue() == 0L) {
- responseContext.put(CTX_TIMEOUT_AT, JodaUtils.MAX_INSTANT);
+ responseContext.put(ResponseContext.CTX_TIMEOUT_AT, JodaUtils.MAX_INSTANT);
}
return engine.process((ScanQuery) query, segment, responseContext);
}
diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java
index 5011eed..5465f87 100644
--- a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java
@@ -43,6 +43,7 @@ import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.Result;
import org.apache.druid.query.ResultGranularTimestampComparator;
import org.apache.druid.query.aggregation.MetricManipulationFn;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.DimFilter;
@@ -329,7 +330,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResul
@Override
public Sequence<Result<SearchResultValue>> run(
QueryPlus<Result<SearchResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
SearchQuery searchQuery = (SearchQuery) queryPlus.getQuery();
@@ -363,7 +364,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResul
@Override
public Sequence<Result<SearchResultValue>> run(
QueryPlus<Result<SearchResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
Query<Result<SearchResultValue>> input = queryPlus.getQuery();
diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchQueryRunner.java b/processing/src/main/java/org/apache/druid/query/search/SearchQueryRunner.java
index 26d7843..ecc8c2a 100644
--- a/processing/src/main/java/org/apache/druid/query/search/SearchQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/search/SearchQueryRunner.java
@@ -34,6 +34,7 @@ import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.Result;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.ColumnSelectorStrategy;
import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
@@ -47,7 +48,6 @@ import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.data.IndexedInts;
import java.util.List;
-import java.util.Map;
/**
*/
@@ -207,7 +207,7 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
@Override
public Sequence<Result<SearchResultValue>> run(
final QueryPlus<Result<SearchResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
Query<Result<SearchResultValue>> input = queryPlus.getQuery();
diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java
index e0a5855..7ff4dfd 100644
--- a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java
@@ -40,6 +40,7 @@ import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.Result;
import org.apache.druid.query.ResultGranularTimestampComparator;
import org.apache.druid.query.aggregation.MetricManipulationFn;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.timeline.LogicalSegment;
@@ -322,7 +323,7 @@ public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResul
@Override
public Sequence<Result<SelectResultValue>> run(
QueryPlus<Result<SelectResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
SelectQuery selectQuery = (SelectQuery) queryPlus.getQuery();
diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/select/SelectQueryRunnerFactory.java
index db94788..9145f09 100644
--- a/processing/src/main/java/org/apache/druid/query/select/SelectQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/select/SelectQueryRunnerFactory.java
@@ -30,9 +30,9 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.Result;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.Segment;
-import java.util.Map;
import java.util.concurrent.ExecutorService;
/**
@@ -91,7 +91,7 @@ public class SelectQueryRunnerFactory
@Override
public Sequence<Result<SelectResultValue>> run(
QueryPlus<Result<SelectResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
Query<Result<SelectResultValue>> input = queryPlus.getQuery();
diff --git a/processing/src/main/java/org/apache/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/org/apache/druid/query/spec/SpecificSegmentQueryRunner.java
index a267364..94c5f8f 100644
--- a/processing/src/main/java/org/apache/druid/query/spec/SpecificSegmentQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/spec/SpecificSegmentQueryRunner.java
@@ -31,14 +31,13 @@ import org.apache.druid.java.util.common.guava.YieldingAccumulator;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
-import org.apache.druid.query.Result;
import org.apache.druid.query.SegmentDescriptor;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.SegmentMissingException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
-import java.util.Map;
/**
*/
@@ -57,7 +56,7 @@ public class SpecificSegmentQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> input, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> input, final ResponseContext responseContext)
{
final QueryPlus<T> queryPlus = input.withQuerySegmentSpec(specificSpec);
final Query<T> query = queryPlus.getQuery();
@@ -151,12 +150,13 @@ public class SpecificSegmentQueryRunner<T> implements QueryRunner<T>
);
}
- private void appendMissingSegment(Map<String, Object> responseContext)
+ private void appendMissingSegment(ResponseContext responseContext)
{
- List<SegmentDescriptor> missingSegments = (List<SegmentDescriptor>) responseContext.get(Result.MISSING_SEGMENTS_KEY);
+ List<SegmentDescriptor> missingSegments =
+ (List<SegmentDescriptor>) responseContext.get(ResponseContext.CTX_MISSING_SEGMENTS);
if (missingSegments == null) {
missingSegments = new ArrayList<>();
- responseContext.put(Result.MISSING_SEGMENTS_KEY, missingSegments);
+ responseContext.put(ResponseContext.CTX_MISSING_SEGMENTS, missingSegments);
}
missingSegments.add(specificSpec.getDescriptor());
}
diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java
index 9fc0e88..6499b88 100644
--- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java
@@ -39,11 +39,11 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.MetricManipulationFn;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.timeline.LogicalSegment;
import java.nio.ByteBuffer;
import java.util.List;
-import java.util.Map;
import java.util.stream.Collectors;
/**
@@ -101,7 +101,7 @@ public class TimeBoundaryQueryQueryToolChest
protected Sequence<Result<TimeBoundaryResultValue>> doRun(
QueryRunner<Result<TimeBoundaryResultValue>> baseRunner,
QueryPlus<Result<TimeBoundaryResultValue>> input,
- Map<String, Object> context
+ ResponseContext context
)
{
TimeBoundaryQuery query = (TimeBoundaryQuery) input.getQuery();
diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java
index c8a97fc..c39b6f9 100644
--- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java
@@ -35,6 +35,7 @@ import org.apache.druid.query.QueryRunnerHelper;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.Result;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.Segment;
@@ -46,7 +47,6 @@ import org.joda.time.DateTime;
import java.util.Iterator;
import java.util.List;
-import java.util.Map;
import java.util.concurrent.ExecutorService;
/**
@@ -131,7 +131,7 @@ public class TimeBoundaryQueryRunnerFactory
@Override
public Sequence<Result<TimeBoundaryResultValue>> run(
final QueryPlus<Result<TimeBoundaryResultValue>> queryPlus,
- final Map<String, Object> responseContext
+ final ResponseContext responseContext
)
{
Query<Result<TimeBoundaryResultValue>> input = queryPlus.getQuery();
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
index 023e017..416fb03 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
@@ -48,6 +48,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.MetricManipulationFn;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.groupby.RowBasedColumnSelectorFactory;
import org.joda.time.DateTime;
@@ -108,7 +109,7 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
public Sequence<Result<TimeseriesResultValue>> doRun(
QueryRunner<Result<TimeseriesResultValue>> baseRunner,
QueryPlus<Result<TimeseriesResultValue>> queryPlus,
- Map<String, Object> context
+ ResponseContext context
)
{
int limit = ((TimeseriesQuery) queryPlus.getQuery()).getLimit();
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java
index 2fa5762..fd6651c 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java
@@ -30,10 +30,10 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.Result;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.StorageAdapter;
-import java.util.Map;
import java.util.concurrent.ExecutorService;
/**
@@ -92,7 +92,7 @@ public class TimeseriesQueryRunnerFactory
@Override
public Sequence<Result<TimeseriesResultValue>> run(
QueryPlus<Result<TimeseriesResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
Query<Result<TimeseriesResultValue>> input = queryPlus.getQuery();
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java
index c0f12b1..06aa427 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java
@@ -45,6 +45,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.MetricManipulationFn;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.DimensionHandlerUtils;
@@ -435,7 +436,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
@Override
public Sequence<Result<TopNResultValue>> run(
QueryPlus<Result<TopNResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
TopNQuery topNQuery = (TopNQuery) queryPlus.getQuery();
@@ -475,7 +476,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
@Override
public Sequence<Result<TopNResultValue>> run(
- final QueryPlus<Result<TopNResultValue>> queryPlus, final Map<String, Object> responseContext
+ final QueryPlus<Result<TopNResultValue>> queryPlus, final ResponseContext responseContext
)
{
// thresholdRunner.run throws ISE if query is not TopNQuery
@@ -543,7 +544,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
@Override
public Sequence<Result<TopNResultValue>> run(
QueryPlus<Result<TopNResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
Query<Result<TopNResultValue>> input = queryPlus.getQuery();
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java
index 3443925..7044afc 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java
@@ -31,10 +31,10 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.Result;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.Segment;
import java.nio.ByteBuffer;
-import java.util.Map;
import java.util.concurrent.ExecutorService;
/**
@@ -66,7 +66,7 @@ public class TopNQueryRunnerFactory implements QueryRunnerFactory<Result<TopNRes
@Override
public Sequence<Result<TopNResultValue>> run(
QueryPlus<Result<TopNResultValue>> input,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
if (!(input.getQuery() instanceof TopNQuery)) {
diff --git a/processing/src/test/java/org/apache/druid/query/AsyncQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/AsyncQueryRunnerTest.java
index e2fb8bf0..29c7e33 100644
--- a/processing/src/test/java/org/apache/druid/query/AsyncQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/AsyncQueryRunnerTest.java
@@ -24,12 +24,12 @@ import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Test;
import java.util.Collections;
-import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@@ -60,7 +60,7 @@ public class AsyncQueryRunnerTest
QueryRunner baseRunner = new QueryRunner()
{
@Override
- public Sequence run(QueryPlus queryPlus, Map responseContext)
+ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext)
{
try {
latch.await();
@@ -78,7 +78,7 @@ public class AsyncQueryRunnerTest
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
- Sequence lazy = asyncRunner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP);
+ Sequence lazy = asyncRunner.run(QueryPlus.wrap(query));
latch.countDown();
Assert.assertEquals(Collections.singletonList(1), lazy.toList());
}
@@ -89,7 +89,7 @@ public class AsyncQueryRunnerTest
QueryRunner baseRunner = new QueryRunner()
{
@Override
- public Sequence run(QueryPlus queryPlus, Map responseContext)
+ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext)
{
try {
Thread.sleep(Long.MAX_VALUE);
@@ -107,10 +107,8 @@ public class AsyncQueryRunnerTest
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
- Sequence lazy = asyncRunner.run(
- QueryPlus.wrap(query.withOverriddenContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 1))),
- Collections.EMPTY_MAP
- );
+ Sequence lazy =
+ asyncRunner.run(QueryPlus.wrap(query.withOverriddenContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 1))));
try {
lazy.toList();
@@ -133,7 +131,7 @@ public class AsyncQueryRunnerTest
AsyncQueryRunner asyncRunner = new AsyncQueryRunner<>(baseRunner, executor, mock);
- asyncRunner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP);
+ asyncRunner.run(QueryPlus.wrap(query));
EasyMock.verify(mock);
}
}
diff --git a/processing/src/test/java/org/apache/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/ChainedExecutionQueryRunnerTest.java
index b370e4c..275fb63 100644
--- a/processing/src/test/java/org/apache/druid/query/ChainedExecutionQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/ChainedExecutionQueryRunnerTest.java
@@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.easymock.Capture;
import org.easymock.EasyMock;
@@ -36,8 +37,6 @@ import org.junit.Before;
import org.junit.Test;
import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ArrayBlockingQueue;
@@ -119,13 +118,12 @@ public class ChainedExecutionQueryRunnerTest
runners
)
);
- Map<String, Object> context = ImmutableMap.of();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource("test")
.intervals("2014/2015")
.aggregators(Collections.singletonList(new CountAggregatorFactory("count")))
.build();
- final Sequence seq = chainedRunner.run(QueryPlus.wrap(query), context);
+ final Sequence seq = chainedRunner.run(QueryPlus.wrap(query));
Future resultFuture = Executors.newFixedThreadPool(1).submit(
new Runnable()
@@ -244,14 +242,13 @@ public class ChainedExecutionQueryRunnerTest
runners
)
);
- HashMap<String, Object> context = new HashMap<String, Object>();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource("test")
.intervals("2014/2015")
.aggregators(Collections.singletonList(new CountAggregatorFactory("count")))
.context(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 100, "queryId", "test"))
.build();
- final Sequence seq = chainedRunner.run(QueryPlus.wrap(query), context);
+ final Sequence seq = chainedRunner.run(QueryPlus.wrap(query));
Future resultFuture = Executors.newFixedThreadPool(1).submit(
new Runnable()
@@ -327,7 +324,7 @@ public class ChainedExecutionQueryRunnerTest
}
@Override
- public Sequence<Integer> run(QueryPlus<Integer> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Integer> run(QueryPlus<Integer> queryPlus, ResponseContext responseContext)
{
// do a lot of work
synchronized (this) {
diff --git a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java
index f784a86..690461a 100644
--- a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java
+++ b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java
@@ -70,7 +70,6 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
@@ -271,7 +270,7 @@ public class DoubleStorageTest
)
.merge(true)
.build();
- List<SegmentAnalysis> results = runner.run(QueryPlus.wrap(segmentMetadataQuery), new HashMap<>()).toList();
+ List<SegmentAnalysis> results = runner.run(QueryPlus.wrap(segmentMetadataQuery)).toList();
Assert.assertEquals(Collections.singletonList(expectedSegmentAnalysis), results);
@@ -292,8 +291,7 @@ public class DoubleStorageTest
.virtualColumns()
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
ScanResultValue expectedScanResult = new ScanResultValue(
SEGMENT_ID.toString(),
diff --git a/processing/src/test/java/org/apache/druid/query/IntervalChunkingQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/IntervalChunkingQueryRunnerTest.java
index 0334e06..87e90c9 100644
--- a/processing/src/test/java/org/apache/druid/query/IntervalChunkingQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/IntervalChunkingQueryRunnerTest.java
@@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.query.Druids.TimeseriesQueryBuilder;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.easymock.EasyMock;
import org.junit.Before;
import org.junit.Test;
@@ -62,12 +63,13 @@ public class IntervalChunkingQueryRunnerTest
public void testDefaultNoChunking()
{
QueryPlus queryPlus = QueryPlus.wrap(queryBuilder.intervals("2014/2016").build());
+ final ResponseContext context = ResponseContext.createEmpty();
- EasyMock.expect(baseRunner.run(queryPlus, Collections.EMPTY_MAP)).andReturn(Sequences.empty());
+ EasyMock.expect(baseRunner.run(queryPlus, context)).andReturn(Sequences.empty());
EasyMock.replay(baseRunner);
QueryRunner runner = decorator.decorate(baseRunner, toolChest);
- runner.run(queryPlus, Collections.EMPTY_MAP);
+ runner.run(queryPlus, context);
EasyMock.verify(baseRunner);
}
@@ -84,7 +86,7 @@ public class IntervalChunkingQueryRunnerTest
EasyMock.replay(toolChest);
QueryRunner runner = decorator.decorate(baseRunner, toolChest);
- runner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP);
+ runner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
EasyMock.verify(executors);
}
@@ -101,7 +103,7 @@ public class IntervalChunkingQueryRunnerTest
EasyMock.replay(toolChest);
QueryRunner runner = decorator.decorate(baseRunner, toolChest);
- runner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP);
+ runner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
EasyMock.verify(executors);
}
diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
index a9c8467..2f75700 100644
--- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
+++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
@@ -84,7 +84,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -1035,8 +1034,7 @@ public class MultiValuedDimensionTest
new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
null
);
- Map<String, Object> context = new HashMap<>();
- Sequence<Result<TopNResultValue>> result = runner.run(QueryPlus.wrap(query), context);
+ Sequence<Result<TopNResultValue>> result = runner.run(QueryPlus.wrap(query));
List<Result<TopNResultValue>> expectedResults = Collections.singletonList(
new Result<TopNResultValue>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
@@ -1089,8 +1087,7 @@ public class MultiValuedDimensionTest
new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
null
);
- Map<String, Object> context = new HashMap<>();
- Sequence<Result<TopNResultValue>> result = runner.run(QueryPlus.wrap(query), context);
+ Sequence<Result<TopNResultValue>> result = runner.run(QueryPlus.wrap(query));
List<Map<String, Object>> expected =
ImmutableList.<Map<String, Object>>builder()
.add(ImmutableMap.of("texpr", "t3foo", "count", 2L))
@@ -1150,8 +1147,7 @@ public class MultiValuedDimensionTest
new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
null
);
- Map<String, Object> context = new HashMap<>();
- Sequence<Result<TopNResultValue>> result = runner.run(QueryPlus.wrap(query), context);
+ Sequence<Result<TopNResultValue>> result = runner.run(QueryPlus.wrap(query));
List<Map<String, Object>> expected =
ImmutableList.<Map<String, Object>>builder()
diff --git a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java
index 944139e..7f53e30 100644
--- a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java
+++ b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java
@@ -47,6 +47,7 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFact
import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
import org.apache.druid.query.aggregation.post.ConstantPostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
@@ -430,7 +431,7 @@ public class QueryRunnerTestHelper
return new QueryRunner<T>()
{
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
return runner.run(queryPlus, responseContext);
}
@@ -454,7 +455,7 @@ public class QueryRunnerTestHelper
new QueryRunner<T>()
{
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
Query<T> query = queryPlus.getQuery();
List<TimelineObjectHolder> segments = new ArrayList<>();
@@ -497,7 +498,7 @@ public class QueryRunnerTestHelper
return new QueryRunner<T>()
{
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
return delegate.run(queryPlus, responseContext);
}
diff --git a/processing/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
index 378c024..0b143e3 100644
--- a/processing/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
@@ -25,6 +25,8 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.context.ConcurrentResponseContext;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.timeseries.TimeseriesResultValue;
@@ -38,9 +40,6 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
public class RetryQueryRunnerTest
{
@@ -90,15 +89,15 @@ public class RetryQueryRunnerTest
@Test
public void testRunWithMissingSegments()
{
- ConcurrentMap<String, Object> context = new ConcurrentHashMap<>();
- context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
+ ResponseContext context = ConcurrentResponseContext.createEmpty();
+ context.put(ResponseContext.CTX_MISSING_SEGMENTS, new ArrayList<>());
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
new QueryRunner<Result<TimeseriesResultValue>>()
{
@Override
- public Sequence<Result<TimeseriesResultValue>> run(QueryPlus queryPlus, Map context)
+ public Sequence<Result<TimeseriesResultValue>> run(QueryPlus queryPlus, ResponseContext context)
{
- ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add(
+ ((List) context.get(ResponseContext.CTX_MISSING_SEGMENTS)).add(
new SegmentDescriptor(Intervals.utc(178888, 1999999), "test", 1)
);
return Sequences.empty();
@@ -125,7 +124,7 @@ public class RetryQueryRunnerTest
Assert.assertTrue(
"Should have one entry in the list of missing segments",
- ((List) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 1
+ ((List) context.get(ResponseContext.CTX_MISSING_SEGMENTS)).size() == 1
);
Assert.assertTrue("Should return an empty sequence as a result", ((List) actualResults).size() == 0);
}
@@ -134,20 +133,20 @@ public class RetryQueryRunnerTest
@Test
public void testRetry()
{
- ConcurrentMap<String, Object> context = new ConcurrentHashMap<>();
+ ResponseContext context = ConcurrentResponseContext.createEmpty();
context.put("count", 0);
- context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
+ context.put(ResponseContext.CTX_MISSING_SEGMENTS, new ArrayList<>());
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
new QueryRunner<Result<TimeseriesResultValue>>()
{
@Override
public Sequence<Result<TimeseriesResultValue>> run(
QueryPlus<Result<TimeseriesResultValue>> queryPlus,
- Map<String, Object> context
+ ResponseContext context
)
{
if ((int) context.get("count") == 0) {
- ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add(
+ ((List) context.get(ResponseContext.CTX_MISSING_SEGMENTS)).add(
new SegmentDescriptor(Intervals.utc(178888, 1999999), "test", 1)
);
context.put("count", 1);
@@ -175,27 +174,27 @@ public class RetryQueryRunnerTest
Assert.assertTrue("Should return a list with one element", ((List) actualResults).size() == 1);
Assert.assertTrue(
"Should have nothing in missingSegment list",
- ((List) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 0
+ ((List) context.get(ResponseContext.CTX_MISSING_SEGMENTS)).size() == 0
);
}
@Test
public void testRetryMultiple()
{
- ConcurrentMap<String, Object> context = new ConcurrentHashMap<>();
+ ResponseContext context = ConcurrentResponseContext.createEmpty();
context.put("count", 0);
- context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
+ context.put(ResponseContext.CTX_MISSING_SEGMENTS, new ArrayList<>());
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
new QueryRunner<Result<TimeseriesResultValue>>()
{
@Override
public Sequence<Result<TimeseriesResultValue>> run(
QueryPlus<Result<TimeseriesResultValue>> queryPlus,
- Map<String, Object> context
+ ResponseContext context
)
{
if ((int) context.get("count") < 3) {
- ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add(
+ ((List) context.get(ResponseContext.CTX_MISSING_SEGMENTS)).add(
new SegmentDescriptor(Intervals.utc(178888, 1999999), "test", 1)
);
context.put("count", (int) context.get("count") + 1);
@@ -223,25 +222,25 @@ public class RetryQueryRunnerTest
Assert.assertTrue("Should return a list with one element", ((List) actualResults).size() == 1);
Assert.assertTrue(
"Should have nothing in missingSegment list",
- ((List) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 0
+ ((List) context.get(ResponseContext.CTX_MISSING_SEGMENTS)).size() == 0
);
}
@Test(expected = SegmentMissingException.class)
public void testException()
{
- ConcurrentMap<String, Object> context = new ConcurrentHashMap<>();
- context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
+ ResponseContext context = ConcurrentResponseContext.createEmpty();
+ context.put(ResponseContext.CTX_MISSING_SEGMENTS, new ArrayList<>());
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
new QueryRunner<Result<TimeseriesResultValue>>()
{
@Override
public Sequence<Result<TimeseriesResultValue>> run(
QueryPlus<Result<TimeseriesResultValue>> queryPlus,
- Map<String, Object> context
+ ResponseContext context
)
{
- ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add(
+ ((List) context.get(ResponseContext.CTX_MISSING_SEGMENTS)).add(
new SegmentDescriptor(Intervals.utc(178888, 1999999), "test", 1)
);
return Sequences.empty();
@@ -255,32 +254,32 @@ public class RetryQueryRunnerTest
Assert.assertTrue(
"Should have one entry in the list of missing segments",
- ((List) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 1
+ ((List) context.get(ResponseContext.CTX_MISSING_SEGMENTS)).size() == 1
);
}
@Test
public void testNoDuplicateRetry()
{
- ConcurrentMap<String, Object> context = new ConcurrentHashMap<>();
+ ResponseContext context = ConcurrentResponseContext.createEmpty();
context.put("count", 0);
- context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
+ context.put(ResponseContext.CTX_MISSING_SEGMENTS, new ArrayList<>());
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
new QueryRunner<Result<TimeseriesResultValue>>()
{
@Override
public Sequence<Result<TimeseriesResultValue>> run(
QueryPlus<Result<TimeseriesResultValue>> queryPlus,
- Map<String, Object> context
+ ResponseContext context
)
{
final Query<Result<TimeseriesResultValue>> query = queryPlus.getQuery();
if ((int) context.get("count") == 0) {
// assume 2 missing segments at first run
- ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add(
+ ((List) context.get(ResponseContext.CTX_MISSING_SEGMENTS)).add(
new SegmentDescriptor(Intervals.utc(178888, 1999999), "test", 1)
);
- ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add(
+ ((List) context.get(ResponseContext.CTX_MISSING_SEGMENTS)).add(
new SegmentDescriptor(Intervals.utc(178888, 1999999), "test", 2)
);
context.put("count", 1);
@@ -298,7 +297,7 @@ public class RetryQueryRunnerTest
// this is first retry
Assert.assertTrue("Should retry with 2 missing segments", ((MultipleSpecificSegmentSpec) ((BaseQuery) query).getQuerySegmentSpec()).getDescriptors().size() == 2);
// assume only left 1 missing at first retry
- ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add(
+ ((List) context.get(ResponseContext.CTX_MISSING_SEGMENTS)).add(
new SegmentDescriptor(Intervals.utc(178888, 1999999), "test", 2)
);
context.put("count", 2);
@@ -339,7 +338,7 @@ public class RetryQueryRunnerTest
Assert.assertTrue("Should return a list with 3 elements", ((List) actualResults).size() == 3);
Assert.assertTrue(
"Should have nothing in missingSegment list",
- ((List) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 0
+ ((List) context.get(ResponseContext.CTX_MISSING_SEGMENTS)).size() == 0
);
}
}
diff --git a/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java b/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java
index c15a2e6..889e80f 100644
--- a/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java
+++ b/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java
@@ -115,7 +115,7 @@ public class SchemaEvolutionTest
)
),
(QueryToolChest<T, Query<T>>) factory.getToolchest()
- ).run(QueryPlus.wrap(query), new HashMap<>());
+ ).run(QueryPlus.wrap(query));
return results.toList();
}
diff --git a/processing/src/test/java/org/apache/druid/query/TimewarpOperatorTest.java b/processing/src/test/java/org/apache/druid/query/TimewarpOperatorTest.java
index dedc611..10d788e 100644
--- a/processing/src/test/java/org/apache/druid/query/TimewarpOperatorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/TimewarpOperatorTest.java
@@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.granularity.PeriodGranularity;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.timeboundary.TimeBoundaryResultValue;
import org.apache.druid.query.timeseries.TimeseriesResultValue;
import org.joda.time.DateTime;
@@ -37,14 +38,10 @@ import org.junit.Assert;
import org.junit.Test;
import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
public class TimewarpOperatorTest
{
- public static final ImmutableMap<String, Object> CONTEXT = ImmutableMap.of();
-
TimewarpOperator<Result<TimeseriesResultValue>> testOperator = new TimewarpOperator<>(
new Interval(DateTimes.of("2014-01-01"), DateTimes.of("2014-01-15")),
new Period("P1W"),
@@ -88,7 +85,7 @@ public class TimewarpOperatorTest
@Override
public Sequence<Result<TimeseriesResultValue>> run(
QueryPlus<Result<TimeseriesResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
return Sequences.simple(
@@ -134,7 +131,7 @@ public class TimewarpOperatorTest
new TimeseriesResultValue(ImmutableMap.of("metric", 5))
)
),
- queryRunner.run(QueryPlus.wrap(query), CONTEXT).toList()
+ queryRunner.run(QueryPlus.wrap(query)).toList()
);
@@ -150,7 +147,7 @@ public class TimewarpOperatorTest
@Override
public Sequence<Result<TimeBoundaryResultValue>> run(
QueryPlus<Result<TimeBoundaryResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
return Sequences.simple(
@@ -183,7 +180,7 @@ public class TimewarpOperatorTest
new TimeBoundaryResultValue(ImmutableMap.<String, Object>of("maxTime", DateTimes.of("2014-08-02")))
)
),
- timeBoundaryRunner.run(QueryPlus.wrap(timeBoundaryQuery), CONTEXT).toList()
+ timeBoundaryRunner.run(QueryPlus.wrap(timeBoundaryQuery)).toList()
);
}
@@ -197,7 +194,7 @@ public class TimewarpOperatorTest
@Override
public Sequence<Result<TimeseriesResultValue>> run(
QueryPlus<Result<TimeseriesResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
return Sequences.simple(
@@ -244,7 +241,7 @@ public class TimewarpOperatorTest
new TimeseriesResultValue(ImmutableMap.of("metric", 5))
)
),
- queryRunner.run(QueryPlus.wrap(query), CONTEXT).toList()
+ queryRunner.run(QueryPlus.wrap(query)).toList()
);
}
@@ -257,7 +254,7 @@ public class TimewarpOperatorTest
@Override
public Sequence<Result<TimeseriesResultValue>> run(
QueryPlus<Result<TimeseriesResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
return Sequences.simple(
@@ -304,7 +301,7 @@ public class TimewarpOperatorTest
new TimeseriesResultValue(ImmutableMap.of("metric", 5))
)
),
- queryRunner.run(QueryPlus.wrap(query), CONTEXT).toList()
+ queryRunner.run(QueryPlus.wrap(query)).toList()
);
}
@@ -317,7 +314,7 @@ public class TimewarpOperatorTest
@Override
public Sequence<Result<TimeseriesResultValue>> run(
QueryPlus<Result<TimeseriesResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
final Query<Result<TimeseriesResultValue>> query = queryPlus.getQuery();
@@ -356,7 +353,7 @@ public class TimewarpOperatorTest
new TimeseriesResultValue(ImmutableMap.of("metric", 3))
)
),
- queryRunner.run(QueryPlus.wrap(query), new HashMap<>()).toList()
+ queryRunner.run(QueryPlus.wrap(query)).toList()
);
}
}
diff --git a/processing/src/test/java/org/apache/druid/query/UnionQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/UnionQueryRunnerTest.java
index 3cdaed0..78b7712 100644
--- a/processing/src/test/java/org/apache/druid/query/UnionQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/UnionQueryRunnerTest.java
@@ -23,12 +23,11 @@ import com.google.common.collect.Iterables;
import junit.framework.Assert;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.context.ResponseContext;
import org.junit.Test;
import java.util.Arrays;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map;
public class UnionQueryRunnerTest
{
@@ -38,7 +37,7 @@ public class UnionQueryRunnerTest
QueryRunner baseRunner = new QueryRunner()
{
@Override
- public Sequence run(QueryPlus queryPlus, Map responseContext)
+ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext)
{
// verify that table datasource is passed to baseQueryRunner
Assert.assertTrue(queryPlus.getQuery().getDataSource() instanceof TableDataSource);
@@ -68,7 +67,7 @@ public class UnionQueryRunnerTest
.intervals("2014-01-01T00:00:00Z/2015-01-01T00:00:00Z")
.aggregators(QueryRunnerTestHelper.commonDoubleAggregators)
.build();
- Map<String, Object> responseContext = new HashMap<>();
+ ResponseContext responseContext = ResponseContext.createEmpty();
Sequence<?> result = runner.run(QueryPlus.wrap(q), responseContext);
List res = result.toList();
Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6), res);
diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java
index 83b25e0..8d134b3 100644
--- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java
+++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java
@@ -55,6 +55,7 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.QueryToolChest;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
@@ -93,10 +94,8 @@ import java.lang.reflect.Array;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
-import java.util.Map;
/**
* This class provides general utility to test any druid aggregation implementation given raw data,
@@ -633,7 +632,7 @@ public class AggregationTestHelper implements Closeable
toolChest
);
- return baseRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ return baseRunner.run(QueryPlus.wrap(query));
}
public QueryRunner<Row> makeStringSerdeQueryRunner(
@@ -645,10 +644,10 @@ public class AggregationTestHelper implements Closeable
return new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> map)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext map)
{
try {
- Sequence<Row> resultSeq = baseRunner.run(queryPlus, new HashMap<>());
+ Sequence<Row> resultSeq = baseRunner.run(queryPlus);
final Yielder yielder = resultSeq.toYielder(
null,
new YieldingAccumulator()
diff --git a/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java
index 3f20c25..dd5bedf 100644
--- a/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java
@@ -37,6 +37,8 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.context.ConcurrentResponseContext;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.timeline.LogicalSegment;
@@ -51,8 +53,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
public class DataSourceMetadataQueryTest
{
@@ -138,8 +138,8 @@ public class DataSourceMetadataQueryTest
DataSourceMetadataQuery dataSourceMetadataQuery = Druids.newDataSourceMetadataQueryBuilder()
.dataSource("testing")
.build();
- ConcurrentMap<String, Object> context = new ConcurrentHashMap<>();
- context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
+ ResponseContext context = ConcurrentResponseContext.createEmpty();
+ context.put(ResponseContext.CTX_MISSING_SEGMENTS, new ArrayList<>());
Iterable<Result<DataSourceMetadataResultValue>> results =
runner.run(QueryPlus.wrap(dataSourceMetadataQuery), context).toList();
DataSourceMetadataResultValue val = results.iterator().next().getValue();
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java
index 5d8be6d..c776cec 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java
@@ -57,6 +57,7 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
@@ -450,7 +451,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
return Sequences
.simple(
@@ -485,7 +486,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
.setGranularity(Granularities.ALL)
.build();
- Sequence<Row> queryResult = theRunner3.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<Row> queryResult = theRunner3.run(QueryPlus.wrap(query));
List<Row> results = queryResult.toList();
Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow(
@@ -536,7 +537,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
return Sequences
.simple(
@@ -579,7 +580,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
)
.build();
- Sequence<Row> queryResult = theRunner3.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<Row> queryResult = theRunner3.run(QueryPlus.wrap(query));
List<Row> results = queryResult.toList();
Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow(
@@ -689,7 +690,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
return new QueryRunner<T>()
{
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
return delegate.run(queryPlus, responseContext);
}
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java
index 581e80b..0f12908 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java
@@ -59,6 +59,7 @@ import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.ExtractionDimensionSpec;
import org.apache.druid.query.expression.TestExprMacroTable;
@@ -480,7 +481,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
return Sequences
.simple(
@@ -549,7 +550,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
.setGranularity(Granularities.ALL)
.build();
- Sequence<Row> queryResult = finalRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<Row> queryResult = finalRunner.run(QueryPlus.wrap(query));
List<Row> results = queryResult.toList();
Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow(
@@ -613,7 +614,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
return Sequences
.simple(
@@ -670,7 +671,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
.setGranularity(Granularities.ALL)
.build();
- Sequence<Row> queryResult = finalRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<Row> queryResult = finalRunner.run(QueryPlus.wrap(query));
List<Row> results = queryResult.toList();
Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow(
@@ -790,7 +791,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
return new QueryRunner<T>()
{
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
return delegate.run(queryPlus, responseContext);
}
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java
index 62e7a40..39167c5 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java
@@ -54,6 +54,7 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.groupby.having.GreaterThanHavingSpec;
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
@@ -342,7 +343,7 @@ public class GroupByMultiSegmentTest
.setGranularity(Granularities.ALL)
.build();
- Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query));
List<Row> results = queryResult.toList();
Row expectedRow = GroupByQueryRunnerTestHelper.createExpectedRow(
@@ -427,7 +428,7 @@ public class GroupByMultiSegmentTest
{
return new QueryRunner<T>() {
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
return delegate.run(queryPlus, responseContext);
}
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java
index 15279d7..a1e871f 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java
@@ -35,6 +35,7 @@ import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.spec.LegacySegmentSpec;
import org.apache.druid.segment.CloserRule;
@@ -50,9 +51,7 @@ import org.junit.Test;
import java.io.IOException;
import java.util.Arrays;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map;
/**
*/
@@ -97,13 +96,13 @@ public class GroupByQueryRunnerFactoryTest
new QueryRunner()
{
@Override
- public Sequence run(QueryPlus queryPlus, Map responseContext)
+ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext)
{
return factory.getToolchest().mergeResults(
new QueryRunner()
{
@Override
- public Sequence run(QueryPlus queryPlus, Map responseContext)
+ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext)
{
final Query query = queryPlus.getQuery();
try {
@@ -127,7 +126,7 @@ public class GroupByQueryRunnerFactoryTest
}
);
- Sequence<Row> result = mergedRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<Row> result = mergedRunner.run(QueryPlus.wrap(query));
List<Row> expectedResults = Arrays.asList(
GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t1", "count", 2L),
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
index 3eb69c0..c0fd03e 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
@@ -81,6 +81,7 @@ import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
import org.apache.druid.query.aggregation.post.ConstantPostAggregator;
import org.apache.druid.query.aggregation.post.ExpressionPostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.ExtractionDimensionSpec;
import org.apache.druid.query.dimension.ListFilteredDimensionSpec;
@@ -2922,7 +2923,7 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec(
@@ -2953,7 +2954,7 @@ public class GroupByQueryRunnerTest
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L)
);
- Map<String, Object> context = new HashMap<>();
+ ResponseContext context = ResponseContext.createEmpty();
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged");
List<Row> allGranExpectedResults = Arrays.asList(
@@ -3009,10 +3010,9 @@ public class GroupByQueryRunnerTest
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
- Map<String, Object> context = new HashMap<>();
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, limit),
- mergeRunner.run(QueryPlus.wrap(fullQuery), context),
+ mergeRunner.run(QueryPlus.wrap(fullQuery)),
StringUtils.format("limit: %d", limit)
);
}
@@ -3052,10 +3052,9 @@ public class GroupByQueryRunnerTest
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
- Map<String, Object> context = new HashMap<>();
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, limit),
- mergeRunner.run(QueryPlus.wrap(fullQuery), context),
+ mergeRunner.run(QueryPlus.wrap(fullQuery)),
StringUtils.format("limit: %d", limit)
);
}
@@ -3106,10 +3105,9 @@ public class GroupByQueryRunnerTest
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
- Map<String, Object> context = new HashMap<>();
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, limit),
- mergeRunner.run(QueryPlus.wrap(fullQuery), context),
+ mergeRunner.run(QueryPlus.wrap(fullQuery)),
StringUtils.format("limit: %d", limit)
);
}
@@ -3204,7 +3202,7 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec(
@@ -3223,8 +3221,7 @@ public class GroupByQueryRunnerTest
}
);
- Map<String, Object> context = new HashMap<>();
- TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged");
+ TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery)), "merged");
}
@Test
@@ -3256,7 +3253,7 @@ public class GroupByQueryRunnerTest
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L)
);
- Map<String, Object> context = new HashMap<>();
+ ResponseContext context = ResponseContext.createEmpty();
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit");
@@ -3348,7 +3345,7 @@ public class GroupByQueryRunnerTest
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L)
);
- Map<String, Object> context = new HashMap<>();
+ ResponseContext context = ResponseContext.createEmpty();
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit");
TestHelper.assertExpectedObjects(
@@ -3385,7 +3382,7 @@ public class GroupByQueryRunnerTest
new Object[]{"2011-04-01", "technology", 2L, 178.24917602539062D}
);
- Map<String, Object> context = new HashMap<>();
+ ResponseContext context = ResponseContext.createEmpty();
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit");
TestHelper.assertExpectedObjects(
@@ -3429,7 +3426,7 @@ public class GroupByQueryRunnerTest
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L)
);
- Map<String, Object> context = new HashMap<>();
+ ResponseContext context = ResponseContext.createEmpty();
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit");
TestHelper.assertExpectedObjects(
@@ -3950,7 +3947,7 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec(
@@ -3969,8 +3966,7 @@ public class GroupByQueryRunnerTest
}
);
- Map<String, Object> context = new HashMap<>();
- TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged");
+ TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery)), "merged");
}
@Test
@@ -4251,7 +4247,7 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec(
@@ -4270,8 +4266,7 @@ public class GroupByQueryRunnerTest
}
);
- Map<String, Object> context = new HashMap<>();
- TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged");
+ TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery)), "merged");
}
@Test
@@ -4352,7 +4347,7 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec(
@@ -4371,7 +4366,7 @@ public class GroupByQueryRunnerTest
}
);
- Map<String, Object> context = new HashMap<>();
+ ResponseContext context = ResponseContext.createEmpty();
// add an extra layer of merging, simulate broker forwarding query to historical
TestHelper.assertExpectedObjects(
expectedResults,
@@ -4523,8 +4518,7 @@ public class GroupByQueryRunnerTest
);
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
- Map<String, Object> context = new HashMap<>();
- TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit");
+ TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit");
}
@Test
@@ -4575,9 +4569,8 @@ public class GroupByQueryRunnerTest
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "travel", "rows", 2L)
);
- Map<String, Object> context = new HashMap<>();
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
- TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit");
+ TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit");
}
// A subquery identical to the query should yield identical results
@@ -7960,7 +7953,7 @@ public class GroupByQueryRunnerTest
TestHelper.assertExpectedObjects(
bySegmentResults,
- theRunner.run(QueryPlus.wrap(fullQuery), new HashMap<>()),
+ theRunner.run(QueryPlus.wrap(fullQuery)),
"bySegment"
);
exec.shutdownNow();
@@ -8029,7 +8022,7 @@ public class GroupByQueryRunnerTest
TestHelper.assertExpectedObjects(
bySegmentResults,
- theRunner.run(QueryPlus.wrap(fullQuery), new HashMap<>()),
+ theRunner.run(QueryPlus.wrap(fullQuery)),
"bySegment"
);
exec.shutdownNow();
@@ -8094,7 +8087,7 @@ public class GroupByQueryRunnerTest
TestHelper.assertExpectedObjects(
bySegmentResults,
- theRunner.run(QueryPlus.wrap(fullQuery), new HashMap<>()),
+ theRunner.run(QueryPlus.wrap(fullQuery)),
"bySegment-dim-extraction"
);
exec.shutdownNow();
@@ -8601,7 +8594,7 @@ public class GroupByQueryRunnerTest
TestHelper.assertExpectedObjects(
bySegmentResults,
- theRunner.run(QueryPlus.wrap(fullQuery), new HashMap<>()),
+ theRunner.run(QueryPlus.wrap(fullQuery)),
"bySegment-filter"
);
exec.shutdownNow();
@@ -10017,7 +10010,7 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus<Row> queryPlus1 = queryPlus.withQuerySegmentSpec(
@@ -10041,7 +10034,6 @@ public class GroupByQueryRunnerTest
}
}
);
- Map<String, Object> context = new HashMap<>();
List<Row> allGranExpectedResults = Arrays.asList(
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L),
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 2L, "idx", 177L),
@@ -10052,7 +10044,7 @@ public class GroupByQueryRunnerTest
TestHelper.assertExpectedObjects(
allGranExpectedResults,
- mergedRunner.run(QueryPlus.wrap(allGranQuery), context),
+ mergedRunner.run(QueryPlus.wrap(allGranQuery)),
"merged"
);
}
@@ -10083,7 +10075,7 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus<Row> queryPlus1 = queryPlus.withQuerySegmentSpec(
@@ -10107,7 +10099,6 @@ public class GroupByQueryRunnerTest
}
}
);
- Map<String, Object> context = new HashMap<>();
List<Row> allGranExpectedResults = Arrays.asList(
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L),
@@ -10117,7 +10108,7 @@ public class GroupByQueryRunnerTest
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L)
);
- Iterable<Row> results = mergedRunner.run(QueryPlus.wrap(allGranQuery), context).toList();
+ Iterable<Row> results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList();
TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged");
}
@@ -10152,7 +10143,7 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus<Row> queryPlus1 = queryPlus.withQuerySegmentSpec(
@@ -10176,7 +10167,6 @@ public class GroupByQueryRunnerTest
}
}
);
- Map<String, Object> context = new HashMap<>();
List<Row> allGranExpectedResults = Arrays.asList(
GroupByQueryRunnerTestHelper.createExpectedRow(
@@ -10236,7 +10226,7 @@ public class GroupByQueryRunnerTest
)
);
- Iterable<Row> results = mergedRunner.run(QueryPlus.wrap(allGranQuery), context).toList();
+ Iterable<Row> results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList();
TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged");
}
@@ -10284,7 +10274,7 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>()
{
@Override
- public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+ public Sequence<Row> run(QueryPlus<Row> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus<Row> queryPlus1 = queryPlus.withQuerySegmentSpec(
@@ -10308,7 +10298,6 @@ public class GroupByQueryRunnerTest
}
}
);
- Map<String, Object> context = new HashMap<>();
List<Row> allGranExpectedResults = Arrays.asList(
GroupByQueryRunnerTestHelper.createExpectedRow(
@@ -10368,7 +10357,7 @@ public class GroupByQueryRunnerTest
)
);
- Iterable<Row> results = mergedRunner.run(QueryPlus.wrap(allGranQuery), context).toList();
+ Iterable<Row> results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList();
TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged");
}
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTestHelper.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTestHelper.java
index f59b631..277c28d 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTestHelper.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTestHelper.java
@@ -54,7 +54,7 @@ public class GroupByQueryRunnerTestHelper
toolChest
);
- Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+ Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query));
return queryResult.toList();
}
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java
index 464fb6b..fba0343 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java
@@ -40,6 +40,7 @@ import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.timeseries.TimeseriesQueryRunnerTest;
import org.apache.druid.query.timeseries.TimeseriesResultValue;
@@ -53,7 +54,6 @@ import org.junit.runners.Parameterized;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
-import java.util.Map;
/**
*
@@ -87,7 +87,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
final QueryRunner modifiedRunner = new QueryRunner()
{
@Override
- public Sequence run(QueryPlus queryPlus, Map responseContext)
+ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext)
{
TimeseriesQuery tsQuery = (TimeseriesQuery) queryPlus.getQuery();
QueryRunner<Row> newRunner = factory.mergeRunners(
@@ -171,7 +171,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
DateTime expectedEarliest = DateTimes.of("1970-01-01");
DateTime expectedLast = DateTimes.of("2011-04-15");
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
Result<TimeseriesResultValue> result = results.iterator().next();
Assert.assertEquals(expectedEarliest, result.getTimestamp());
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java
index e730abf..735f2d2 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java
@@ -61,6 +61,7 @@ import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.MetricManipulatorFns;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.ExtractionDimensionSpec;
import org.apache.druid.query.extraction.RegexDimExtractionFn;
@@ -417,7 +418,7 @@ public class NestedQueryPushDownTest
.setGranularity(Granularities.ALL)
.build();
- Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, new HashMap<>());
+ Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty());
List<Row> results = queryResult.toList();
Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow(
@@ -470,7 +471,7 @@ public class NestedQueryPushDownTest
.setGranularity(Granularities.ALL)
.build();
- Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, new HashMap<>());
+ Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty());
List<Row> results = queryResult.toList();
Row expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow(
@@ -532,7 +533,7 @@ public class NestedQueryPushDownTest
.setQuerySegmentSpec(intervalSpec)
.build();
- Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, new HashMap<>());
+ Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty());
List<Row> results = queryResult.toList();
Assert.assertEquals(0, results.size());
@@ -581,7 +582,7 @@ public class NestedQueryPushDownTest
"finalSum", 4000L,
"newDimA", "mango"
);
- Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, new HashMap<>());
+ Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty());
List<Row> results = queryResult.toList();
Assert.assertEquals(1, results.size());
@@ -631,7 +632,7 @@ public class NestedQueryPushDownTest
"finalSum", 4000L,
"newDimA", "mango"
);
- Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, new HashMap<>());
+ Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty());
List<Row> results = queryResult.toList();
Assert.assertEquals(1, results.size());
@@ -682,7 +683,7 @@ public class NestedQueryPushDownTest
"finalSum", 4000L,
"extractedDimA", "replacement"
);
- Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, new HashMap<>());
+ Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty());
List<Row> results = queryResult.toList();
Assert.assertEquals(2, results.size());
@@ -728,14 +729,14 @@ public class NestedQueryPushDownTest
"dimB", "sweet",
"finalSum", 90L
);
- Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, new HashMap<>());
+ Sequence<Row> queryResult = runNestedQueryWithForcePushDown(nestedQuery, ResponseContext.createEmpty());
List<Row> results = queryResult.toList();
Assert.assertEquals(1, results.size());
Assert.assertEquals(expectedRow0, results.get(0));
}
- private Sequence<Row> runNestedQueryWithForcePushDown(GroupByQuery nestedQuery, Map<String, Object> context)
+ private Sequence<Row> runNestedQueryWithForcePushDown(GroupByQuery nestedQuery, ResponseContext context)
{
QueryToolChest<Row, GroupByQuery> toolChest = groupByFactory.getToolchest();
GroupByQuery pushDownQuery = nestedQuery;
diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java
index 1fcf16d..c7f65f5 100644
--- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java
@@ -39,7 +39,6 @@ import org.junit.Assert;
import org.junit.Test;
import java.util.EnumSet;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -193,7 +192,6 @@ public class SegmentAnalyzerTest
final SegmentMetadataQuery query = new SegmentMetadataQuery(
new LegacyDataSource("test"), new LegacySegmentSpec("2011/2012"), null, null, null, analyses, false, false
);
- HashMap<String, Object> context = new HashMap<String, Object>();
- return runner.run(QueryPlus.wrap(query), context).toList();
+ return runner.run(QueryPlus.wrap(query)).toList();
}
}
diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java
index e0b45c5..ed6d199 100644
--- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java
@@ -284,7 +284,7 @@ public class SegmentMetadataQueryTest
@SuppressWarnings("unchecked")
public void testSegmentMetadataQuery()
{
- List<SegmentAnalysis> results = runner1.run(QueryPlus.wrap(testQuery), new HashMap<>()).toList();
+ List<SegmentAnalysis> results = runner1.run(QueryPlus.wrap(testQuery)).toList();
Assert.assertEquals(Collections.singletonList(expectedSegmentAnalysis1), results);
}
@@ -351,7 +351,7 @@ public class SegmentMetadataQueryTest
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
- myRunner.run(QueryPlus.wrap(query), new HashMap<>()),
+ myRunner.run(QueryPlus.wrap(query)),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@@ -419,7 +419,7 @@ public class SegmentMetadataQueryTest
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
- myRunner.run(QueryPlus.wrap(query), new HashMap<>()),
+ myRunner.run(QueryPlus.wrap(query)),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@@ -487,7 +487,7 @@ public class SegmentMetadataQueryTest
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
- myRunner.run(QueryPlus.wrap(query), new HashMap<>()),
+ myRunner.run(QueryPlus.wrap(query)),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@@ -616,7 +616,7 @@ public class SegmentMetadataQueryTest
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
- myRunner.run(QueryPlus.wrap(query), new HashMap<>()),
+ myRunner.run(QueryPlus.wrap(query)),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@@ -674,7 +674,7 @@ public class SegmentMetadataQueryTest
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
- myRunner.run(QueryPlus.wrap(query), new HashMap<>()),
+ myRunner.run(QueryPlus.wrap(query)),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@@ -736,7 +736,7 @@ public class SegmentMetadataQueryTest
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
- myRunner.run(QueryPlus.wrap(query), new HashMap<>()),
+ myRunner.run(QueryPlus.wrap(query)),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@@ -794,7 +794,7 @@ public class SegmentMetadataQueryTest
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
- myRunner.run(QueryPlus.wrap(query), new HashMap<>()),
+ myRunner.run(QueryPlus.wrap(query)),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@@ -852,7 +852,7 @@ public class SegmentMetadataQueryTest
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
- myRunner.run(QueryPlus.wrap(query), new HashMap<>()),
+ myRunner.run(QueryPlus.wrap(query)),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@@ -889,10 +889,7 @@ public class SegmentMetadataQueryTest
TestHelper.assertExpectedObjects(
ImmutableList.of(bySegmentResult, bySegmentResult),
- myRunner.run(
- QueryPlus.wrap(testQuery.withOverriddenContext(ImmutableMap.of("bySegment", true))),
- new HashMap<>()
- ),
+ myRunner.run(QueryPlus.wrap(testQuery.withOverriddenContext(ImmutableMap.of("bySegment", true)))),
"failed SegmentMetadata bySegment query"
);
exec.shutdownNow();
diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java
index c1b37b2..e621811 100644
--- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java
@@ -41,7 +41,6 @@ import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
@RunWith(Parameterized.class)
@@ -123,7 +122,7 @@ public class SegmentMetadataUnionQueryTest
SegmentMetadataQuery.AnalysisType.MINMAX
)
.build();
- List result = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ List result = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedObjects(ImmutableList.of(expected), result, "failed SegmentMetadata union query");
}
diff --git a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java
index 7b051b0..1f64d12 100644
--- a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java
@@ -37,6 +37,7 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.select.SelectQueryRunnerTest;
import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.Segment;
@@ -56,9 +57,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;
/**
*
@@ -199,7 +198,7 @@ public class MultiSegmentScanQueryTest
Execs.directExecutor(),
ImmutableList.of(factory.createRunner(segment0), factory.createRunner(segment1))
)
- .run(QueryPlus.wrap(query), new HashMap<>())
+ .run(QueryPlus.wrap(query))
.toList();
int totalCount = 0;
for (ScanResultValue result : results) {
@@ -221,13 +220,13 @@ public class MultiSegmentScanQueryTest
@Override
public Sequence<ScanResultValue> run(
QueryPlus<ScanResultValue> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
// simulate results back from 2 historicals
List<Sequence<ScanResultValue>> sequences = Lists.newArrayListWithExpectedSize(2);
- sequences.add(factory.createRunner(segment0).run(queryPlus, new HashMap<>()));
- sequences.add(factory.createRunner(segment1).run(queryPlus, new HashMap<>()));
+ sequences.add(factory.createRunner(segment0).run(queryPlus));
+ sequences.add(factory.createRunner(segment1).run(queryPlus));
return new MergeSequence<>(
queryPlus.getQuery().getResultOrdering(),
Sequences.simple(sequences)
@@ -236,7 +235,7 @@ public class MultiSegmentScanQueryTest
}
);
ScanQuery query = newBuilder().build();
- List<ScanResultValue> results = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ List<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
int totalCount = 0;
for (ScanResultValue result : results) {
totalCount += ((List) result.getEvents()).size();
diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java
index f0dafbc..c03ddde 100644
--- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java
@@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.Druids;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunnerTestHelper;
+import org.apache.druid.query.context.ResponseContext;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@@ -114,7 +115,7 @@ public class ScanQueryLimitRowIteratorTest
ScanQueryLimitRowIterator itr = new ScanQueryLimitRowIterator(
((queryInput, responseContext) -> Sequences.simple(multiEventScanResultValues)),
queryPlus,
- ImmutableMap.of()
+ ResponseContext.createEmpty()
);
int count = 0;
@@ -154,7 +155,7 @@ public class ScanQueryLimitRowIteratorTest
ScanQueryLimitRowIterator itr = new ScanQueryLimitRowIterator(
((queryInput, responseContext) -> Sequences.simple(singleEventScanResultValues)),
queryPlus,
- ImmutableMap.of()
+ ResponseContext.createEmpty()
);
int count = 0;
@@ -195,7 +196,7 @@ public class ScanQueryLimitRowIteratorTest
ScanQueryLimitRowIterator itr = new ScanQueryLimitRowIterator(
((queryInput, responseContext) -> Sequences.simple(singleEventScanResultValues)),
queryPlus,
- ImmutableMap.of()
+ ResponseContext.createEmpty()
);
int count = 0;
diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java
index 4b65f53..cf76f37 100644
--- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java
@@ -20,7 +20,6 @@
package org.apache.druid.query.scan;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.guava.Sequence;
@@ -31,6 +30,7 @@ import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.SegmentDescriptor;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.spec.LegacySegmentSpec;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
@@ -229,7 +229,7 @@ public class ScanQueryRunnerFactoryTest
factory.nWayMergeAndLimit(
groupedRunners,
QueryPlus.wrap(query),
- ImmutableMap.of()
+ ResponseContext.createEmpty()
).toList();
validateSortedOutput(output, expectedEventTimestamps);
diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java
index 2928ca9..0507da3 100644
--- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java
@@ -180,8 +180,7 @@ public class ScanQueryRunnerTest
.virtualColumns(EXPR_COLUMN)
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
List<ScanResultValue> expectedResults = toExpected(
toFullEvents(V_0112_0114),
@@ -222,8 +221,7 @@ public class ScanQueryRunnerTest
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
List<ScanResultValue> expectedResults = toExpected(
toFullEvents(V_0112_0114),
@@ -246,8 +244,7 @@ public class ScanQueryRunnerTest
)
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
final List<List<Map<String, Object>>> expectedEvents = toEvents(
new String[]{
@@ -293,8 +290,7 @@ public class ScanQueryRunnerTest
.columns(QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.indexMetric)
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
List<ScanResultValue> expectedResults = toExpected(
toEvents(
@@ -328,8 +324,7 @@ public class ScanQueryRunnerTest
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
List<ScanResultValue> expectedResults = toExpected(
toEvents(
@@ -366,8 +361,7 @@ public class ScanQueryRunnerTest
.limit(limit)
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
final List<List<Map<String, Object>>> events = toEvents(
new String[]{
@@ -426,10 +420,10 @@ public class ScanQueryRunnerTest
.columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric)
.build();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
Iterable<ScanResultValue> resultsOptimize = toolChest
.postMergeQueryDecoration(toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)))
- .run(QueryPlus.wrap(query), new HashMap<>())
+ .run(QueryPlus.wrap(query))
.toList();
final List<List<Map<String, Object>>> events = toEvents(
@@ -485,7 +479,7 @@ public class ScanQueryRunnerTest
)
.build();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
List<ScanResultValue> expectedResults = Collections.emptyList();
@@ -500,7 +494,7 @@ public class ScanQueryRunnerTest
.columns("foo", "foo2")
.build();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
final List<List<Map<String, Object>>> events = toEvents(
legacy ? new String[]{getTimestampName() + ":TIME"} : new String[0],
@@ -535,8 +529,7 @@ public class ScanQueryRunnerTest
.context(ImmutableMap.of(ScanQuery.CTX_KEY_OUTERMOST, false))
.build();
- HashMap<String, Object> context = new HashMap<>();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
String[] seg1Results = new String[]{
"2011-01-12T00:00:00.000Z\tspot\tautomotive\tpreferred\tapreferred\t100.000000",
"2011-01-12T00:00:00.000Z\tspot\tbusiness\tpreferred\tbpreferred\t100.000000",
@@ -623,8 +616,7 @@ public class ScanQueryRunnerTest
.order(ScanQuery.Order.DESCENDING)
.build();
- HashMap<String, Object> context = new HashMap<>();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
String[] seg1Results = new String[]{
"2011-01-12T00:00:00.000Z\tspot\tautomotive\tpreferred\tapreferred\t100.000000",
"2011-01-12T00:00:00.000Z\tspot\tbusiness\tpreferred\tbpreferred\t100.000000",
@@ -736,8 +728,7 @@ public class ScanQueryRunnerTest
.limit(limit)
.build();
- HashMap<String, Object> context = new HashMap<>();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
final List<List<Map<String, Object>>> ascendingEvents = toEvents(
new String[]{
legacy ? getTimestampName() + ":TIME" : ColumnHolder.TIME_COLUMN_NAME,
@@ -827,8 +818,7 @@ public class ScanQueryRunnerTest
.limit(limit)
.build();
- HashMap<String, Object> context = new HashMap<>();
- Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
String[] expectedRet = (String[]) ArrayUtils.addAll(seg1Results, seg2Results);
ArrayUtils.reverse(expectedRet);
final List<List<Map<String, Object>>> descendingEvents = toEvents(
diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java
index af2bdf8..3b57315 100644
--- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java
@@ -37,6 +37,7 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.ExtractionDimensionSpec;
import org.apache.druid.query.extraction.ExtractionFn;
@@ -68,7 +69,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
-import java.util.Map;
/**
*/
@@ -166,7 +166,7 @@ public class SearchQueryRunnerTest
@Override
public Sequence<Result<SearchResultValue>> run(
QueryPlus<Result<SearchResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
final QueryPlus<Result<SearchResultValue>> queryPlus1 = queryPlus.withQuerySegmentSpec(
@@ -789,7 +789,7 @@ public class SearchQueryRunnerTest
private void checkSearchQuery(Query searchQuery, QueryRunner runner, List<SearchHit> expectedResults)
{
- Iterable<Result<SearchResultValue>> results = runner.run(QueryPlus.wrap(searchQuery), ImmutableMap.of()).toList();
+ Iterable<Result<SearchResultValue>> results = runner.run(QueryPlus.wrap(searchQuery)).toList();
List<SearchHit> copy = new ArrayList<>(expectedResults);
for (Result<SearchResultValue> result : results) {
Assert.assertEquals(DateTimes.of("2011-01-12T00:00:00.000Z"), result.getTimestamp());
diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java
index f954949..855f50d 100644
--- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java
+++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java
@@ -44,7 +44,6 @@ import org.junit.runners.Parameterized;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -248,9 +247,7 @@ public class SearchQueryRunnerWithCaseTest
private void checkSearchQuery(SearchQuery searchQuery, Map<String, Set<String>> expectedResults)
{
- HashMap<String, List> context = new HashMap<>();
- Iterable<Result<SearchResultValue>> results =
- runner.run(QueryPlus.<Result<SearchResultValue>>wrap(searchQuery), context).toList();
+ Iterable<Result<SearchResultValue>> results = runner.run(QueryPlus.wrap(searchQuery)).toList();
for (Result<SearchResultValue> result : results) {
Assert.assertEquals(DateTimes.of("2011-01-12T00:00:00.000Z"), result.getTimestamp());
diff --git a/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java b/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java
index d338b9e..5d06cc0 100644
--- a/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java
@@ -22,7 +22,6 @@ package org.apache.druid.query.select;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
import com.google.common.io.CharSource;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang.StringUtils;
@@ -243,7 +242,7 @@ public class MultiSegmentSelectQueryTest
private void runAllGranularityTest(SelectQuery query, int[][] expectedOffsets)
{
for (int[] expected : expectedOffsets) {
- List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
+ List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
Assert.assertEquals(1, results.size());
SelectResultValue value = results.get(0).getValue();
@@ -285,7 +284,7 @@ public class MultiSegmentSelectQueryTest
private void runDayGranularityTest(SelectQuery query, int[][] expectedOffsets)
{
for (int[] expected : expectedOffsets) {
- List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
+ List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
Assert.assertEquals(2, results.size());
SelectResultValue value0 = results.get(0).getValue();
@@ -327,12 +326,12 @@ public class MultiSegmentSelectQueryTest
SelectQuery query = selectQueryBuilder.build();
QueryRunner unionQueryRunner = new UnionQueryRunner(runner);
- List<Result<SelectResultValue>> results = unionQueryRunner.run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
+ List<Result<SelectResultValue>> results = unionQueryRunner.run(QueryPlus.wrap(query)).toList();
Map<String, Integer> pagingIdentifiers = results.get(0).getValue().getPagingIdentifiers();
query = query.withPagingSpec(toNextCursor(PagingSpec.merge(Collections.singletonList(pagingIdentifiers)), query, 3));
- unionQueryRunner.run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
+ unionQueryRunner.run(QueryPlus.wrap(query)).toList();
}
private PagingSpec toNextCursor(Map<String, Integer> merged, SelectQuery query, int threshold)
diff --git a/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java
index 55db5ef..16d46a8 100644
--- a/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java
@@ -174,8 +174,7 @@ public class SelectQueryRunnerTest
.intervals(I_0112_0114_SPEC)
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
PagingOffset offset = query.getPagingOffset(segmentIdString);
List<Result<SelectResultValue>> expectedResults = toExpected(
@@ -217,7 +216,7 @@ public class SelectQueryRunnerTest
SelectQuery query = newTestQuery().intervals(I_0112_0114_SPEC).build();
for (int offset : expected) {
- List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
+ List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
Assert.assertEquals(1, results.size());
@@ -231,7 +230,7 @@ public class SelectQueryRunnerTest
query = newTestQuery().intervals(I_0112_0114_SPEC).build();
for (int offset : expected) {
- List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
+ List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
Assert.assertEquals(1, results.size());
@@ -272,8 +271,7 @@ public class SelectQueryRunnerTest
)
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
String segmentIdInThisQuery = QueryRunnerTestHelper.segmentId.toString();
@@ -385,8 +383,7 @@ public class SelectQueryRunnerTest
.metrics(Collections.singletonList(QueryRunnerTestHelper.indexMetric))
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
PagingOffset offset = query.getPagingOffset(segmentIdString);
List<Result<SelectResultValue>> expectedResults = toExpected(
@@ -423,7 +420,7 @@ public class SelectQueryRunnerTest
.pagingSpec(new PagingSpec(toPagingIdentifier(3, descending), 3))
.build();
- Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
PagingOffset offset = query.getPagingOffset(segmentIdString);
List<Result<SelectResultValue>> expectedResults = toExpected(
@@ -458,8 +455,7 @@ public class SelectQueryRunnerTest
.pagingSpec(new PagingSpec(toPagingIdentifier(param[0], descending), param[1]))
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
final List<List<Map<String, Object>>> events = toEvents(
new String[]{
@@ -531,8 +527,7 @@ public class SelectQueryRunnerTest
)
.build();
- HashMap<String, Object> context = new HashMap<>();
- Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
final List<List<Map<String, Object>>> events = toEvents(
new String[]{
@@ -579,10 +574,10 @@ public class SelectQueryRunnerTest
.metrics(Collections.singletonList(QueryRunnerTestHelper.indexMetric))
.build();
- Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
Iterable<Result<SelectResultValue>> resultsOptimize = toolChest
.postMergeQueryDecoration(toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)))
- .run(QueryPlus.wrap(query), new HashMap<>())
+ .run(QueryPlus.wrap(query))
.toList();
final List<List<Map<String, Object>>> events = toEvents(
@@ -634,7 +629,7 @@ public class SelectQueryRunnerTest
)
.build();
- Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
List<Result<SelectResultValue>> expectedResults = Collections.singletonList(
new Result<SelectResultValue>(
@@ -679,7 +674,7 @@ public class SelectQueryRunnerTest
.metrics(Collections.singletonList("foo2"))
.build();
- Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
final List<List<Map<String, Object>>> events = toEvents(
new String[]{
@@ -716,8 +711,7 @@ public class SelectQueryRunnerTest
.intervals(I_0112_0114_SPEC)
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
List<Result<SelectResultValue>> expectedResultsAsc = Collections.singletonList(
new Result<SelectResultValue>(
@@ -831,8 +825,7 @@ public class SelectQueryRunnerTest
.intervals(I_0112_0114_SPEC)
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
List<Result<SelectResultValue>> expectedResultsAsc = Collections.singletonList(
new Result<SelectResultValue>(
diff --git a/processing/src/test/java/org/apache/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/spec/SpecificSegmentQueryRunnerTest.java
index 90f9730..f1b185a 100644
--- a/processing/src/test/java/org/apache/druid/query/spec/SpecificSegmentQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/spec/SpecificSegmentQueryRunnerTest.java
@@ -38,6 +38,7 @@ import org.apache.druid.query.Result;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.aggregation.CountAggregator;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.timeseries.TimeseriesResultBuilder;
import org.apache.druid.query.timeseries.TimeseriesResultValue;
@@ -48,9 +49,7 @@ import org.junit.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map;
public class SpecificSegmentQueryRunnerTest
{
@@ -68,7 +67,7 @@ public class SpecificSegmentQueryRunnerTest
new QueryRunner()
{
@Override
- public Sequence run(QueryPlus queryPlus, Map responseContext)
+ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext)
{
return new Sequence()
{
@@ -93,7 +92,7 @@ public class SpecificSegmentQueryRunnerTest
);
// from accumulate
- Map<String, Object> responseContext = new HashMap<>();
+ ResponseContext responseContext = ResponseContext.createEmpty();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource("foo")
.granularity(Granularities.ALL)
@@ -109,7 +108,7 @@ public class SpecificSegmentQueryRunnerTest
validate(mapper, descriptor, responseContext);
// from toYielder
- responseContext = new HashMap<>();
+ responseContext = ResponseContext.createEmpty();
results = queryRunner.run(QueryPlus.wrap(query), responseContext);
results.toYielder(
null,
@@ -151,7 +150,7 @@ public class SpecificSegmentQueryRunnerTest
new QueryRunner()
{
@Override
- public Sequence run(QueryPlus queryPlus, Map responseContext)
+ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext)
{
return Sequences.withEffect(
Sequences.simple(Collections.singletonList(value)),
@@ -172,7 +171,7 @@ public class SpecificSegmentQueryRunnerTest
)
);
- final Map<String, Object> responseContext = new HashMap<>();
+ final ResponseContext responseContext = ResponseContext.createEmpty();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource("foo")
.granularity(Granularities.ALL)
@@ -195,10 +194,10 @@ public class SpecificSegmentQueryRunnerTest
validate(mapper, descriptor, responseContext);
}
- private void validate(ObjectMapper mapper, SegmentDescriptor descriptor, Map<String, Object> responseContext)
+ private void validate(ObjectMapper mapper, SegmentDescriptor descriptor, ResponseContext responseContext)
throws IOException
{
- Object missingSegments = responseContext.get(Result.MISSING_SEGMENTS_KEY);
+ Object missingSegments = responseContext.get(ResponseContext.CTX_MISSING_SEGMENTS);
Assert.assertTrue(missingSegments != null);
Assert.assertTrue(missingSegments instanceof List);
diff --git a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java
index 769d5cf..deac092 100644
--- a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java
@@ -32,6 +32,8 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.Result;
import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.context.ConcurrentResponseContext;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.Segment;
@@ -52,10 +54,7 @@ import org.junit.runners.Parameterized;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.HashMap;
import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
/**
*/
@@ -162,9 +161,8 @@ public class TimeBoundaryQueryRunnerTest
.filters("quality", "automotive")
.build();
Assert.assertTrue(timeBoundaryQuery.hasFilters());
- HashMap<String, Object> context = new HashMap<String, Object>();
List<Result<TimeBoundaryResultValue>> results =
- customRunner.run(QueryPlus.wrap(timeBoundaryQuery), context).toList();
+ customRunner.run(QueryPlus.wrap(timeBoundaryQuery)).toList();
Assert.assertTrue(Iterables.size(results) > 0);
@@ -186,9 +184,8 @@ public class TimeBoundaryQueryRunnerTest
.filters("quality", "foobar") // foobar dimension does not exist
.build();
Assert.assertTrue(timeBoundaryQuery.hasFilters());
- HashMap<String, Object> context = new HashMap<String, Object>();
List<Result<TimeBoundaryResultValue>> results =
- customRunner.run(QueryPlus.wrap(timeBoundaryQuery), context).toList();
+ customRunner.run(QueryPlus.wrap(timeBoundaryQuery)).toList();
Assert.assertTrue(Iterables.size(results) == 0);
}
@@ -201,8 +198,7 @@ public class TimeBoundaryQueryRunnerTest
.dataSource("testing")
.build();
Assert.assertFalse(timeBoundaryQuery.hasFilters());
- HashMap<String, Object> context = new HashMap<String, Object>();
- Iterable<Result<TimeBoundaryResultValue>> results = runner.run(QueryPlus.wrap(timeBoundaryQuery), context).toList();
+ Iterable<Result<TimeBoundaryResultValue>> results = runner.run(QueryPlus.wrap(timeBoundaryQuery)).toList();
TimeBoundaryResultValue val = results.iterator().next().getValue();
DateTime minTime = val.getMinTime();
DateTime maxTime = val.getMaxTime();
@@ -219,8 +215,8 @@ public class TimeBoundaryQueryRunnerTest
.dataSource("testing")
.bound(TimeBoundaryQuery.MAX_TIME)
.build();
- ConcurrentMap<String, Object> context = new ConcurrentHashMap<>();
- context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
+ ResponseContext context = ConcurrentResponseContext.createEmpty();
+ context.put(ResponseContext.CTX_MISSING_SEGMENTS, new ArrayList<>());
Iterable<Result<TimeBoundaryResultValue>> results = runner.run(QueryPlus.wrap(timeBoundaryQuery), context).toList();
TimeBoundaryResultValue val = results.iterator().next().getValue();
DateTime minTime = val.getMinTime();
@@ -238,8 +234,8 @@ public class TimeBoundaryQueryRunnerTest
.dataSource("testing")
.bound(TimeBoundaryQuery.MIN_TIME)
.build();
- ConcurrentMap<String, Object> context = new ConcurrentHashMap<>();
- context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
+ ResponseContext context = ConcurrentResponseContext.createEmpty();
+ context.put(ResponseContext.CTX_MISSING_SEGMENTS, new ArrayList<>());
Iterable<Result<TimeBoundaryResultValue>> results = runner.run(QueryPlus.wrap(timeBoundaryQuery), context).toList();
TimeBoundaryResultValue val = results.iterator().next().getValue();
DateTime minTime = val.getMinTime();
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java
index 48a322b..44d767e 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java
@@ -34,15 +34,14 @@ import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.UnionDataSource;
import org.apache.druid.query.UnionQueryRunner;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.TestHelper;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.util.Arrays;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map;
@RunWith(Parameterized.class)
public class TimeSeriesUnionQueryRunnerTest
@@ -114,8 +113,7 @@ public class TimeSeriesUnionQueryRunnerTest
)
)
);
- HashMap<String, Object> context = new HashMap<>();
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -178,7 +176,7 @@ public class TimeSeriesUnionQueryRunnerTest
@Override
public Sequence<Result<TimeseriesResultValue>> run(
QueryPlus<Result<TimeseriesResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
if (queryPlus.getQuery().getDataSource().equals(new TableDataSource("ds1"))) {
@@ -218,8 +216,7 @@ public class TimeSeriesUnionQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results =
- mergingrunner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
+ Iterable<Result<TimeseriesResultValue>> results = mergingrunner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java
index 22134a1..b1ca4c7 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java
@@ -45,7 +45,6 @@ import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.util.Arrays;
-import java.util.HashMap;
import java.util.List;
@RunWith(Parameterized.class)
@@ -129,8 +128,7 @@ public class TimeseriesQueryRunnerBonusTest
.aggregators(new CountAggregatorFactory("rows"))
.descending(descending)
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
- return runner.run(QueryPlus.wrap(query), context).toList();
+ return runner.run(QueryPlus.wrap(query)).toList();
}
private static <T> QueryRunner<T> makeQueryRunner(QueryRunnerFactory<T, Query<T>> factory, Segment adapter)
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
index 4c8d91c..0560847 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
@@ -86,8 +86,6 @@ import java.util.stream.StreamSupport;
@RunWith(Parameterized.class)
public class TimeseriesQueryRunnerTest
{
- public static final Map<String, Object> CONTEXT = ImmutableMap.of();
-
@Rule
public ExpectedException expectedException = ExpectedException.none();
@@ -188,7 +186,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, actualResults);
}
@@ -210,7 +208,7 @@ public class TimeseriesQueryRunnerTest
.context(makeContext())
.build();
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
final String[] expectedIndex = descending ?
QueryRunnerTestHelper.expectedFullOnIndexValuesDesc :
@@ -311,7 +309,7 @@ public class TimeseriesQueryRunnerTest
.context(makeContext())
.build();
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
final DateTime expectedLast = descending ?
QueryRunnerTestHelper.earliest :
@@ -354,7 +352,7 @@ public class TimeseriesQueryRunnerTest
DateTime expectedEarliest = DateTimes.of("2011-01-12");
DateTime expectedLast = DateTimes.of("2011-04-15");
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
Result<TimeseriesResultValue> result = results.iterator().next();
Assert.assertEquals(expectedEarliest, result.getTimestamp());
@@ -397,7 +395,7 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.earliest :
QueryRunnerTestHelper.last;
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
for (Result<TimeseriesResultValue> result : results) {
DateTime current = result.getTimestamp();
@@ -460,7 +458,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -555,7 +553,7 @@ public class TimeseriesQueryRunnerTest
toolChest
);
- final List results = finalRunner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ final List results = finalRunner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, results);
}
@@ -606,7 +604,7 @@ public class TimeseriesQueryRunnerTest
toolChest
);
- final List results = finalRunner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ final List results = finalRunner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, results);
}
@@ -654,7 +652,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -700,7 +698,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -739,7 +737,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results1 = runner.run(QueryPlus.wrap(query1), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results1 = runner.run(QueryPlus.wrap(query1)).toList();
assertExpectedResults(expectedResults1, results1);
TimeseriesQuery query2 = Druids.newTimeseriesQueryBuilder()
@@ -772,7 +770,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results2 = runner.run(QueryPlus.wrap(query2), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results2 = runner.run(QueryPlus.wrap(query2)).toList();
assertExpectedResults(expectedResults2, results2);
}
@@ -822,7 +820,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results1 = runner.run(QueryPlus.wrap(query1), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results1 = runner.run(QueryPlus.wrap(query1)).toList();
assertExpectedResults(expectedResults1, results1);
}
@@ -889,7 +887,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results1 = runner.run(QueryPlus.wrap(query1), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results1 = runner.run(QueryPlus.wrap(query1)).toList();
assertExpectedResults(expectedResults1, results1);
}
@@ -929,7 +927,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results1 = runner.run(QueryPlus.wrap(query1), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results1 = runner.run(QueryPlus.wrap(query1)).toList();
assertExpectedResults(expectedResults1, results1);
}
@@ -967,7 +965,7 @@ public class TimeseriesQueryRunnerTest
)
)
);
- Iterable<Result<TimeseriesResultValue>> results1 = runner.run(QueryPlus.wrap(query1), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results1 = runner.run(QueryPlus.wrap(query1)).toList();
assertExpectedResults(expectedResults1, results1);
TimeseriesQuery query2 = Druids.newTimeseriesQueryBuilder()
@@ -1001,7 +999,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results2 = runner.run(QueryPlus.wrap(query2), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results2 = runner.run(QueryPlus.wrap(query2)).toList();
assertExpectedResults(expectedResults2, results2);
}
@@ -1031,7 +1029,7 @@ public class TimeseriesQueryRunnerTest
List<Result<TimeseriesResultValue>> expectedResults = Collections.emptyList();
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1078,7 +1076,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1129,7 +1127,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1176,7 +1174,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1223,7 +1221,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1270,7 +1268,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1317,7 +1315,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1364,7 +1362,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1411,7 +1409,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1464,7 +1462,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1523,7 +1521,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1570,7 +1568,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1609,7 +1607,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1630,7 +1628,7 @@ public class TimeseriesQueryRunnerTest
List<Result<TimeseriesResultValue>> expectedResults = Collections.emptyList();
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), new HashMap<String, Object>())
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query))
.toList();
assertExpectedResults(expectedResults, results);
}
@@ -1674,7 +1672,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), new HashMap<String, Object>())
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query))
.toList();
assertExpectedResults(expectedResults, results);
}
@@ -1718,7 +1716,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), new HashMap<String, Object>())
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query))
.toList();
assertExpectedResults(expectedResults, results);
}
@@ -1757,7 +1755,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1799,7 +1797,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, results);
}
@@ -1837,7 +1835,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, actualResults);
}
@@ -1876,7 +1874,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, actualResults);
}
@@ -1981,7 +1979,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
if (descending) {
TestHelper.assertExpectedResults(expectedDescendingResults, actualResults);
} else {
@@ -2014,8 +2012,8 @@ public class TimeseriesQueryRunnerTest
.descending(descending)
.context(makeContext())
.build();
- Iterable<Result<TimeseriesResultValue>> expectedResults = runner.run(QueryPlus.wrap(query1), CONTEXT).toList();
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> expectedResults = runner.run(QueryPlus.wrap(query1)).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, actualResults);
}
@@ -2045,8 +2043,8 @@ public class TimeseriesQueryRunnerTest
.descending(descending)
.context(makeContext())
.build();
- Iterable<Result<TimeseriesResultValue>> expectedResults = runner.run(QueryPlus.wrap(query1), CONTEXT).toList();
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> expectedResults = runner.run(QueryPlus.wrap(query1)).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, actualResults);
}
@@ -2085,8 +2083,8 @@ public class TimeseriesQueryRunnerTest
.descending(descending)
.context(makeContext())
.build();
- Iterable<Result<TimeseriesResultValue>> expectedResults = runner.run(QueryPlus.wrap(query2), CONTEXT).toList();
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> expectedResults = runner.run(QueryPlus.wrap(query2)).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, actualResults);
}
@@ -2125,8 +2123,8 @@ public class TimeseriesQueryRunnerTest
.descending(descending)
.context(makeContext())
.build();
- Iterable<Result<TimeseriesResultValue>> expectedResults = runner.run(QueryPlus.wrap(query2), CONTEXT).toList();
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> expectedResults = runner.run(QueryPlus.wrap(query2)).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, actualResults);
}
@@ -2154,7 +2152,7 @@ public class TimeseriesQueryRunnerTest
.context(makeContext())
.build();
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
List<Result<TimeseriesResultValue>> expectedResults = Collections.singletonList(
new Result<>(
DateTimes.of("2011-04-01"),
@@ -2199,7 +2197,7 @@ public class TimeseriesQueryRunnerTest
.context(makeContext())
.build();
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
List<Result<TimeseriesResultValue>> expectedResults = Collections.singletonList(
new Result<>(
@@ -2245,7 +2243,7 @@ public class TimeseriesQueryRunnerTest
.context(makeContext())
.build();
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
List<Result<TimeseriesResultValue>> expectedResults = Collections.singletonList(
new Result<>(
@@ -2293,7 +2291,7 @@ public class TimeseriesQueryRunnerTest
.context(makeContext())
.build();
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
List<Result<TimeseriesResultValue>> expectedResults = Collections.singletonList(
new Result<>(
DateTimes.of("2011-04-01"),
@@ -2338,7 +2336,7 @@ public class TimeseriesQueryRunnerTest
.context(makeContext())
.build();
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
List<Result<TimeseriesResultValue>> expectedResults = Collections.singletonList(
new Result<>(
DateTimes.of("2011-04-01"),
@@ -2392,7 +2390,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
assertExpectedResults(expectedResults, actualResults);
}
@@ -2474,7 +2472,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, results);
}
@@ -2531,7 +2529,7 @@ public class TimeseriesQueryRunnerTest
)
);
- Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ Iterable<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, results);
QueryToolChest<Result<TimeseriesResultValue>, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest(
@@ -2540,7 +2538,7 @@ public class TimeseriesQueryRunnerTest
QueryRunner<Result<TimeseriesResultValue>> optimizedRunner = toolChest.postMergeQueryDecoration(
toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)));
Iterable<Result<TimeseriesResultValue>> results2 = new FinalizeResultsQueryRunner(optimizedRunner, toolChest)
- .run(QueryPlus.wrap(query), CONTEXT)
+ .run(QueryPlus.wrap(query))
.toList();
TestHelper.assertExpectedResults(expectedResults, results2);
@@ -2575,7 +2573,7 @@ public class TimeseriesQueryRunnerTest
toolChest
);
- final List list = finalRunner.run(QueryPlus.wrap(query), CONTEXT).toList();
+ final List list = finalRunner.run(QueryPlus.wrap(query)).toList();
Assert.assertEquals(10, list.size());
}
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java
index f9080e7..05342e5 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java
@@ -51,6 +51,7 @@ import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
import org.apache.druid.query.aggregation.post.ConstantPostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.segment.IncrementalIndexSegment;
@@ -270,21 +271,16 @@ public class TopNQueryQueryToolChestTest
TopNQuery query1 = builder.threshold(10).context(null).build();
MockQueryRunner mockRunner = new MockQueryRunner(runner);
- new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config).run(
- QueryPlus.wrap(query1),
- ImmutableMap.of()
- );
+ new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config).run(QueryPlus.wrap(query1));
Assert.assertEquals(1000, mockRunner.query.getThreshold());
TopNQuery query2 = builder.threshold(10).context(context).build();
- new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config)
- .run(QueryPlus.wrap(query2), ImmutableMap.of());
+ new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config).run(QueryPlus.wrap(query2));
Assert.assertEquals(500, mockRunner.query.getThreshold());
TopNQuery query3 = builder.threshold(2000).context(context).build();
- new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config)
- .run(QueryPlus.wrap(query3), ImmutableMap.of());
+ new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config).run(QueryPlus.wrap(query3));
Assert.assertEquals(2000, mockRunner.query.getThreshold());
}
}
@@ -563,7 +559,7 @@ public class TopNQueryQueryToolChestTest
@Override
public Sequence<Result<TopNResultValue>> run(
QueryPlus<Result<TopNResultValue>> queryPlus,
- Map<String, Object> responseContext
+ ResponseContext responseContext
)
{
this.query = (TopNQuery) queryPlus.getQuery();
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java
index 171f422..eb7e382 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java
@@ -57,8 +57,6 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
private static final String marketDimension = "market";
private static final SegmentId segmentId = SegmentId.dummy("testSegment");
- private static final HashMap<String, Object> context = new HashMap<String, Object>();
-
private static final TopNQuery query = new TopNQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
@@ -132,7 +130,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
@Test
public void testmMapped()
{
- testCaseMap.get(TestCases.mMappedTestIndex).run(QueryPlus.wrap(query), context);
+ testCaseMap.get(TestCases.mMappedTestIndex).run(QueryPlus.wrap(query));
}
@Ignore
@@ -140,7 +138,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
@Test
public void testrtIndex()
{
- testCaseMap.get(TestCases.rtIndex).run(QueryPlus.wrap(query), context);
+ testCaseMap.get(TestCases.rtIndex).run(QueryPlus.wrap(query));
}
@Ignore
@@ -148,7 +146,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
@Test
public void testMerged()
{
- testCaseMap.get(TestCases.mergedRealtimeIndex).run(QueryPlus.wrap(query), context);
+ testCaseMap.get(TestCases.mergedRealtimeIndex).run(QueryPlus.wrap(query));
}
@Ignore
@@ -156,6 +154,6 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
@Test
public void testOffHeap()
{
- testCaseMap.get(TestCases.rtIndexOffheap).run(QueryPlus.wrap(query), context);
+ testCaseMap.get(TestCases.rtIndexOffheap).run(QueryPlus.wrap(query));
}
}
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
index 4b9fb51..6a06d73 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
@@ -65,6 +65,7 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFact
import org.apache.druid.query.aggregation.last.FloatLastAggregatorFactory;
import org.apache.druid.query.aggregation.last.LongLastAggregatorFactory;
import org.apache.druid.query.aggregation.post.ExpressionPostAggregator;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.dimension.ExtractionDimensionSpec;
@@ -249,14 +250,12 @@ public class TopNQueryRunnerTest
return retval;
}
- private Sequence<Result<TopNResultValue>> runWithMerge(
- TopNQuery query
- )
+ private Sequence<Result<TopNResultValue>> runWithMerge(TopNQuery query)
{
- return runWithMerge(query, ImmutableMap.of());
+ return runWithMerge(query, ResponseContext.createEmpty());
}
- private Sequence<Result<TopNResultValue>> runWithMerge(TopNQuery query, Map<String, Object> context)
+ private Sequence<Result<TopNResultValue>> runWithMerge(TopNQuery query, ResponseContext context)
{
final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest(
new TopNQueryConfig(),
@@ -1301,10 +1300,10 @@ public class TopNQueryRunnerTest
)
)
);
- Sequence<Result<TopNResultValue>> results = runWithMerge(
- query,
- specialContext
- );
+
+ final ResponseContext responseContext = ResponseContext.createEmpty();
+ responseContext.putAll(specialContext);
+ Sequence<Result<TopNResultValue>> results = runWithMerge(query, responseContext);
List<Result<BySegmentTopNResultValue>> resultList = results
.map((Result<TopNResultValue> input) -> {
// Stupid type erasure
@@ -4137,7 +4136,7 @@ public class TopNQueryRunnerTest
)
)
);
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), new HashMap<String, Object>()));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@Test
@@ -4176,7 +4175,7 @@ public class TopNQueryRunnerTest
)
)
);
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), new HashMap<String, Object>()));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@@ -4292,10 +4291,10 @@ public class TopNQueryRunnerTest
private Sequence<Result<TopNResultValue>> runWithPreMergeAndMerge(TopNQuery query)
{
- return runWithPreMergeAndMerge(query, ImmutableMap.of());
+ return runWithMerge(query, ResponseContext.createEmpty());
}
- private Sequence<Result<TopNResultValue>> runWithPreMergeAndMerge(TopNQuery query, Map<String, Object> context)
+ private Sequence<Result<TopNResultValue>> runWithPreMergeAndMerge(TopNQuery query, ResponseContext context)
{
final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest(
new TopNQueryConfig(),
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java
index 6f6f9ee..897b483 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java
@@ -42,7 +42,6 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -180,8 +179,7 @@ public class TopNUnionQueryTest
)
)
);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
diff --git a/processing/src/test/java/org/apache/druid/segment/AppendTest.java b/processing/src/test/java/org/apache/druid/segment/AppendTest.java
index 0bb2036..7bea453 100644
--- a/processing/src/test/java/org/apache/druid/segment/AppendTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/AppendTest.java
@@ -68,7 +68,6 @@ import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -185,8 +184,7 @@ public class AppendTest
.dataSource(dataSource)
.build();
QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@Test
@@ -210,8 +208,7 @@ public class AppendTest
.dataSource(dataSource)
.build();
QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment2);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@Test
@@ -235,8 +232,7 @@ public class AppendTest
TimeseriesQuery query = makeTimeseriesQuery();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@Test
@@ -260,8 +256,7 @@ public class AppendTest
TimeseriesQuery query = makeTimeseriesQuery();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@Test
@@ -285,8 +280,7 @@ public class AppendTest
TimeseriesQuery query = makeFilteredTimeseriesQuery();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@Test
@@ -310,8 +304,7 @@ public class AppendTest
TimeseriesQuery query = makeFilteredTimeseriesQuery();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@Test
@@ -357,8 +350,7 @@ public class AppendTest
TopNQuery query = makeTopNQuery();
try (CloseableStupidPool<ByteBuffer> pool = TestQueryRunners.createDefaultNonBlockingPool()) {
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment, pool);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
}
@@ -405,8 +397,7 @@ public class AppendTest
TopNQuery query = makeTopNQuery();
try (CloseableStupidPool<ByteBuffer> pool = TestQueryRunners.createDefaultNonBlockingPool()) {
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2, pool);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
}
@@ -435,8 +426,7 @@ public class AppendTest
TopNQuery query = makeFilteredTopNQuery();
try (CloseableStupidPool<ByteBuffer> pool = TestQueryRunners.createDefaultNonBlockingPool()) {
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment, pool);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
}
@@ -455,8 +445,7 @@ public class AppendTest
TopNQuery query = makeFilteredTopNQuery();
try (CloseableStupidPool<ByteBuffer> pool = TestQueryRunners.createDefaultNonBlockingPool()) {
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2, pool);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
}
@@ -479,8 +468,7 @@ public class AppendTest
SearchQuery query = makeSearchQuery();
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@Test
@@ -501,8 +489,7 @@ public class AppendTest
SearchQuery query = makeSearchQuery();
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@Test
@@ -522,8 +509,7 @@ public class AppendTest
SearchQuery query = makeFilteredSearchQuery();
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@Test
@@ -544,8 +530,7 @@ public class AppendTest
SearchQuery query = makeFilteredSearchQuery();
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@Test
@@ -586,8 +571,7 @@ public class AppendTest
.postAggregators(addRowsIndexConstant)
.build();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment3);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
private TimeseriesQuery makeTimeseriesQuery()
diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java
index f8819ab..72b639e 100644
--- a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java
@@ -59,7 +59,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Random;
import java.util.concurrent.ThreadLocalRandom;
@@ -584,7 +583,7 @@ public class IndexMergerV9WithSpatialIndexTest
factory.getToolchest()
);
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), new HashMap<>()));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
catch (Exception e) {
throw new RuntimeException(e);
@@ -638,7 +637,7 @@ public class IndexMergerV9WithSpatialIndexTest
factory.getToolchest()
);
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), new HashMap<>()));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
catch (Exception e) {
throw new RuntimeException(e);
@@ -727,7 +726,7 @@ public class IndexMergerV9WithSpatialIndexTest
factory.getToolchest()
);
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), new HashMap<>()));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
catch (Exception e) {
throw new RuntimeException(e);
diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessTestFullTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessTestFullTest.java
index 3786564..a5487bc 100644
--- a/processing/src/test/java/org/apache/druid/segment/SchemalessTestFullTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/SchemalessTestFullTest.java
@@ -67,7 +67,6 @@ import org.junit.runners.Parameterized;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -1466,8 +1465,7 @@ public class SchemalessTestFullTest
.build();
failMsg += " timeseries ";
- HashMap<String, Object> context = new HashMap<>();
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
}
@@ -1497,8 +1495,7 @@ public class SchemalessTestFullTest
.build();
failMsg += " filtered timeseries ";
- HashMap<String, Object> context = new HashMap<>();
- Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<TimeseriesResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
}
@@ -1528,8 +1525,7 @@ public class SchemalessTestFullTest
.build();
failMsg += " topN ";
- HashMap<String, Object> context = new HashMap<>();
- Iterable<Result<TopNResultValue>> actualResults = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<TopNResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
}
@@ -1561,8 +1557,7 @@ public class SchemalessTestFullTest
.build();
failMsg += " filtered topN ";
- HashMap<String, Object> context = new HashMap<>();
- Iterable<Result<TopNResultValue>> actualResults = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<TopNResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
}
@@ -1576,8 +1571,7 @@ public class SchemalessTestFullTest
.build();
failMsg += " search ";
- HashMap<String, Object> context = new HashMap<>();
- Iterable<Result<SearchResultValue>> actualResults = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<SearchResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
}
@@ -1592,8 +1586,7 @@ public class SchemalessTestFullTest
.build();
failMsg += " filtered search ";
- HashMap<String, Object> context = new HashMap<>();
- Iterable<Result<SearchResultValue>> actualResults = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<SearchResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
}
@@ -1608,8 +1601,7 @@ public class SchemalessTestFullTest
.build();
failMsg += " timeBoundary ";
- HashMap<String, Object> context = new HashMap<>();
- Iterable<Result<TimeBoundaryResultValue>> actualResults = runner.run(QueryPlus.wrap(query), context).toList();
+ Iterable<Result<TimeBoundaryResultValue>> actualResults = runner.run(QueryPlus.wrap(query)).toList();
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
}
}
diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessTestSimpleTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessTestSimpleTest.java
index b0fbf62..4cea909 100644
--- a/processing/src/test/java/org/apache/druid/segment/SchemalessTestSimpleTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/SchemalessTestSimpleTest.java
@@ -67,7 +67,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
/**
@@ -161,8 +160,7 @@ public class SchemalessTestSimpleTest
)
);
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment);
- HashMap<String, Object> context = new HashMap();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@@ -238,8 +236,7 @@ public class SchemalessTestSimpleTest
try (CloseableStupidPool<ByteBuffer> pool = TestQueryRunners.createDefaultNonBlockingPool()) {
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment, pool);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
}
@@ -268,8 +265,7 @@ public class SchemalessTestSimpleTest
);
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
@Test
@@ -294,7 +290,6 @@ public class SchemalessTestSimpleTest
);
QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
}
diff --git a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java
index 407b700..319e66c 100644
--- a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java
@@ -80,10 +80,8 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
-import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
@@ -442,7 +440,7 @@ public class IncrementalIndexTest
);
- List<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), new HashMap<String, Object>()).toList();
+ List<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
Result<TimeseriesResultValue> result = Iterables.getOnlyElement(results);
boolean isRollup = index.isRollup();
Assert.assertEquals(rows * (isRollup ? 1 : 2), result.getValue().getLongMetric("rows").intValue());
@@ -597,8 +595,7 @@ public class IncrementalIndexTest
factory.createRunner(incrementalIndexSegment),
factory.getToolchest()
);
- Map<String, Object> context = new HashMap<String, Object>();
- Sequence<Result<TimeseriesResultValue>> sequence = runner.run(QueryPlus.wrap(query), context);
+ Sequence<Result<TimeseriesResultValue>> sequence = runner.run(QueryPlus.wrap(query));
Double[] results = sequence.accumulate(
new Double[0],
@@ -653,8 +650,7 @@ public class IncrementalIndexTest
.intervals(ImmutableList.of(queryInterval))
.aggregators(queryAggregatorFactories)
.build();
- Map<String, Object> context = new HashMap<String, Object>();
- List<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ List<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
boolean isRollup = index.isRollup();
for (Result<TimeseriesResultValue> result : results) {
Assert.assertEquals(
diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java
index e143caa..bf29d87 100644
--- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java
@@ -68,7 +68,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Random;
@@ -516,8 +515,7 @@ public class SpatialFilterBonusTest
factory.createRunner(segment),
factory.getToolchest()
);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
catch (Exception e) {
throw new RuntimeException(e);
@@ -604,8 +602,7 @@ public class SpatialFilterBonusTest
factory.createRunner(segment),
factory.getToolchest()
);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
catch (Exception e) {
throw new RuntimeException(e);
@@ -698,8 +695,7 @@ public class SpatialFilterBonusTest
factory.createRunner(segment),
factory.getToolchest()
);
- HashMap<String, Object> context = new HashMap<String, Object>();
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
catch (Exception e) {
throw new RuntimeException(e);
diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java
index 74b3070..7142536 100644
--- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java
@@ -65,7 +65,6 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Random;
import java.util.concurrent.ThreadLocalRandom;
@@ -573,7 +572,7 @@ public class SpatialFilterTest
factory.getToolchest()
);
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), new HashMap<>()));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
catch (Exception e) {
throw new RuntimeException(e);
@@ -626,7 +625,7 @@ public class SpatialFilterTest
factory.getToolchest()
);
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), new HashMap<>()));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
catch (Exception e) {
throw new RuntimeException(e);
@@ -714,7 +713,7 @@ public class SpatialFilterTest
factory.getToolchest()
);
- TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), new HashMap<>()));
+ TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query)));
}
catch (Exception e) {
throw new RuntimeException(e);
diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java
index 0f1944a..924ab99 100644
--- a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java
+++ b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java
@@ -66,9 +66,7 @@ import org.junit.runners.Parameterized;
import java.lang.reflect.InvocationTargetException;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
@@ -394,8 +392,7 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark
.intervals(ImmutableList.of(queryInterval))
.aggregators(queryAggregatorFactories)
.build();
- Map<String, Object> context = new HashMap<String, Object>();
- List<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ List<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
for (Result<TimeseriesResultValue> result : results) {
if (someoneRan.get()) {
Assert.assertTrue(result.getValue().getDoubleMetric("doubleSumResult0") > 0);
@@ -427,8 +424,7 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark
.intervals(ImmutableList.of(queryInterval))
.aggregators(queryAggregatorFactories)
.build();
- Map<String, Object> context = new HashMap<String, Object>();
- List<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ List<Result<TimeseriesResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
final int expectedVal = elementsPerThread * taskCount;
for (Result<TimeseriesResultValue> result : results) {
Assert.assertEquals(elementsPerThread, result.getValue().getLongMetric("rows").intValue());
diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java
index 1facccd..06c4530 100644
--- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java
+++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java
@@ -60,6 +60,7 @@ import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.Result;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.aggregation.MetricManipulatorFns;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.filter.DimFilterUtils;
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
import org.apache.druid.server.QueryResource;
@@ -148,7 +149,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
return new QueryRunner<T>()
{
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
return CachingClusteredClient.this.run(queryPlus, responseContext, timeline -> timeline);
}
@@ -161,7 +162,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
*/
private <T> Sequence<T> run(
final QueryPlus<T> queryPlus,
- final Map<String, Object> responseContext,
+ final ResponseContext responseContext,
final UnaryOperator<TimelineLookup<String, ServerSelector>> timelineConverter
)
{
@@ -174,7 +175,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
return new QueryRunner<T>()
{
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
return CachingClusteredClient.this.run(
queryPlus,
@@ -207,7 +208,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
private class SpecificQueryRunnable<T>
{
private final QueryPlus<T> queryPlus;
- private final Map<String, Object> responseContext;
+ private final ResponseContext responseContext;
private final Query<T> query;
private final QueryToolChest<T, Query<T>> toolChest;
@Nullable
@@ -220,7 +221,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
private final Map<String, Cache.NamedKey> cachePopulatorKeyMap = new HashMap<>();
private final List<Interval> intervals;
- SpecificQueryRunnable(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ SpecificQueryRunnable(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
this.queryPlus = queryPlus;
this.responseContext = responseContext;
@@ -357,8 +358,8 @@ public class CachingClusteredClient implements QuerySegmentWalker
// Which is not necessarily an indication that the data doesn't exist or is
// incomplete. The data could exist and just not be loaded yet. In either
// case, though, this query will not include any data from the identified intervals.
- responseContext.put("uncoveredIntervals", uncoveredIntervals);
- responseContext.put("uncoveredIntervalsOverflowed", uncoveredIntervalsOverflowed);
+ responseContext.put(ResponseContext.CTX_UNCOVERED_INTERVALS, uncoveredIntervals);
+ responseContext.put(ResponseContext.CTX_UNCOVERED_INTERVALS_OVERFLOWED, uncoveredIntervalsOverflowed);
}
}
@@ -395,7 +396,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
hasher.putBytes(queryCacheKey == null ? strategy.computeCacheKey(query) : queryCacheKey);
String currEtag = StringUtils.encodeBase64String(hasher.hash().asBytes());
- responseContext.put(QueryResource.HEADER_ETAG, currEtag);
+ responseContext.put(ResponseContext.CTX_ETAG, currEtag);
return currEtag;
} else {
return null;
diff --git a/server/src/main/java/org/apache/druid/client/CachingQueryRunner.java b/server/src/main/java/org/apache/druid/client/CachingQueryRunner.java
index 0334620..1301139 100644
--- a/server/src/main/java/org/apache/druid/client/CachingQueryRunner.java
+++ b/server/src/main/java/org/apache/druid/client/CachingQueryRunner.java
@@ -34,11 +34,11 @@ import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.SegmentDescriptor;
+import org.apache.druid.query.context.ResponseContext;
import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
-import java.util.Map;
public class CachingQueryRunner<T> implements QueryRunner<T>
{
@@ -73,7 +73,7 @@ public class CachingQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
Query<T> query = queryPlus.getQuery();
final CacheStrategy strategy = toolChest.getCacheStrategy(query);
diff --git a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java
index 0db3335..9880240 100644
--- a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java
+++ b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java
@@ -33,7 +33,6 @@ import org.apache.druid.java.util.common.guava.BaseSequence;
import org.apache.druid.java.util.common.guava.CloseQuietly;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
-import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.http.client.HttpClient;
@@ -51,6 +50,9 @@ import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.aggregation.MetricManipulatorFns;
+import org.apache.druid.query.context.ConcurrentResponseContext;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.server.QueryResource;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;
import org.jboss.netty.handler.codec.http.HttpChunk;
@@ -65,10 +67,7 @@ import java.io.InputStream;
import java.io.SequenceInputStream;
import java.net.URL;
import java.util.Enumeration;
-import java.util.Map;
import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
@@ -83,7 +82,6 @@ import java.util.concurrent.atomic.AtomicReference;
public class DirectDruidClient<T> implements QueryRunner<T>
{
public static final String QUERY_FAIL_TIME = "queryFailTime";
- public static final String QUERY_TOTAL_BYTES_GATHERED = "queryTotalBytesGathered";
private static final Logger log = new Logger(DirectDruidClient.class);
@@ -101,15 +99,15 @@ public class DirectDruidClient<T> implements QueryRunner<T>
/**
* Removes the magical fields added by {@link #makeResponseContextForQuery()}.
*/
- public static void removeMagicResponseContextFields(Map<String, Object> responseContext)
+ public static void removeMagicResponseContextFields(ResponseContext responseContext)
{
- responseContext.remove(DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED);
+ responseContext.remove(ResponseContext.CTX_QUERY_TOTAL_BYTES_GATHERED);
}
- public static ConcurrentMap<String, Object> makeResponseContextForQuery()
+ public static ResponseContext makeResponseContextForQuery()
{
- final ConcurrentMap<String, Object> responseContext = new ConcurrentHashMap<>();
- responseContext.put(DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED, new AtomicLong());
+ final ResponseContext responseContext = ConcurrentResponseContext.createEmpty();
+ responseContext.put(ResponseContext.CTX_QUERY_TOTAL_BYTES_GATHERED, new AtomicLong());
return responseContext;
}
@@ -141,7 +139,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> context)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
{
final Query<T> query = queryPlus.getQuery();
QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
@@ -158,7 +156,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
final long requestStartTimeNs = System.nanoTime();
final long timeoutAt = query.getContextValue(QUERY_FAIL_TIME);
final long maxScatterGatherBytes = QueryContexts.getMaxScatterGatherBytes(query);
- final AtomicLong totalBytesGathered = (AtomicLong) context.get(QUERY_TOTAL_BYTES_GATHERED);
+ final AtomicLong totalBytesGathered = (AtomicLong) context.get(ResponseContext.CTX_QUERY_TOTAL_BYTES_GATHERED);
final long maxQueuedBytes = QueryContexts.getMaxQueuedBytes(query, 0);
final boolean usingBackpressure = maxQueuedBytes > 0;
@@ -229,15 +227,10 @@ public class DirectDruidClient<T> implements QueryRunner<T>
final boolean continueReading;
try {
- final String responseContext = response.headers().get("X-Druid-Response-Context");
+ final String responseContext = response.headers().get(QueryResource.HEADER_RESPONSE_CONTEXT);
// context may be null in case of error or query timeout
if (responseContext != null) {
- context.putAll(
- objectMapper.<Map<String, Object>>readValue(
- responseContext,
- JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
- )
- );
+ context.putAll(ResponseContext.deserialize(responseContext, objectMapper));
}
continueReading = enqueue(response.getContent(), 0L);
}
diff --git a/server/src/main/java/org/apache/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/org/apache/druid/query/ResultLevelCachingQueryRunner.java
index 6a9a640..addad8d 100644
--- a/server/src/main/java/org/apache/druid/query/ResultLevelCachingQueryRunner.java
+++ b/server/src/main/java/org/apache/druid/query/ResultLevelCachingQueryRunner.java
@@ -34,6 +34,7 @@ import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.SequenceWrapper;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.server.QueryResource;
import javax.annotation.Nullable;
@@ -41,7 +42,6 @@ import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
-import java.util.Map;
public class ResultLevelCachingQueryRunner<T> implements QueryRunner<T>
{
@@ -76,7 +76,7 @@ public class ResultLevelCachingQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(QueryPlus queryPlus, Map responseContext)
+ public Sequence<T> run(QueryPlus queryPlus, ResponseContext responseContext)
{
if (useResultCache || populateResultCache) {
@@ -92,7 +92,7 @@ public class ResultLevelCachingQueryRunner<T> implements QueryRunner<T>
QueryPlus.wrap(query),
responseContext
);
- String newResultSetId = (String) responseContext.get(QueryResource.HEADER_ETAG);
+ String newResultSetId = (String) responseContext.get(ResponseContext.CTX_ETAG);
if (useResultCache && newResultSetId != null && newResultSetId.equals(existingResultSetId)) {
log.debug("Return cached result set as there is no change in identifiers for query %s ", query.getId());
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java
index 57920a5..2c07d4d 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java
@@ -42,6 +42,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.incremental.IncrementalIndexAddResult;
import org.apache.druid.segment.incremental.IndexSizeExceededException;
import org.apache.druid.segment.indexing.DataSchema;
@@ -172,7 +173,7 @@ public class AppenderatorPlumber implements Plumber
return new QueryRunner<T>()
{
@Override
- public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> responseContext)
+ public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
return queryPlus.run(appenderator, responseContext);
}
diff --git a/server/src/main/java/org/apache/druid/server/BrokerQueryResource.java b/server/src/main/java/org/apache/druid/server/BrokerQueryResource.java
index e3354d3..b75ce5d 100644
--- a/server/src/main/java/org/apache/druid/server/BrokerQueryResource.java
+++ b/server/src/main/java/org/apache/druid/server/BrokerQueryResource.java
@@ -89,10 +89,11 @@ public class BrokerQueryResource extends QueryResource
@Context final HttpServletRequest req
) throws IOException
{
- final ResponseContext context = createContext(req.getContentType(), pretty != null);
+ final ResourceIOReaderWriter ioReaderWriter =
+ createResourceIOReaderWriter(req.getContentType(), pretty != null);
try {
- Query<?> query = context.getObjectMapper().readValue(in, Query.class);
- return context.ok(
+ Query<?> query = ioReaderWriter.getInputMapper().readValue(in, Query.class);
+ return ioReaderWriter.ok(
ServerViewUtil.getTargetLocations(
brokerServerView,
query.getDataSource(),
@@ -102,7 +103,7 @@ public class BrokerQueryResource extends QueryResource
);
}
catch (Exception e) {
- return context.gotError(e);
+ return ioReaderWriter.gotError(e);
}
}
}
diff --git a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java
index fe42ad0..243036f 100644
--- a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java
+++ b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java
@@ -40,6 +40,7 @@ import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.server.log.RequestLogger;
import org.apache.druid.server.security.Access;
import org.apache.druid.server.security.AuthenticationResult;
@@ -51,7 +52,6 @@ import javax.servlet.http.HttpServletRequest;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.UUID;
-import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
/**
@@ -249,7 +249,7 @@ public class QueryLifecycle
{
transition(State.AUTHORIZED, State.EXECUTING);
- final ConcurrentMap<String, Object> responseContext = DirectDruidClient.makeResponseContextForQuery();
+ final ResponseContext responseContext = DirectDruidClient.makeResponseContextForQuery();
final Sequence res = QueryPlus.wrap(baseQuery)
.withIdentity(authenticationResult.getIdentity())
@@ -368,9 +368,9 @@ public class QueryLifecycle
public static class QueryResponse
{
private final Sequence results;
- private final Map<String, Object> responseContext;
+ private final ResponseContext responseContext;
- private QueryResponse(final Sequence results, final Map<String, Object> responseContext)
+ private QueryResponse(final Sequence results, final ResponseContext responseContext)
{
this.results = results;
this.responseContext = responseContext;
@@ -381,7 +381,7 @@ public class QueryLifecycle
return results;
}
- public Map<String, Object> getResponseContext()
+ public ResponseContext getResponseContext()
{
return responseContext;
}
diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java
index ec772c5..1512486 100644
--- a/server/src/main/java/org/apache/druid/server/QueryResource.java
+++ b/server/src/main/java/org/apache/druid/server/QueryResource.java
@@ -42,6 +42,7 @@ import org.apache.druid.query.GenericQueryMetricsFactory;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryInterruptedException;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.server.metrics.QueryCountStatsProvider;
import org.apache.druid.server.security.Access;
import org.apache.druid.server.security.AuthConfig;
@@ -66,7 +67,6 @@ import javax.ws.rs.core.StreamingOutput;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
-import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.atomic.AtomicLong;
@@ -81,8 +81,15 @@ public class QueryResource implements QueryCountStatsProvider
@Deprecated // use SmileMediaTypes.APPLICATION_JACKSON_SMILE
protected static final String APPLICATION_SMILE = "application/smile";
+ /**
+ * The maximum length of {@link ResponseContext} serialized string that might be put into an HTTP response header
+ */
protected static final int RESPONSE_CTX_HEADER_LEN_LIMIT = 7 * 1024;
+ /**
+ * HTTP response header name containing {@link ResponseContext} serialized string
+ */
+ public static final String HEADER_RESPONSE_CONTEXT = "X-Druid-Response-Context";
public static final String HEADER_IF_NONE_MATCH = "If-None-Match";
public static final String HEADER_ETAG = "ETag";
@@ -168,11 +175,11 @@ public class QueryResource implements QueryCountStatsProvider
acceptHeader = req.getContentType();
}
- final ResponseContext context = createContext(acceptHeader, pretty != null);
+ final ResourceIOReaderWriter ioReaderWriter = createResourceIOReaderWriter(acceptHeader, pretty != null);
final String currThreadName = Thread.currentThread().getName();
try {
- queryLifecycle.initialize(readQuery(req, in, context));
+ queryLifecycle.initialize(readQuery(req, in));
query = queryLifecycle.getQuery();
final String queryId = query.getId();
@@ -189,10 +196,10 @@ public class QueryResource implements QueryCountStatsProvider
final QueryLifecycle.QueryResponse queryResponse = queryLifecycle.execute();
final Sequence<?> results = queryResponse.getResults();
- final Map<String, Object> responseContext = queryResponse.getResponseContext();
+ final ResponseContext responseContext = queryResponse.getResponseContext();
final String prevEtag = getPreviousEtag(req);
- if (prevEtag != null && prevEtag.equals(responseContext.get(HEADER_ETAG))) {
+ if (prevEtag != null && prevEtag.equals(responseContext.get(ResponseContext.CTX_ETAG))) {
queryLifecycle.emitLogsAndMetrics(null, req.getRemoteAddr(), -1);
successfulQueryCount.incrementAndGet();
return Response.notModified().build();
@@ -205,7 +212,7 @@ public class QueryResource implements QueryCountStatsProvider
boolean serializeDateTimeAsLong =
QueryContexts.isSerializeDateTimeAsLong(query, false)
|| (!shouldFinalize && QueryContexts.isSerializeDateTimeAsLongInner(query, false));
- final ObjectWriter jsonWriter = context.newOutputWriter(serializeDateTimeAsLong);
+ final ObjectWriter jsonWriter = ioReaderWriter.newOutputWriter(serializeDateTimeAsLong);
Response.ResponseBuilder builder = Response
.ok(
new StreamingOutput()
@@ -241,13 +248,13 @@ public class QueryResource implements QueryCountStatsProvider
}
}
},
- context.getContentType()
+ ioReaderWriter.getContentType()
)
.header("X-Druid-Query-Id", queryId);
- if (responseContext.get(HEADER_ETAG) != null) {
- builder.header(HEADER_ETAG, responseContext.get(HEADER_ETAG));
- responseContext.remove(HEADER_ETAG);
+ if (responseContext.get(ResponseContext.CTX_ETAG) != null) {
+ builder.header(HEADER_ETAG, responseContext.get(ResponseContext.CTX_ETAG));
+ responseContext.remove(ResponseContext.CTX_ETAG);
}
DirectDruidClient.removeMagicResponseContextFields(responseContext);
@@ -255,14 +262,14 @@ public class QueryResource implements QueryCountStatsProvider
//Limit the response-context header, see https://github.com/apache/incubator-druid/issues/2331
//Note that Response.ResponseBuilder.header(String key,Object value).build() calls value.toString()
//and encodes the string using ASCII, so 1 char is = 1 byte
- String responseCtxString = jsonMapper.writeValueAsString(responseContext);
+ String responseCtxString = responseContext.serializeWith(jsonMapper);
if (responseCtxString.length() > RESPONSE_CTX_HEADER_LEN_LIMIT) {
log.warn("Response Context truncated for id [%s] . Full context is [%s].", queryId, responseCtxString);
responseCtxString = responseCtxString.substring(0, RESPONSE_CTX_HEADER_LEN_LIMIT);
}
return builder
- .header("X-Druid-Response-Context", responseCtxString)
+ .header(HEADER_RESPONSE_CONTEXT, responseCtxString)
.build();
}
catch (Exception e) {
@@ -278,7 +285,7 @@ public class QueryResource implements QueryCountStatsProvider
catch (QueryInterruptedException e) {
interruptedQueryCount.incrementAndGet();
queryLifecycle.emitLogsAndMetrics(e, req.getRemoteAddr(), -1);
- return context.gotError(e);
+ return ioReaderWriter.gotError(e);
}
catch (ForbiddenException e) {
// don't do anything for an authorization failure, ForbiddenExceptionMapper will catch this later and
@@ -295,7 +302,7 @@ public class QueryResource implements QueryCountStatsProvider
.addData("peer", req.getRemoteAddr())
.emit();
- return context.gotError(e);
+ return ioReaderWriter.gotError(e);
}
finally {
Thread.currentThread().setName(currThreadName);
@@ -304,8 +311,7 @@ public class QueryResource implements QueryCountStatsProvider
private Query<?> readQuery(
final HttpServletRequest req,
- final InputStream in,
- final ResponseContext context
+ final InputStream in
) throws IOException
{
Query baseQuery = getMapperForRequest(req.getContentType()).readValue(in, Query.class);
@@ -337,12 +343,12 @@ public class QueryResource implements QueryCountStatsProvider
return mapper.copy().registerModule(new SimpleModule().addSerializer(DateTime.class, new DateTimeSerializer()));
}
- protected ResponseContext createContext(String requestType, boolean pretty)
+ protected ResourceIOReaderWriter createResourceIOReaderWriter(String requestType, boolean pretty)
{
boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(requestType) ||
APPLICATION_SMILE.equals(requestType);
String contentType = isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON;
- return new ResponseContext(
+ return new ResourceIOReaderWriter(
contentType,
isSmile ? smileMapper : jsonMapper,
isSmile ? serializeDateTimeAsLongSmileMapper : serializeDateTimeAsLongJsonMapper,
@@ -350,14 +356,14 @@ public class QueryResource implements QueryCountStatsProvider
);
}
- protected static class ResponseContext
+ protected static class ResourceIOReaderWriter
{
private final String contentType;
private final ObjectMapper inputMapper;
private final ObjectMapper serializeDateTimeAsLongInputMapper;
private final boolean isPretty;
- ResponseContext(
+ ResourceIOReaderWriter(
String contentType,
ObjectMapper inputMapper,
ObjectMapper serializeDateTimeAsLongInputMapper,
@@ -375,7 +381,7 @@ public class QueryResource implements QueryCountStatsProvider
return contentType;
}
- public ObjectMapper getObjectMapper()
+ ObjectMapper getInputMapper()
{
return inputMapper;
}
diff --git a/server/src/main/java/org/apache/druid/server/SetAndVerifyContextQueryRunner.java b/server/src/main/java/org/apache/druid/server/SetAndVerifyContextQueryRunner.java
index 0bddccc..aa27a85 100644
--- a/server/src/main/java/org/apache/druid/server/SetAndVerifyContextQueryRunner.java
+++ b/server/src/main/java/org/apache/druid/server/SetAndVerifyContextQueryRunner.java
@@ -26,10 +26,9 @@ import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.server.initialization.ServerConfig;
-import java.util.Map;
-
/**
* Use this QueryRunner to set and verify Query contexts.
*/
@@ -47,7 +46,7 @@ public class SetAndVerifyContextQueryRunner<T> implements QueryRunner<T>
}
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
return baseRunner.run(
QueryPlus.wrap(withTimeoutAndMaxScatterGatherBytes(queryPlus.getQuery(), serverConfig)),
diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java
index d05cfc8..b261442 100644
--- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java
+++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java
@@ -47,6 +47,7 @@ import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.select.SelectQueryConfig;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.timeline.DataSegment;
@@ -64,9 +65,7 @@ import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Collections;
import java.util.Comparator;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map;
import java.util.Set;
import java.util.concurrent.Executor;
@@ -124,52 +123,52 @@ public class CachingClusteredClientFunctionalityTest
3
));
- Map<String, Object> responseContext = new HashMap<>();
+ ResponseContext responseContext = ResponseContext.createEmpty();
runQuery(client, builder.build(), responseContext);
Assert.assertNull(responseContext.get("uncoveredIntervals"));
builder.intervals("2015-01-01/2015-01-03");
- responseContext = new HashMap<>();
+ responseContext = ResponseContext.createEmpty();
runQuery(client, builder.build(), responseContext);
assertUncovered(responseContext, false, "2015-01-01/2015-01-02");
builder.intervals("2015-01-01/2015-01-04");
- responseContext = new HashMap<>();
+ responseContext = ResponseContext.createEmpty();
runQuery(client, builder.build(), responseContext);
assertUncovered(responseContext, false, "2015-01-01/2015-01-02", "2015-01-03/2015-01-04");
builder.intervals("2015-01-02/2015-01-04");
- responseContext = new HashMap<>();
+ responseContext = ResponseContext.createEmpty();
runQuery(client, builder.build(), responseContext);
assertUncovered(responseContext, false, "2015-01-03/2015-01-04");
builder.intervals("2015-01-01/2015-01-30");
- responseContext = new HashMap<>();
+ responseContext = ResponseContext.createEmpty();
runQuery(client, builder.build(), responseContext);
assertUncovered(responseContext, false, "2015-01-01/2015-01-02", "2015-01-03/2015-01-04", "2015-01-05/2015-01-30");
builder.intervals("2015-01-02/2015-01-30");
- responseContext = new HashMap<>();
+ responseContext = ResponseContext.createEmpty();
runQuery(client, builder.build(), responseContext);
assertUncovered(responseContext, false, "2015-01-03/2015-01-04", "2015-01-05/2015-01-30");
builder.intervals("2015-01-04/2015-01-30");
- responseContext = new HashMap<>();
+ responseContext = ResponseContext.createEmpty();
runQuery(client, builder.build(), responseContext);
assertUncovered(responseContext, false, "2015-01-05/2015-01-30");
builder.intervals("2015-01-10/2015-01-30");
- responseContext = new HashMap<>();
+ responseContext = ResponseContext.createEmpty();
runQuery(client, builder.build(), responseContext);
assertUncovered(responseContext, false, "2015-01-10/2015-01-30");
builder.intervals("2015-01-01/2015-02-25");
- responseContext = new HashMap<>();
+ responseContext = ResponseContext.createEmpty();
runQuery(client, builder.build(), responseContext);
assertUncovered(responseContext, true, "2015-01-01/2015-01-02", "2015-01-03/2015-01-04", "2015-01-05/2015-02-04");
}
- private void assertUncovered(Map<String, Object> context, boolean uncoveredIntervalsOverflowed, String... intervals)
+ private void assertUncovered(ResponseContext context, boolean uncoveredIntervalsOverflowed, String... intervals)
{
List<Interval> expectedList = Lists.newArrayListWithExpectedSize(intervals.length);
for (String interval : intervals) {
@@ -321,7 +320,7 @@ public class CachingClusteredClientFunctionalityTest
private static <T> Sequence<T> runQuery(
CachingClusteredClient client,
final Query<T> query,
- final Map<String, Object> responseContext
+ final ResponseContext responseContext
)
{
return client.getQueryRunnerForIntervals(query, query.getIntervals()).run(
diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java
index 8974cc1..389b49f 100644
--- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java
+++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java
@@ -91,6 +91,7 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFact
import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
import org.apache.druid.query.aggregation.post.ConstantPostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.filter.AndDimFilter;
import org.apache.druid.query.filter.BoundDimFilter;
@@ -154,7 +155,6 @@ import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@@ -513,7 +513,6 @@ public class CachingClusteredClientTest
);
- HashMap<String, List> context = new HashMap<String, List>();
TimeseriesQuery query = builder.intervals("2011-01-01/2011-01-10")
.aggregators(RENAMED_AGGS)
.postAggregators(RENAMED_POST_AGGS)
@@ -534,7 +533,7 @@ public class CachingClusteredClientTest
DateTimes.of("2011-01-09"), 18, 521,
DateTimes.of("2011-01-09T01"), 181, 52
),
- runner.run(QueryPlus.wrap(query), context)
+ runner.run(QueryPlus.wrap(query))
);
}
@@ -555,7 +554,7 @@ public class CachingClusteredClientTest
.context(CONTEXT)
.build();
- final Map<String, Object> context = new HashMap<>();
+ final ResponseContext context = ResponseContext.createEmpty();
final Cache cache = EasyMock.createStrictMock(Cache.class);
final Capture<Iterable<Cache.NamedKey>> cacheKeyCapture = EasyMock.newCapture();
EasyMock.expect(cache.getBulk(EasyMock.capture(cacheKeyCapture)))
@@ -651,7 +650,7 @@ public class CachingClusteredClientTest
DateTimes.of("2011-01-09T00"), 18, 521,
DateTimes.of("2011-01-09T02"), 181, 52
),
- runner.run(QueryPlus.wrap(query), new HashMap<>())
+ runner.run(QueryPlus.wrap(query))
);
}
@@ -685,7 +684,6 @@ public class CachingClusteredClientTest
new DateTime("2011-11-07", TIMEZONE), 85, 102
)
);
- HashMap<String, List> context = new HashMap<String, List>();
TimeseriesQuery query = builder
.intervals("2011-11-04/2011-11-08")
.aggregators(RENAMED_AGGS)
@@ -698,7 +696,7 @@ public class CachingClusteredClientTest
new DateTime("2011-11-06", TIMEZONE), 23, 85312,
new DateTime("2011-11-07", TIMEZONE), 85, 102
),
- runner.run(QueryPlus.wrap(query), context)
+ runner.run(QueryPlus.wrap(query))
);
}
@@ -822,7 +820,6 @@ public class CachingClusteredClientTest
DateTimes.of("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983
)
);
- HashMap<String, List> context = new HashMap<String, List>();
TopNQuery query = builder
.intervals("2011-01-01/2011-01-10")
.metric("imps")
@@ -844,7 +841,7 @@ public class CachingClusteredClientTest
DateTimes.of("2011-01-09"), "c1", 50, 4985, "b", 50, 4984, "c", 50, 4983,
DateTimes.of("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983
),
- runner.run(QueryPlus.wrap(query), context)
+ runner.run(QueryPlus.wrap(query))
);
}
@@ -883,7 +880,6 @@ public class CachingClusteredClientTest
new DateTime("2011-11-07", TIMEZONE), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986
)
);
- HashMap<String, List> context = new HashMap<String, List>();
TopNQuery query = builder
.intervals("2011-11-04/2011-11-08")
.metric("imps")
@@ -898,7 +894,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(QueryPlus.wrap(query), context)
+ runner.run(QueryPlus.wrap(query))
);
}
@@ -1004,7 +1000,6 @@ public class CachingClusteredClientTest
)
);
- HashMap<String, List> context = new HashMap<String, List>();
TopNQuery query = builder
.intervals("2011-01-01/2011-01-10")
.metric("imps")
@@ -1024,7 +1019,7 @@ public class CachingClusteredClientTest
DateTimes.of("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983,
DateTimes.of("2011-01-09T01"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983
),
- runner.run(QueryPlus.wrap(query), context)
+ runner.run(QueryPlus.wrap(query))
);
}
@@ -1077,7 +1072,6 @@ public class CachingClusteredClientTest
)
);
- HashMap<String, List> context = new HashMap<String, List>();
TopNQuery query = builder
.intervals("2011-01-01/2011-01-10")
.metric("avg_imps_per_row_double")
@@ -1097,7 +1091,7 @@ public class CachingClusteredClientTest
DateTimes.of("2011-01-09"), "c1", 50, 4985, "b", 50, 4984, "c", 50, 4983,
DateTimes.of("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983
),
- runner.run(QueryPlus.wrap(query), context)
+ runner.run(QueryPlus.wrap(query))
);
}
@@ -1150,7 +1144,6 @@ public class CachingClusteredClientTest
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
);
- HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(
makeSearchResults(
TOP_DIM,
@@ -1167,7 +1160,7 @@ public class CachingClusteredClientTest
DateTimes.of("2011-01-09"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4,
DateTimes.of("2011-01-09T01"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4
),
- runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context)
+ runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()))
);
}
@@ -1220,7 +1213,7 @@ public class CachingClusteredClientTest
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
);
- HashMap<String, Object> context = new HashMap<String, Object>();
+ ResponseContext context = ResponseContext.createEmpty();
TestHelper.assertExpectedResults(
makeSearchResults(
TOP_DIM,
@@ -1318,7 +1311,6 @@ public class CachingClusteredClientTest
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
);
- HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(
makeSelectResults(
dimensions,
@@ -1336,7 +1328,7 @@ public class CachingClusteredClientTest
DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9),
DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9)
),
- runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context)
+ runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()))
);
}
@@ -1395,7 +1387,7 @@ public class CachingClusteredClientTest
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
);
- HashMap<String, Object> context = new HashMap<String, Object>();
+ ResponseContext context = ResponseContext.createEmpty();
TestHelper.assertExpectedResults(
makeSelectResults(
dimensions,
@@ -1514,7 +1506,6 @@ public class CachingClusteredClientTest
getDefaultQueryRunner(),
WAREHOUSE.getToolChest(query)
);
- HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedObjects(
makeGroupByResults(
DateTimes.of("2011-01-05T"),
@@ -1538,7 +1529,7 @@ public class CachingClusteredClientTest
DateTimes.of("2011-01-09T01"),
ImmutableMap.of("a", "g", "rows", 7, "imps", 7, "impers", 7, "uniques", collector)
),
- runner.run(QueryPlus.wrap(builder.setInterval("2011-01-05/2011-01-10").build()), context),
+ runner.run(QueryPlus.wrap(builder.setInterval("2011-01-05/2011-01-10").build())),
""
);
}
@@ -1701,7 +1692,6 @@ public class CachingClusteredClientTest
.postAggregators(RENAMED_POST_AGGS);
TimeseriesQuery query = builder.build();
- Map<String, Object> context = new HashMap<>();
final Interval interval1 = Intervals.of("2011-01-06/2011-01-07");
final Interval interval2 = Intervals.of("2011-01-07/2011-01-08");
@@ -1729,7 +1719,7 @@ public class CachingClusteredClientTest
timeline.add(interval3, "v", new StringPartitionChunk<>(null, null, 6, selector6));
final Capture<QueryPlus> capture = Capture.newInstance();
- final Capture<Map<String, Object>> contextCap = Capture.newInstance();
+ final Capture<ResponseContext> contextCap = Capture.newInstance();
QueryRunner mockRunner = EasyMock.createNiceMock(QueryRunner.class);
EasyMock.expect(mockRunner.run(EasyMock.capture(capture), EasyMock.capture(contextCap)))
@@ -1748,7 +1738,7 @@ public class CachingClusteredClientTest
descriptors.add(new SegmentDescriptor(interval3, "v", 6));
MultipleSpecificSegmentSpec expected = new MultipleSpecificSegmentSpec(descriptors);
- runner.run(QueryPlus.wrap(query), context).toList();
+ runner.run(QueryPlus.wrap(query)).toList();
Assert.assertEquals(expected, ((TimeseriesQuery) capture.getValue().getQuery()).getQuerySegmentSpec());
}
@@ -1873,7 +1863,7 @@ public class CachingClusteredClientTest
.times(0, 1);
final Capture<? extends QueryPlus> capture = Capture.newInstance();
- final Capture<? extends Map> context = Capture.newInstance();
+ final Capture<? extends ResponseContext> context = Capture.newInstance();
QueryRunner queryable = expectations.getQueryRunner();
if (query instanceof TimeseriesQuery) {
@@ -1914,7 +1904,6 @@ public class CachingClusteredClientTest
@Override
public void run()
{
- HashMap<String, Object> context = new HashMap<>();
for (int i = 0; i < numTimesToQuery; ++i) {
TestHelper.assertExpectedResults(
expected,
@@ -1927,8 +1916,7 @@ public class CachingClusteredClientTest
)
)
)
- ),
- context
+ )
)
);
if (queryCompletedCallback != null) {
@@ -2022,7 +2010,7 @@ public class CachingClusteredClientTest
.once();
final Capture<? extends QueryPlus> capture = Capture.newInstance();
- final Capture<? extends Map> context = Capture.newInstance();
+ final Capture<? extends ResponseContext> context = Capture.newInstance();
queryCaptures.add(capture);
QueryRunner queryable = expectations.getQueryRunner();
@@ -2120,7 +2108,6 @@ public class CachingClusteredClientTest
@Override
public void run()
{
- HashMap<String, List> context = new HashMap<String, List>();
for (int i = 0; i < numTimesToQuery; ++i) {
TestHelper.assertExpectedResults(
new MergeIterable<>(
@@ -2152,8 +2139,7 @@ public class CachingClusteredClientTest
query.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(ImmutableList.of(actualQueryInterval))
)
- ),
- context
+ )
)
);
if (queryCompletedCallback != null) {
@@ -3051,7 +3037,7 @@ public class CachingClusteredClientTest
getDefaultQueryRunner(),
WAREHOUSE.getToolChest(query1)
);
- HashMap<String, Object> context = new HashMap<String, Object>();
+ ResponseContext context = ResponseContext.createEmpty();
TestHelper.assertExpectedObjects(
makeGroupByResults(
DateTimes.of("2011-01-05T"), ImmutableMap.of("output", "c", "rows", 3, "imps", 3, "impers", 3),
@@ -3123,10 +3109,10 @@ public class CachingClusteredClientTest
.build();
- Map<String, Object> responseContext = new HashMap<>();
+ ResponseContext responseContext = ResponseContext.createEmpty();
getDefaultQueryRunner().run(QueryPlus.wrap(query), responseContext);
- Assert.assertEquals("MDs2yIUvYLVzaG6zmwTH1plqaYE=", responseContext.get("ETag"));
+ Assert.assertEquals("MDs2yIUvYLVzaG6zmwTH1plqaYE=", responseContext.get(ResponseContext.CTX_ETAG));
}
@Test
@@ -3169,7 +3155,7 @@ public class CachingClusteredClientTest
.build();
- final Map<String, Object> responseContext = new HashMap<>();
+ final ResponseContext responseContext = ResponseContext.createEmpty();
getDefaultQueryRunner().run(QueryPlus.wrap(query), responseContext);
final Object etag1 = responseContext.get("ETag");
@@ -3184,7 +3170,7 @@ public class CachingClusteredClientTest
return new QueryRunner()
{
@Override
- public Sequence run(final QueryPlus queryPlus, final Map responseContext)
+ public Sequence run(final QueryPlus queryPlus, final ResponseContext responseContext)
{
return client.getQueryRunnerForIntervals(queryPlus.getQuery(), queryPlus.getQuery().getIntervals())
.run(queryPlus, responseContext);
diff --git a/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java
index 630321a..b350d68 100644
--- a/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java
+++ b/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java
@@ -56,6 +56,7 @@ import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
import org.apache.druid.query.timeseries.TimeseriesResultValue;
@@ -74,7 +75,6 @@ import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
@@ -290,7 +290,7 @@ public class CachingQueryRunnerTest
new QueryRunner()
{
@Override
- public Sequence run(QueryPlus queryPlus, Map responseContext)
+ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext)
{
return resultSeq;
}
@@ -319,8 +319,7 @@ public class CachingQueryRunnerTest
cacheStrategy.computeCacheKey(query)
);
- HashMap<String, Object> context = new HashMap<String, Object>();
- Sequence res = runner.run(QueryPlus.wrap(query), context);
+ Sequence res = runner.run(QueryPlus.wrap(query));
// base sequence is not closed yet
Assert.assertFalse("sequence must not be closed", closable.isClosed());
Assert.assertNull("cache must be empty", cache.get(cacheKey));
@@ -378,7 +377,7 @@ public class CachingQueryRunnerTest
new QueryRunner()
{
@Override
- public Sequence run(QueryPlus queryPlus, Map responseContext)
+ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext)
{
return Sequences.empty();
}
@@ -400,8 +399,7 @@ public class CachingQueryRunnerTest
}
);
- HashMap<String, Object> context = new HashMap<String, Object>();
- List<Result> results = runner.run(QueryPlus.wrap(query), context).toList();
+ List<Result> results = runner.run(QueryPlus.wrap(query)).toList();
Assert.assertEquals(expectedResults.toString(), results.toString());
}
diff --git a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java
index 65a8cd6..e50fe6d 100644
--- a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java
+++ b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java
@@ -62,20 +62,9 @@ import java.net.URL;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
public class DirectDruidClientTest
{
- private final Map<String, Object> defaultContext;
-
- public DirectDruidClientTest()
- {
- defaultContext = new HashMap<>();
- defaultContext.put(DirectDruidClient.QUERY_FAIL_TIME, Long.MAX_VALUE);
- defaultContext.put(DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED, new AtomicLong());
- }
-
@Test
public void testRun() throws Exception
{
@@ -164,22 +153,22 @@ public class DirectDruidClientTest
TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build();
query = query.withOverriddenContext(ImmutableMap.of(DirectDruidClient.QUERY_FAIL_TIME, Long.MAX_VALUE));
- Sequence s1 = client1.run(QueryPlus.wrap(query), defaultContext);
+ Sequence s1 = client1.run(QueryPlus.wrap(query));
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
- client1.run(QueryPlus.wrap(query), defaultContext);
+ client1.run(QueryPlus.wrap(query));
Assert.assertEquals(2, client1.getNumOpenConnections());
futureException.setException(new ReadTimeoutException());
Assert.assertEquals(1, client1.getNumOpenConnections());
// subsequent connections should work
- client1.run(QueryPlus.wrap(query), defaultContext);
- client1.run(QueryPlus.wrap(query), defaultContext);
- client1.run(QueryPlus.wrap(query), defaultContext);
+ client1.run(QueryPlus.wrap(query));
+ client1.run(QueryPlus.wrap(query));
+ client1.run(QueryPlus.wrap(query));
Assert.assertTrue(client1.getNumOpenConnections() == 4);
@@ -194,8 +183,8 @@ public class DirectDruidClientTest
Assert.assertEquals(DateTimes.of("2014-01-01T01:02:03Z"), results.get(0).getTimestamp());
Assert.assertEquals(3, client1.getNumOpenConnections());
- client2.run(QueryPlus.wrap(query), defaultContext);
- client2.run(QueryPlus.wrap(query), defaultContext);
+ client2.run(QueryPlus.wrap(query));
+ client2.run(QueryPlus.wrap(query));
Assert.assertTrue(client2.getNumOpenConnections() == 2);
@@ -269,7 +258,7 @@ public class DirectDruidClientTest
TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build();
query = query.withOverriddenContext(ImmutableMap.of(DirectDruidClient.QUERY_FAIL_TIME, Long.MAX_VALUE));
cancellationFuture.set(new StatusResponseHolder(HttpResponseStatus.OK, new StringBuilder("cancelled")));
- Sequence results = client1.run(QueryPlus.wrap(query), defaultContext);
+ Sequence results = client1.run(QueryPlus.wrap(query));
Assert.assertEquals(HttpMethod.DELETE, capturedRequest.getValue().getMethod());
Assert.assertEquals(0, client1.getNumOpenConnections());
@@ -345,7 +334,7 @@ public class DirectDruidClientTest
StringUtils.toUtf8("{\"error\":\"testing1\",\"errorMessage\":\"testing2\"}")
)
);
- Sequence results = client1.run(QueryPlus.wrap(query), defaultContext);
+ Sequence results = client1.run(QueryPlus.wrap(query));
QueryInterruptedException actualException = null;
try {
diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTest.java
index e9a168d..6590e2c 100644
--- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTest.java
+++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTest.java
@@ -37,6 +37,7 @@ import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.Result;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.scan.ScanQuery;
import org.apache.druid.query.scan.ScanResultValue;
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
@@ -51,7 +52,6 @@ import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
@@ -508,7 +508,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results1 =
- QueryPlus.wrap(query1).run(appenderator, ImmutableMap.of()).toList();
+ QueryPlus.wrap(query1).run(appenderator, ResponseContext.createEmpty()).toList();
Assert.assertEquals(
"query1",
ImmutableList.of(
@@ -534,7 +534,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results2 =
- QueryPlus.wrap(query2).run(appenderator, ImmutableMap.of()).toList();
+ QueryPlus.wrap(query2).run(appenderator, ResponseContext.createEmpty()).toList();
Assert.assertEquals(
"query2",
ImmutableList.of(
@@ -564,7 +564,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results3 =
- QueryPlus.wrap(query3).run(appenderator, ImmutableMap.of()).toList();
+ QueryPlus.wrap(query3).run(appenderator, ResponseContext.createEmpty()).toList();
Assert.assertEquals(
ImmutableList.of(
new Result<>(
@@ -598,7 +598,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results4 =
- QueryPlus.wrap(query4).run(appenderator, ImmutableMap.of()).toList();
+ QueryPlus.wrap(query4).run(appenderator, ResponseContext.createEmpty()).toList();
Assert.assertEquals(
ImmutableList.of(
new Result<>(
@@ -654,7 +654,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results1 =
- QueryPlus.wrap(query1).run(appenderator, ImmutableMap.of()).toList();
+ QueryPlus.wrap(query1).run(appenderator, ResponseContext.createEmpty()).toList();
Assert.assertEquals(
"query1",
ImmutableList.of(
@@ -690,7 +690,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results2 =
- QueryPlus.wrap(query2).run(appenderator, ImmutableMap.of()).toList();
+ QueryPlus.wrap(query2).run(appenderator, ResponseContext.createEmpty()).toList();
Assert.assertEquals(
"query2",
ImmutableList.of(
@@ -731,7 +731,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results3 =
- QueryPlus.wrap(query3).run(appenderator, ImmutableMap.of()).toList();
+ QueryPlus.wrap(query3).run(appenderator, ResponseContext.createEmpty()).toList();
Assert.assertEquals(
"query3",
ImmutableList.of(
@@ -766,7 +766,7 @@ public class AppenderatorTest
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build();
final List<ScanResultValue> results4 =
- QueryPlus.wrap(query4).run(appenderator, new HashMap<>()).toList();
+ QueryPlus.wrap(query4).run(appenderator, ResponseContext.createEmpty()).toList();
Assert.assertEquals(2, results4.size()); // 2 segments, 1 row per segment
Assert.assertArrayEquals(new String[]{"__time", "dim", "count", "met"}, results4.get(0).getColumns().toArray());
Assert.assertArrayEquals(
diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java
index 07cbba4..d1820cd 100644
--- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java
+++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java
@@ -40,6 +40,7 @@ import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.Result;
import org.apache.druid.query.SegmentDescriptor;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.timeboundary.TimeBoundaryResultValue;
import org.apache.druid.server.log.TestRequestLogger;
import org.apache.druid.server.metrics.NoopServiceEmitter;
@@ -69,7 +70,6 @@ import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.List;
-import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executors;
@@ -91,7 +91,7 @@ public class QueryResourceTest
return new QueryRunner<T>()
{
@Override
- public Sequence<T> run(QueryPlus<T> query, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> query, ResponseContext responseContext)
{
return Sequences.empty();
}
diff --git a/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java
index 14ec19c..636e69e 100644
--- a/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java
@@ -54,6 +54,7 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.MetricManipulationFn;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.search.SearchQuery;
import org.apache.druid.query.search.SearchResultValue;
import org.apache.druid.segment.AbstractSegment;
@@ -80,10 +81,8 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
-import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@@ -446,8 +445,7 @@ public class ServerManagerTest
@Override
public void run()
{
- Map<String, Object> context = new HashMap<String, Object>();
- Sequence<Result<SearchResultValue>> seq = runner.run(QueryPlus.wrap(query), context);
+ Sequence<Result<SearchResultValue>> seq = runner.run(QueryPlus.wrap(query));
seq.toList();
Iterator<SegmentForTesting> adaptersIter = factory.getAdapters().iterator();
@@ -684,7 +682,7 @@ public class ServerManagerTest
}
@Override
- public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
+ public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
return new BlockingSequence<>(runner.run(queryPlus, responseContext), waitLatch, waitYieldLatch, notifyLatch);
}
diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java
index 97008eb..9ba2819 100644
--- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java
+++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java
@@ -58,6 +58,7 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.metadata.metadata.ListColumnIncluderator;
import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
@@ -90,7 +91,6 @@ import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.EnumSet;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -485,7 +485,7 @@ public class DumpSegment extends GuiceRunnable
return factory
.getToolchest()
.mergeResults(factory.mergeRunners(Execs.directExecutor(), ImmutableList.of(runner)))
- .run(QueryPlus.wrap(query), new HashMap<>());
+ .run(QueryPlus.wrap(query), ResponseContext.createEmpty());
}
private static <T> void evaluateSequenceForSideEffects(final Sequence<T> sequence)
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org