remove select query (#8739)

* remove select query

* thanks teamcity

* oops

* oops

* add back a SelectQuery class that throws RuntimeExceptions linking to docs

* adjust text

* update docs per review

* deprecated
This commit is contained in:
Clint Wylie 2019-10-30 19:29:56 -07:00 committed by Gian Merlino
parent 2363b61983
commit 3ff5e02237
56 changed files with 181 additions and 6228 deletions

View File

@ -1,385 +0,0 @@
/*
* 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.benchmark.query;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.io.Files;
import org.apache.commons.io.FileUtils;
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo;
import org.apache.druid.benchmark.datagen.BenchmarkSchemas;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.Row;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.Druids;
import org.apache.druid.query.FinalizeResultsQueryRunner;
import org.apache.druid.query.Query;
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.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;
import org.apache.druid.query.select.SelectQuery;
import org.apache.druid.query.select.SelectQueryConfig;
import org.apache.druid.query.select.SelectQueryEngine;
import org.apache.druid.query.select.SelectQueryQueryToolChest;
import org.apache.druid.query.select.SelectQueryRunnerFactory;
import org.apache.druid.query.select.SelectResultValue;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexMergerV9;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexSegment;
import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
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;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
@State(Scope.Benchmark)
@Fork(value = 1)
@Warmup(iterations = 10)
@Measurement(iterations = 25)
public class SelectBenchmark
{
@Param({"1"})
private int numSegments;
@Param({"25000"})
private int rowsPerSegment;
@Param({"basic.A"})
private String schemaAndQuery;
@Param({"1000"})
private int pagingThreshold;
private static final Logger log = new Logger(SelectBenchmark.class);
private static final int RNG_SEED = 9999;
private static final IndexMergerV9 INDEX_MERGER_V9;
private static final IndexIO INDEX_IO;
public static final ObjectMapper JSON_MAPPER;
private List<IncrementalIndex> incIndexes;
private List<QueryableIndex> qIndexes;
private QueryRunnerFactory factory;
private BenchmarkSchemaInfo schemaInfo;
private Druids.SelectQueryBuilder queryBuilder;
private SelectQuery query;
private File tmpDir;
private ExecutorService executorService;
static {
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
new ColumnConfig()
{
@Override
public int columnCacheSizeBytes()
{
return 0;
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
}
private static final Map<String, Map<String, Druids.SelectQueryBuilder>> SCHEMA_QUERY_MAP = new LinkedHashMap<>();
private void setupQueries()
{
// queries for the basic schema
Map<String, Druids.SelectQueryBuilder> basicQueries = new LinkedHashMap<>();
BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic");
{ // basic.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
Druids.SelectQueryBuilder queryBuilderA =
Druids.newSelectQueryBuilder()
.dataSource(new TableDataSource("blah"))
.dimensionSpecs(DefaultDimensionSpec.toSpec(Collections.emptyList()))
.metrics(Collections.emptyList())
.intervals(intervalSpec)
.granularity(Granularities.ALL)
.descending(false);
basicQueries.put("A", queryBuilderA);
}
SCHEMA_QUERY_MAP.put("basic", basicQueries);
}
@Setup
public void setup() throws IOException
{
log.info("SETUP CALLED AT " + System.currentTimeMillis());
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde());
executorService = Execs.multiThreaded(numSegments, "SelectThreadPool");
setupQueries();
String[] schemaQuery = schemaAndQuery.split("\\.");
String schemaName = schemaQuery[0];
String queryName = schemaQuery[1];
schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName);
queryBuilder = SCHEMA_QUERY_MAP.get(schemaName).get(queryName);
queryBuilder.pagingSpec(PagingSpec.newSpec(pagingThreshold));
query = queryBuilder.build();
incIndexes = new ArrayList<>();
for (int i = 0; i < numSegments; i++) {
BenchmarkDataGenerator gen = new BenchmarkDataGenerator(
schemaInfo.getColumnSchemas(),
RNG_SEED + i,
schemaInfo.getDataInterval(),
rowsPerSegment
);
IncrementalIndex incIndex = makeIncIndex();
for (int j = 0; j < rowsPerSegment; j++) {
InputRow row = gen.nextRow();
if (j % 10000 == 0) {
log.info(j + " rows generated.");
}
incIndex.add(row);
}
incIndexes.add(incIndex);
}
tmpDir = Files.createTempDir();
log.info("Using temp dir: " + tmpDir.getAbsolutePath());
qIndexes = new ArrayList<>();
for (int i = 0; i < numSegments; i++) {
File indexFile = INDEX_MERGER_V9.persist(
incIndexes.get(i),
tmpDir,
new IndexSpec(),
null
);
QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile);
qIndexes.add(qIndex);
}
final Supplier<SelectQueryConfig> selectConfigSupplier = Suppliers.ofInstance(new SelectQueryConfig(true));
factory = new SelectQueryRunnerFactory(
new SelectQueryQueryToolChest(
JSON_MAPPER,
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
),
new SelectQueryEngine(),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
);
}
@TearDown
public void tearDown() throws IOException
{
FileUtils.deleteDirectory(tmpDir);
}
private IncrementalIndex makeIncIndex()
{
return new IncrementalIndex.Builder()
.setSimpleTestingIndexSchema(schemaInfo.getAggsArray())
.setReportParseExceptions(false)
.setMaxRowCount(rowsPerSegment)
.buildOnheap();
}
private static <T> List<T> runQuery(QueryRunnerFactory factory, QueryRunner runner, Query<T> query)
{
QueryToolChest toolChest = factory.getToolchest();
QueryRunner<T> theRunner = new FinalizeResultsQueryRunner<>(
toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)),
toolChest
);
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty());
return queryResult.toList();
}
/**
* Don't run this benchmark with a query that doesn't use {@link Granularities#ALL},
* this pagination function probably doesn't work correctly in that case.
*/
private SelectQuery incrementQueryPagination(SelectQuery query, SelectResultValue prevResult)
{
Map<String, Integer> pagingIdentifiers = prevResult.getPagingIdentifiers();
Map<String, Integer> newPagingIdentifers = new HashMap<>();
for (String segmentId : pagingIdentifiers.keySet()) {
int newOffset = pagingIdentifiers.get(segmentId) + 1;
newPagingIdentifers.put(segmentId, newOffset);
}
return query.withPagingSpec(new PagingSpec(newPagingIdentifers, pagingThreshold));
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void queryIncrementalIndex(Blackhole blackhole)
{
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
SegmentId segmentId = SegmentId.dummy("incIndex");
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
segmentId,
new IncrementalIndexSegment(incIndexes.get(0), segmentId)
);
boolean done = false;
while (!done) {
List<Result<SelectResultValue>> results = SelectBenchmark.runQuery(factory, runner, queryCopy);
SelectResultValue result = results.get(0).getValue();
if (result.getEvents().size() == 0) {
done = true;
} else {
for (EventHolder eh : result.getEvents()) {
blackhole.consume(eh);
}
queryCopy = incrementQueryPagination(queryCopy, result);
}
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void queryQueryableIndex(Blackhole blackhole)
{
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
SegmentId segmentId = SegmentId.dummy("qIndex");
QueryRunner<Result<SelectResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
segmentId,
new QueryableIndexSegment(qIndexes.get(0), segmentId)
);
boolean done = false;
while (!done) {
List<Result<SelectResultValue>> results = SelectBenchmark.runQuery(factory, runner, queryCopy);
SelectResultValue result = results.get(0).getValue();
if (result.getEvents().size() == 0) {
done = true;
} else {
blackhole.consume(result);
queryCopy = incrementQueryPagination(queryCopy, result);
}
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void queryMultiQueryableIndex(Blackhole blackhole)
{
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
List<QueryRunner<Result<SelectResultValue>>> singleSegmentRunners = new ArrayList<>();
QueryToolChest toolChest = factory.getToolchest();
for (int i = 0; i < numSegments; i++) {
SegmentId segmentId = SegmentId.dummy("qIndex" + i);
QueryRunner<Result<SelectResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
segmentId,
new QueryableIndexSegment(qIndexes.get(i), segmentId)
);
singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner));
}
QueryRunner theRunner = toolChest.postMergeQueryDecoration(
new FinalizeResultsQueryRunner<>(
toolChest.mergeResults(factory.mergeRunners(executorService, singleSegmentRunners)),
toolChest
)
);
boolean done = false;
while (!done) {
Sequence<Result<SelectResultValue>> queryResult = theRunner.run(QueryPlus.wrap(queryCopy), ResponseContext.createEmpty());
List<Result<SelectResultValue>> results = queryResult.toList();
SelectResultValue result = results.get(0).getValue();
if (result.getEvents().size() == 0) {
done = true;
} else {
blackhole.consume(result);
queryCopy = incrementQueryPagination(queryCopy, result);
}
}
}
}

View File

@ -41,7 +41,7 @@ public class TimestampSpec
DateTime lastDateTime = null;
}
private static final String DEFAULT_COLUMN = "timestamp";
public static final String DEFAULT_COLUMN = "timestamp";
private static final String DEFAULT_FORMAT = "auto";
private static final DateTime DEFAULT_MISSING_VALUE = null;

View File

@ -1174,7 +1174,7 @@ You can optionally configure caching to be enabled on the peons by setting cachi
|--------|---------------|-----------|-------|
|`druid.realtime.cache.useCache`|true, false|Enable the cache on the realtime.|false|
|`druid.realtime.cache.populateCache`|true, false|Populate the cache on the realtime.|false|
|`druid.realtime.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`|
|`druid.realtime.cache.unCacheable`|All druid query types|All query types to not cache.|`[]`|
|`druid.realtime.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000|
See [cache configuration](#cache-configuration) for how to configure cache settings.
@ -1308,7 +1308,7 @@ You can optionally configure caching to be enabled on the Indexer by setting cac
|--------|---------------|-----------|-------|
|`druid.realtime.cache.useCache`|true, false|Enable the cache on the realtime.|false|
|`druid.realtime.cache.populateCache`|true, false|Populate the cache on the realtime.|false|
|`druid.realtime.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`|
|`druid.realtime.cache.unCacheable`|All druid query types|All query types to not cache.|`[]`|
|`druid.realtime.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000|
See [cache configuration](#cache-configuration) for how to configure cache settings.
@ -1404,7 +1404,7 @@ You can optionally only configure caching to be enabled on the Historical by set
|--------|---------------|-----------|-------|
|`druid.historical.cache.useCache`|true, false|Enable the cache on the Historical.|false|
|`druid.historical.cache.populateCache`|true, false|Populate the cache on the Historical.|false|
|`druid.historical.cache.unCacheable`|All druid query types|All query types to not cache.|["groupBy", "select"]|
|`druid.historical.cache.unCacheable`|All druid query types|All query types to not cache.|`[]`|
|`druid.historical.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000|
See [cache configuration](#cache-configuration) for how to configure cache settings.
@ -1541,7 +1541,7 @@ You can optionally only configure caching to be enabled on the Broker by setting
|`druid.broker.cache.useResultLevelCache`|true, false|Enable result level caching on the Broker.|false|
|`druid.broker.cache.populateResultLevelCache`|true, false|Populate the result level cache on the Broker.|false|
|`druid.broker.cache.resultLevelCacheLimit`|positive integer|Maximum size of query response that can be cached.|`Integer.MAX_VALUE`|
|`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`|
|`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|`[]`|
|`druid.broker.cache.cacheBulkMergeLimit`|positive integer or 0|Queries with more segments than this number will not attempt to fetch from cache at the broker level, leaving potential caching fetches (and cache result merging) to the Historicals|`Integer.MAX_VALUE`|
|`druid.broker.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000|

View File

@ -24,10 +24,7 @@ sidebar_label: "Scan"
-->
The Scan query returns raw Apache Druid (incubating) rows in streaming mode. The biggest difference between the Select query and the Scan
query is that the Scan query does not retain all the returned rows in memory before they are returned to the client.
The Select query _will_ retain the rows in memory, causing memory pressure if too many rows are returned.
The Scan query can return all the rows without issuing another pagination query.
The Scan query returns raw Apache Druid (incubating) rows in streaming mode.
In addition to straightforward usage where a Scan query is issued to the Broker, the Scan query can also be issued
directly to Historical processes or streaming ingestion tasks. This can be useful if you want to retrieve large

View File

@ -23,236 +23,7 @@ sidebar_label: "Select"
~ under the License.
-->
Older versions of Apache Druid (incubating) included a Select query type. Since Druid 0.17.0, it has been removed and replaced by the [Scan query](../querying/scan-query.md), which offers improved memory usage and performance. This solves issues that users had with Select queries causing Druid to run out of memory or slow down.
> We encourage you to use the [Scan query](../querying/scan-query.md) type rather than Select whenever possible.
> In situations involving larger numbers of segments, the Select query can have very high memory and performance overhead.
> The Scan query does not have this issue.
> The major difference between the two is that the Scan query does not support pagination.
> However, the Scan query type is able to return a virtually unlimited number of results even without pagination, making it unnecessary in many cases.
Select queries return raw Druid rows and support pagination.
```json
{
"queryType": "select",
"dataSource": "wikipedia",
"descending": "false",
"dimensions":[],
"metrics":[],
"granularity": "all",
"intervals": [
"2013-01-01/2013-01-02"
],
"pagingSpec":{"pagingIdentifiers": {}, "threshold":5}
}
```
There are several main parts to a select query:
|property|description|required?|
|--------|-----------|---------|
|queryType|This String should always be "select"; this is the first thing Druid looks at to figure out how to interpret the query|yes|
|dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../querying/datasource.md) for more information.|yes|
|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes|
|descending|Whether to make descending ordered result. Default is `false`(ascending). When this is `true`, page identifier and offsets will be negative value.|no|
|filter|See [Filters](../querying/filters.md)|no|
|dimensions|A JSON list of dimensions to select; or see [DimensionSpec](../querying/dimensionspecs.md) for ways to extract dimensions. If left empty, all dimensions are returned.|no|
|metrics|A String array of metrics to select. If left empty, all metrics are returned.|no|
|granularity|Defines the granularity of the query. See [Granularities](../querying/granularities.md). Default is `Granularity.ALL`.|no|
|pagingSpec|A JSON object indicating offsets into different scanned segments. Query results will return a `pagingIdentifiers` value that can be reused in the next query for pagination.|yes|
|context|An additional JSON Object which can be used to specify certain flags.|no|
The format of the result is:
```json
[{
"timestamp" : "2013-01-01T00:00:00.000Z",
"result" : {
"pagingIdentifiers" : {
"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9" : 4
},
"events" : [ {
"segmentId" : "wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9",
"offset" : 0,
"event" : {
"timestamp" : "2013-01-01T00:00:00.000Z",
"robot" : "1",
"namespace" : "article",
"anonymous" : "0",
"unpatrolled" : "0",
"page" : "11._korpus_(NOVJ)",
"language" : "sl",
"newpage" : "0",
"user" : "EmausBot",
"count" : 1.0,
"added" : 39.0,
"delta" : 39.0,
"variation" : 39.0,
"deleted" : 0.0
}
}, {
"segmentId" : "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9",
"offset" : 1,
"event" : {
"timestamp" : "2013-01-01T00:00:00.000Z",
"robot" : "0",
"namespace" : "article",
"anonymous" : "0",
"unpatrolled" : "0",
"page" : "112_U.S._580",
"language" : "en",
"newpage" : "1",
"user" : "MZMcBride",
"count" : 1.0,
"added" : 70.0,
"delta" : 70.0,
"variation" : 70.0,
"deleted" : 0.0
}
}, {
"segmentId" : "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9",
"offset" : 2,
"event" : {
"timestamp" : "2013-01-01T00:00:00.000Z",
"robot" : "0",
"namespace" : "article",
"anonymous" : "0",
"unpatrolled" : "0",
"page" : "113_U.S._243",
"language" : "en",
"newpage" : "1",
"user" : "MZMcBride",
"count" : 1.0,
"added" : 77.0,
"delta" : 77.0,
"variation" : 77.0,
"deleted" : 0.0
}
}, {
"segmentId" : "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9",
"offset" : 3,
"event" : {
"timestamp" : "2013-01-01T00:00:00.000Z",
"robot" : "0",
"namespace" : "article",
"anonymous" : "0",
"unpatrolled" : "0",
"page" : "113_U.S._73",
"language" : "en",
"newpage" : "1",
"user" : "MZMcBride",
"count" : 1.0,
"added" : 70.0,
"delta" : 70.0,
"variation" : 70.0,
"deleted" : 0.0
}
}, {
"segmentId" : "wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9",
"offset" : 4,
"event" : {
"timestamp" : "2013-01-01T00:00:00.000Z",
"robot" : "0",
"namespace" : "article",
"anonymous" : "0",
"unpatrolled" : "0",
"page" : "113_U.S._756",
"language" : "en",
"newpage" : "1",
"user" : "MZMcBride",
"count" : 1.0,
"added" : 68.0,
"delta" : 68.0,
"variation" : 68.0,
"deleted" : 0.0
}
} ]
}
} ]
```
The `threshold` determines how many hits are returned, with each hit indexed by an offset. When `descending` is true, the offset will be negative value.
The results above include:
```json
"pagingIdentifiers" : {
"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9" : 4
},
```
### Result pagination
The PagingSpec allows the user to specify that the results of a select query should be returned as a paginated set.
The `threshold` option controls how many rows are returned in each block of paginated results.
To initiate a paginated query, the user should specify a PagingSpec with a `threshold` set and a blank `pagingIdentifiers` field, e.g.:
```json
"pagingSpec":{"pagingIdentifiers": {}, "threshold":5}
```
When the query returns, the results will contain a `pagingIndentifers` field indicating the current pagination point in the result set (an identifier and an offset).
To retrieve the next part of the result set, the user should issue the same select query, but replace the `pagingIdentifiers` field of the query with the `pagingIdentifiers` from the previous result.
When an empty result set is received, all rows have been returned.
#### fromNext backwards compatibility
In older versions of Druid, when using paginated select queries, it was necessary for the user to manually increment the paging offset by 1 in each `pagingIdentifiers` before submitting the next query to retrieve the next set of results. This offset increment happens automatically in the current version of Druid by default, the user does not need to modify the `pagingIdentifiers` offset to retrieve the next result set.
Setting the `fromNext` field of the PagingSpec to false instructs Druid to operate in the older mode where the user must manually increment the offset (or decrement for descending queries).
For example, suppose the user issues the following initial paginated query, with `fromNext` false:
```json
{
"queryType": "select",
"dataSource": "wikipedia",
"descending": "false",
"dimensions":[],
"metrics":[],
"granularity": "all",
"intervals": [
"2013-01-01/2013-01-02"
],
"pagingSpec":{"fromNext": "false", "pagingIdentifiers": {}, "threshold":5}
}
```
The paginated query with `fromNext` set to false returns a result set with the following `pagingIdentifiers`:
```json
"pagingIdentifiers" : {
"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9" : 4
},
```
To retrieve the next result set, the next query must be sent with the paging offset (4) incremented by 1.
```json
{
"queryType": "select",
"dataSource": "wikipedia",
"dimensions":[],
"metrics":[],
"granularity": "all",
"intervals": [
"2013-01-01/2013-01-02"
],
"pagingSpec":{"fromNext": "false", "pagingIdentifiers": {"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9" : 5}, "threshold":5}
}
```
Note that specifying the `fromNext` option in the `pagingSpec` overrides the default value set by `druid.query.select.enableFromNextDefault` in the server configuration. See [Server configuration](#server-configuration) for more details.
### Server configuration
The following runtime properties apply to select queries:
|Property|Description|Default|
|--------|-----------|-------|
|`druid.query.select.enableFromNextDefault`|If the `fromNext` property in a query's `pagingSpec` is left unspecified, the system will use the value of this property as the default value for `fromNext`. This option is true by default: the option of setting `fromNext` to false by default is intended to support backwards compatibility for deployments where some users may still expect behavior from older versions of Druid.|true|
The Scan query has a different syntax, but supports many of the features of the Select query, including time ordering and limiting. Scan does not include the Select query's pagination feature; however, in many cases pagination is unnecessary with Scan due to its ability to return a virtually unlimited number of results in one call.

View File

@ -1,157 +0,0 @@
/*
* 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.aggregation;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.io.Resources;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.Result;
import org.apache.druid.query.select.SelectResultValue;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.io.File;
import java.nio.charset.StandardCharsets;
import java.sql.Timestamp;
import java.util.List;
import java.util.zip.ZipFile;
@RunWith(Parameterized.class)
public class TimestampAggregationSelectTest
{
private AggregationTestHelper helper;
@Rule
public final TemporaryFolder temporaryFolder = new TemporaryFolder();
private ColumnSelectorFactory selectorFactory;
private TestObjectColumnSelector selector;
private Timestamp[] values = new Timestamp[10];
@Parameterized.Parameters(name = "{index}: Test for {0}")
public static Iterable<Object[]> constructorFeeder()
{
return Iterables.transform(
ImmutableList.of(
ImmutableList.of("timeMin", "tmin", DateTimes.of("2011-01-12T01:00:00.000Z").getMillis()),
ImmutableList.of("timeMax", "tmax", DateTimes.of("2011-01-31T01:00:00.000Z").getMillis())
),
new Function<List<?>, Object[]>()
{
@Nullable
@Override
public Object[] apply(List<?> input)
{
return input.toArray();
}
}
);
}
private String aggType;
private String aggField;
private Long expected;
public TimestampAggregationSelectTest(String aggType, String aggField, Long expected)
{
this.aggType = aggType;
this.aggField = aggField;
this.expected = expected;
}
@Before
public void setup()
{
helper = AggregationTestHelper.createSelectQueryAggregationTestHelper(
new TimestampMinMaxModule().getJacksonModules(),
temporaryFolder
);
selector = new TestObjectColumnSelector<>(values);
selectorFactory = EasyMock.createMock(ColumnSelectorFactory.class);
EasyMock.expect(selectorFactory.makeColumnValueSelector("test")).andReturn(selector);
EasyMock.replay(selectorFactory);
}
@Test
public void testSimpleDataIngestionAndSelectTest() throws Exception
{
String recordParser = "{\n" +
" \"type\": \"string\",\n" +
" \"parseSpec\": {\n" +
" \"format\": \"tsv\",\n" +
" \"timestampSpec\": {\n" +
" \"column\": \"timestamp\",\n" +
" \"format\": \"auto\"\n" +
" },\n" +
" \"dimensionsSpec\": {\n" +
" \"dimensions\": [\n" +
" \"product\"\n" +
" ],\n" +
" \"dimensionExclusions\": [],\n" +
" \"spatialDimensions\": []\n" +
" },\n" +
" \"columns\": [\n" +
" \"timestamp\",\n" +
" \"cat\",\n" +
" \"product\",\n" +
" \"prefer\",\n" +
" \"prefer2\",\n" +
" \"pty_country\"\n" +
" ]\n" +
" }\n" +
"}";
String aggregator = "[\n" +
" {\n" +
" \"type\": \"" + aggType + "\",\n" +
" \"name\": \"" + aggField + "\",\n" +
" \"fieldName\": \"timestamp\"\n" +
" }\n" +
"]";
ZipFile zip = new ZipFile(new File(this.getClass().getClassLoader().getResource("druid.sample.tsv.zip").toURI()));
Sequence<?> seq = helper.createIndexAndRunQueryOnSegment(
zip.getInputStream(zip.getEntry("druid.sample.tsv")),
recordParser,
aggregator,
0,
Granularities.MONTH,
100,
Resources.toString(Resources.getResource("select.json"), StandardCharsets.UTF_8)
);
Result<SelectResultValue> result = (Result<SelectResultValue>) Iterables.getOnlyElement(seq.toList());
Assert.assertEquals(36, result.getValue().getEvents().size());
Assert.assertEquals(expected, result.getValue().getEvents().get(0).getEvent().get(aggField));
}
}

View File

@ -1,11 +0,0 @@
{
"queryType": "select",
"dataSource": "test_datasource",
"dimensions":[],
"metrics":[],
"granularity": "ALL",
"intervals": [
"2011-01-01T00:00:00.000Z/2011-05-01T00:00:00.000Z"
],
"pagingSpec":{"pagingIdentifiers": {}, "threshold":100}
}

View File

@ -1,204 +0,0 @@
/*
* 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.segment;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.io.CharSource;
import org.apache.druid.data.input.impl.DelimitedParseSpec;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.query.Druids;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.Result;
import org.apache.druid.query.select.EventHolder;
import org.apache.druid.query.select.PagingSpec;
import org.apache.druid.query.select.SelectQuery;
import org.apache.druid.query.select.SelectQueryConfig;
import org.apache.druid.query.select.SelectQueryEngine;
import org.apache.druid.query.select.SelectQueryQueryToolChest;
import org.apache.druid.query.select.SelectQueryRunnerFactory;
import org.apache.druid.query.select.SelectResultValue;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.timeline.SegmentId;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
/**
*/
@RunWith(Parameterized.class)
public class MapVirtualColumnSelectTest
{
@Parameterized.Parameters
public static Iterable<Object[]> constructorFeeder() throws IOException
{
final Supplier<SelectQueryConfig> selectConfigSupplier = Suppliers.ofInstance(new SelectQueryConfig(true));
SelectQueryRunnerFactory factory = new SelectQueryRunnerFactory(
new SelectQueryQueryToolChest(
new DefaultObjectMapper(),
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
new SelectQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
.withMinTimestamp(DateTimes.of("2011-01-12T00:00:00.000Z").getMillis())
.build();
final IncrementalIndex index = new IncrementalIndex.Builder()
.setIndexSchema(schema)
.setMaxRowCount(10000)
.buildOnheap();
final StringInputRowParser parser = new StringInputRowParser(
new DelimitedParseSpec(
new TimestampSpec("ts", "iso", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("dim", "keys", "values")), null, null),
"\t",
",",
Arrays.asList("ts", "dim", "keys", "values"),
false,
0
),
"utf8"
);
CharSource input = CharSource.wrap(
"2011-01-12T00:00:00.000Z\ta\tkey1,key2,key3\tvalue1,value2,value3\n" +
"2011-01-12T00:00:00.000Z\tb\tkey4,key5,key6\tvalue4\n" +
"2011-01-12T00:00:00.000Z\tc\tkey1,key5\tvalue1,value5,value9\n"
);
IncrementalIndex index1 = TestIndex.loadIncrementalIndex(() -> index, input, parser);
QueryableIndex index2 = TestIndex.persistRealtimeAndLoadMMapped(index1);
return QueryRunnerTestHelper.transformToConstructionFeeder(
Arrays.asList(
QueryRunnerTestHelper.makeQueryRunner(
factory,
SegmentId.dummy("index1"),
new IncrementalIndexSegment(index1, SegmentId.dummy("index1")),
"incremental"
),
QueryRunnerTestHelper.makeQueryRunner(
factory,
SegmentId.dummy("index2"),
new QueryableIndexSegment(index2, SegmentId.dummy("index2")),
"queryable"
)
)
);
}
private final QueryRunner runner;
public MapVirtualColumnSelectTest(QueryRunner runner)
{
this.runner = runner;
}
private Druids.SelectQueryBuilder testBuilder()
{
return Druids.newSelectQueryBuilder()
.dataSource(QueryRunnerTestHelper.DATA_SOURCE)
.granularity(QueryRunnerTestHelper.ALL_GRAN)
.intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC)
.pagingSpec(new PagingSpec(null, 3));
}
@Test
public void testSerde() throws IOException
{
final ObjectMapper mapper = new DefaultObjectMapper();
new DruidVirtualColumnsModule().getJacksonModules().forEach(mapper::registerModule);
final MapVirtualColumn column = new MapVirtualColumn("keys", "values", "params");
final String json = mapper.writeValueAsString(column);
final VirtualColumn fromJson = mapper.readValue(json, VirtualColumn.class);
Assert.assertEquals(column, fromJson);
}
@Test
public void testBasic()
{
Druids.SelectQueryBuilder builder = testBuilder();
List<Map> expectedResults = Arrays.asList(
MapVirtualColumnTestBase.mapOf(
"dim", "a",
"params.key1", "value1",
"params.key3", "value3",
"params.key5", null,
"params", MapVirtualColumnTestBase.mapOf("key1", "value1", "key2", "value2", "key3", "value3")
),
MapVirtualColumnTestBase.mapOf(
"dim", "b",
"params.key1", null,
"params.key3", null,
"params.key5", null,
"params", MapVirtualColumnTestBase.mapOf("key4", "value4")
),
MapVirtualColumnTestBase.mapOf(
"dim", "c",
"params.key1", "value1",
"params.key3", null,
"params.key5", "value5",
"params", MapVirtualColumnTestBase.mapOf("key1", "value1", "key5", "value5")
)
);
List<VirtualColumn> virtualColumns = Collections.singletonList(new MapVirtualColumn("keys", "values", "params"));
SelectQuery selectQuery = builder.dimensions(Collections.singletonList("dim"))
.metrics(Arrays.asList("params.key1", "params.key3", "params.key5", "params"))
.virtualColumns(virtualColumns)
.build();
checkSelectQuery(selectQuery, expectedResults);
}
private void checkSelectQuery(SelectQuery searchQuery, List<Map> expected)
{
List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(searchQuery)).toList();
Assert.assertEquals(1, results.size());
List<EventHolder> events = results.get(0).getValue().getEvents();
Assert.assertEquals(expected.size(), events.size());
for (int i = 0; i < events.size(); i++) {
Map event = events.get(i).getEvent();
event.remove(EventHolder.TIMESTAMP_KEY);
Assert.assertEquals(expected.get(i), event);
}
}
}

View File

@ -34,7 +34,6 @@ import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
import org.apache.druid.query.groupby.ResultRow;
import org.apache.druid.query.select.SelectResultValue;
import org.apache.druid.query.timeseries.TimeseriesResultValue;
import org.apache.druid.query.topn.DimensionAndMetricValueExtractor;
import org.apache.druid.query.topn.TopNResultValue;
@ -268,30 +267,6 @@ public class SketchAggregationWithSimpleDataTest
Assert.assertEquals("product_3", value.getDimensionValue("product"));
}
@Test
public void testSimpleDataIngestAndSelectQuery() throws Exception
{
SketchModule.registerSerde();
SketchModule sm = new SketchModule();
AggregationTestHelper selectQueryAggregationTestHelper = AggregationTestHelper.createSelectQueryAggregationTestHelper(
sm.getJacksonModules(),
tempFolder
);
Sequence seq = selectQueryAggregationTestHelper.runQueryOnSegments(
ImmutableList.of(s1, s2),
readFileFromClasspathAsString("select_query.json")
);
Result<SelectResultValue> result = (Result<SelectResultValue>) Iterables.getOnlyElement(seq.toList());
Assert.assertEquals(DateTimes.of("2014-10-20T00:00:00.000Z"), result.getTimestamp());
Assert.assertEquals(100, result.getValue().getEvents().size());
Assert.assertEquals(
"AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=",
result.getValue().getEvents().get(0).getEvent().get("pty_country")
);
}
@Test
public void testTopNQueryWithSketchConstant() throws Exception
{

View File

@ -1,11 +0,0 @@
{
"queryType": "select",
"dataSource": "test_datasource",
"dimensions":[],
"metrics":[],
"granularity": "ALL",
"intervals": [
"2014-10-19T00:00:00.000Z/2014-10-22T00:00:00.000Z"
],
"pagingSpec":{"pagingIdentifiers": {}, "threshold":100}
}

View File

@ -1,19 +0,0 @@
{
"queryType": "select",
"intervals": ["2013-08-31/2013-09-01"],
"dataSource": "wikipedia_index_test",
"granularity": "all",
"filter": {
"type": "selector",
"dimension": "language",
"value": "en"
},
"pagingSpec": {
"threshold": 10
},
"context": {
"useCache": "false",
"populateCache": "false",
"timeout": 60000
}
}

View File

@ -1,19 +0,0 @@
{
"queryType": "select",
"intervals": ["2013-08-31/2013-09-01"],
"dataSource": "wikipedia_reindex_test",
"granularity": "all",
"filter": {
"type": "selector",
"dimension": "language",
"value": "en"
},
"pagingSpec": {
"threshold": 10
},
"context": {
"useCache": "false",
"populateCache": "false",
"timeout": 60000
}
}

View File

@ -1,121 +0,0 @@
{
"query": {
"queryType": "select",
"intervals": ["2013-08-31/2013-09-01"],
"dataSource": {
"type": "union",
"dataSources": [
"wikipedia_index_test"
]
},
"granularity": "all",
"filter": {
"type": "selector",
"dimension": "language",
"value": "en"
},
"pagingSpec": {
"threshold": 10
},
"context": {
"useCache": "false",
"populateCache": "false",
"timeout": 60000
}
},
"expectedResults": [
{
"timestamp": "2013-08-31T01:02:33.000Z",
"result": {
"pagingIdentifiers": {
"wikipedia_index_test0_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:43.993Z": 0,
"wikipedia_index_test1_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.108Z": 0,
"wikipedia_index_test2_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.236Z": 0,
"wikipedia_index_test3_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.374Z": 0
},
"events": [
{
"segmentId": "wikipedia_index_test0_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:43.993Z",
"offset": 0,
"event": {
"timestamp": "2013-08-31T01:02:33.000Z",
"page": "Gypsy Danger",
"added": 57.0,
"deleted": 200.0
}
},
{
"segmentId": "wikipedia_index_test1_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.108Z",
"offset": 0,
"event": {
"timestamp": "2013-08-31T01:02:33.000Z",
"page": "Gypsy Danger",
"added": 57.0,
"deleted": 200.0
}
},
{
"segmentId": "wikipedia_index_test2_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.236Z",
"offset": 0,
"event": {
"timestamp": "2013-08-31T01:02:33.000Z",
"page": "Gypsy Danger",
"added": 57.0,
"deleted": 200.0
}
},
{
"segmentId": "wikipedia_index_test3_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.374Z",
"offset": 0,
"event": {
"timestamp": "2013-08-31T01:02:33.000Z",
"page": "Gypsy Danger",
"added": 57.0,
"deleted": 200.0
}
},
{
"segmentId": "wikipedia_index_test0_2013-08-31T0org.apache.druid.java.util.common.ISE: one or more twitter queries failed0:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:43.993Z",
"offset": 0,
"event": {
"timestamp": "2013-08-31T03:32:45.000Z",
"page": "Striker Eureka",
"added": 459.0,
"deleted": 129.0
}
},
{
"segmentId": "wikipedia_index_test1_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.108Z",
"offset": 0,
"event": {
"timestamp": "2013-08-31T03:32:45.000Z",
"page": "Striker Eureka",
"added": 459.0,
"deleted": 129.0
}
},
{
"segmentId": "wikipedia_index_test2_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.236Z",
"offset": 0,
"event": {
"timestamp": "2013-08-31T03:32:45.000Z",
"page": "Striker Eureka",
"added": 459.0,
"deleted": 129.0
}
},
{
"segmentId": "wikipedia_index_test3_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.374Z",
"offset": 0,
"event": {
"timestamp": "2013-08-31T03:32:45.000Z",
"page": "Striker Eureka",
"added": 459.0,
"deleted": 129.0
}
}
]
}
}
]
}

View File

@ -43,8 +43,6 @@ import org.apache.druid.query.search.InsensitiveContainsSearchQuerySpec;
import org.apache.druid.query.search.SearchQuery;
import org.apache.druid.query.search.SearchQuerySpec;
import org.apache.druid.query.search.SearchSortSpec;
import org.apache.druid.query.select.PagingSpec;
import org.apache.druid.query.select.SelectQuery;
import org.apache.druid.query.spec.LegacySegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.query.timeboundary.TimeBoundaryQuery;
@ -730,173 +728,6 @@ public class Druids
return new SegmentMetadataQueryBuilder();
}
/**
* A Builder for SelectQuery.
* <p/>
* Required: dataSource(), intervals() must be called before build()
* <p/>
* Usage example:
* <pre><code>
* SelectQuery query = new SelectQueryBuilder()
* .dataSource("Example")
* .interval("2010/2013")
* .build();
* </code></pre>
*
* @see SelectQuery
*/
public static class SelectQueryBuilder
{
private DataSource dataSource;
private QuerySegmentSpec querySegmentSpec;
private boolean descending;
private Map<String, Object> context;
private DimFilter dimFilter;
private Granularity granularity;
private List<DimensionSpec> dimensions;
private List<String> metrics;
private VirtualColumns virtualColumns;
private PagingSpec pagingSpec;
public SelectQueryBuilder()
{
dataSource = null;
querySegmentSpec = null;
descending = false;
context = null;
dimFilter = null;
granularity = Granularities.ALL;
dimensions = new ArrayList<>();
metrics = new ArrayList<>();
virtualColumns = null;
pagingSpec = null;
}
public SelectQuery build()
{
return new SelectQuery(
dataSource,
querySegmentSpec,
descending,
dimFilter,
granularity,
dimensions,
metrics,
virtualColumns,
pagingSpec,
context
);
}
public static SelectQueryBuilder copy(SelectQuery query)
{
return new SelectQueryBuilder()
.dataSource(query.getDataSource())
.intervals(query.getQuerySegmentSpec())
.descending(query.isDescending())
.filters(query.getFilter())
.granularity(query.getGranularity())
.dimensionSpecs(query.getDimensions())
.metrics(query.getMetrics())
.virtualColumns(query.getVirtualColumns())
.pagingSpec(query.getPagingSpec())
.context(query.getContext());
}
public SelectQueryBuilder dataSource(String ds)
{
dataSource = new TableDataSource(ds);
return this;
}
public SelectQueryBuilder dataSource(DataSource ds)
{
dataSource = ds;
return this;
}
public SelectQueryBuilder intervals(QuerySegmentSpec q)
{
querySegmentSpec = q;
return this;
}
public SelectQueryBuilder intervals(String s)
{
querySegmentSpec = new LegacySegmentSpec(s);
return this;
}
public SelectQueryBuilder descending(boolean descending)
{
this.descending = descending;
return this;
}
public SelectQueryBuilder context(Map<String, Object> c)
{
context = c;
return this;
}
public SelectQueryBuilder filters(DimFilter f)
{
dimFilter = f;
return this;
}
public SelectQueryBuilder granularity(Granularity g)
{
granularity = g;
return this;
}
public SelectQueryBuilder dimensionSpecs(List<DimensionSpec> d)
{
dimensions = d;
return this;
}
public SelectQueryBuilder dimensions(List<String> d)
{
dimensions = DefaultDimensionSpec.toSpec(d);
return this;
}
public SelectQueryBuilder metrics(List<String> m)
{
metrics = m;
return this;
}
public SelectQueryBuilder virtualColumns(VirtualColumns vcs)
{
virtualColumns = vcs;
return this;
}
public SelectQueryBuilder virtualColumns(List<VirtualColumn> vcs)
{
return virtualColumns(VirtualColumns.create(vcs));
}
public SelectQueryBuilder virtualColumns(VirtualColumn... vcs)
{
return virtualColumns(VirtualColumns.create(Arrays.asList(vcs)));
}
public SelectQueryBuilder pagingSpec(PagingSpec p)
{
pagingSpec = p;
return this;
}
}
public static SelectQueryBuilder newSelectQueryBuilder()
{
return new SelectQueryBuilder();
}
/**
* A Builder for ScanQuery.
* <p/>

View File

@ -31,7 +31,6 @@ import org.apache.druid.segment.QueryableIndexStorageAdapter;
* @see org.apache.druid.query.search.SearchQueryConfig
* @see org.apache.druid.query.topn.TopNQueryConfig
* @see org.apache.druid.query.metadata.SegmentMetadataQueryConfig
* @see org.apache.druid.query.select.SelectQueryConfig
* @see org.apache.druid.query.scan.ScanQueryConfig
*/
public class QueryConfig

View File

@ -154,8 +154,7 @@ import java.util.List;
* dimensions than the default generic QueryMetrics. So those subinterfaces shouldn't be taken as direct examples for
* following the plan specified above.
*
* Refer {@link SearchQueryMetricsFactory}
* and {@link org.apache.druid.query.select.SelectQueryMetricsFactory} as an implementation example of this procedure.
* Refer {@link SearchQueryMetricsFactory} as an implementation example of this procedure.
*
* @param <QueryType>
*/

View File

@ -29,10 +29,7 @@ import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import java.util.Objects;
/**
*/
@ -43,18 +40,6 @@ public class DefaultDimensionSpec implements DimensionSpec
return new DefaultDimensionSpec(dimensionName, dimensionName);
}
public static List<DimensionSpec> toSpec(String... dimensionNames)
{
return toSpec(Arrays.asList(dimensionNames));
}
public static List<DimensionSpec> toSpec(Iterable<String> dimensionNames)
{
return StreamSupport.stream(dimensionNames.spliterator(), false)
.map(input -> new DefaultDimensionSpec(input, input))
.collect(Collectors.toList());
}
private static final byte CACHE_TYPE_ID = 0x0;
private final String dimension;
private final String outputName;
@ -174,13 +159,13 @@ public class DefaultDimensionSpec implements DimensionSpec
DefaultDimensionSpec that = (DefaultDimensionSpec) o;
if (dimension != null ? !dimension.equals(that.dimension) : that.dimension != null) {
if (!Objects.equals(dimension, that.dimension)) {
return false;
}
if (outputName != null ? !outputName.equals(that.outputName) : that.outputName != null) {
if (!Objects.equals(outputName, that.outputName)) {
return false;
}
if (outputType != null ? !outputType.equals(that.outputType) : that.outputType != null) {
if (!Objects.equals(outputType, that.outputType)) {
return false;
}

View File

@ -1,263 +0,0 @@
/*
* 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.select;
import org.apache.druid.collections.bitmap.BitmapFactory;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.query.BitmapResultFactory;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryMetrics;
import org.apache.druid.query.filter.Filter;
import org.joda.time.Interval;
import java.util.List;
/**
* This class is implemented with delegation to another QueryMetrics for compatibility, see "Making subinterfaces of
* QueryMetrics for emitting custom dimensions and/or metrics for specific query types" section in {@link QueryMetrics}
* javadoc.
*/
public class DefaultSelectQueryMetrics implements SelectQueryMetrics
{
private QueryMetrics<Query<?>> delegateQueryMetrics;
// queryMetrics.query(query) must already be called on the provided queryMetrics.
public DefaultSelectQueryMetrics(QueryMetrics<Query<?>> queryMetrics)
{
this.delegateQueryMetrics = queryMetrics;
}
@Override
public void query(SelectQuery query)
{
// delegateQueryMetrics.query(query) must already be called on the provided queryMetrics.
}
@Override
public void dataSource(SelectQuery query)
{
throw new ISE("Unsupported method in default query metrics implementation.");
}
@Override
public void queryType(SelectQuery query)
{
throw new ISE("Unsupported method in default query metrics implementation.");
}
@Override
public void interval(SelectQuery query)
{
throw new ISE("Unsupported method in default query metrics implementation.");
}
@Override
public void hasFilters(SelectQuery query)
{
throw new ISE("Unsupported method in default query metrics implementation.");
}
@Override
public void duration(SelectQuery query)
{
throw new ISE("Unsupported method in default query metrics implementation.");
}
@Override
public void queryId(SelectQuery query)
{
throw new ISE("Unsupported method in default query metrics implementation.");
}
@Override
public void sqlQueryId(SelectQuery query)
{
throw new ISE("Unsupported method in default query metrics implementation.");
}
@Override
public void granularity(SelectQuery query)
{
// Don't emit by default
}
@Override
public void context(SelectQuery query)
{
delegateQueryMetrics.context(query);
}
@Override
public void server(String host)
{
delegateQueryMetrics.server(host);
}
@Override
public void remoteAddress(String remoteAddress)
{
delegateQueryMetrics.remoteAddress(remoteAddress);
}
@Override
public void status(String status)
{
delegateQueryMetrics.status(status);
}
@Override
public void success(boolean success)
{
delegateQueryMetrics.success(success);
}
@Override
public void segment(String segmentIdentifier)
{
delegateQueryMetrics.segment(segmentIdentifier);
}
@Override
public void chunkInterval(Interval interval)
{
delegateQueryMetrics.chunkInterval(interval);
}
@Override
public void preFilters(List<Filter> preFilters)
{
delegateQueryMetrics.preFilters(preFilters);
}
@Override
public void postFilters(List<Filter> postFilters)
{
delegateQueryMetrics.postFilters(postFilters);
}
@Override
public void identity(String identity)
{
delegateQueryMetrics.identity(identity);
}
@Override
public void vectorized(final boolean vectorized)
{
delegateQueryMetrics.vectorized(vectorized);
}
@Override
public BitmapResultFactory<?> makeBitmapResultFactory(BitmapFactory factory)
{
return delegateQueryMetrics.makeBitmapResultFactory(factory);
}
@Override
public QueryMetrics reportQueryTime(long timeNs)
{
return delegateQueryMetrics.reportQueryTime(timeNs);
}
@Override
public QueryMetrics reportQueryBytes(long byteCount)
{
return delegateQueryMetrics.reportQueryBytes(byteCount);
}
@Override
public QueryMetrics reportWaitTime(long timeNs)
{
return delegateQueryMetrics.reportWaitTime(timeNs);
}
@Override
public QueryMetrics reportSegmentTime(long timeNs)
{
return delegateQueryMetrics.reportSegmentTime(timeNs);
}
@Override
public QueryMetrics reportSegmentAndCacheTime(long timeNs)
{
return delegateQueryMetrics.reportSegmentAndCacheTime(timeNs);
}
@Override
public QueryMetrics reportIntervalChunkTime(long timeNs)
{
return delegateQueryMetrics.reportIntervalChunkTime(timeNs);
}
@Override
public QueryMetrics reportCpuTime(long timeNs)
{
return delegateQueryMetrics.reportCpuTime(timeNs);
}
@Override
public QueryMetrics reportNodeTimeToFirstByte(long timeNs)
{
return delegateQueryMetrics.reportNodeTimeToFirstByte(timeNs);
}
@Override
public QueryMetrics reportBackPressureTime(long timeNs)
{
return delegateQueryMetrics.reportBackPressureTime(timeNs);
}
@Override
public QueryMetrics reportNodeTime(long timeNs)
{
return delegateQueryMetrics.reportNodeTime(timeNs);
}
@Override
public QueryMetrics reportNodeBytes(long byteCount)
{
return delegateQueryMetrics.reportNodeBytes(byteCount);
}
@Override
public QueryMetrics reportBitmapConstructionTime(long timeNs)
{
return delegateQueryMetrics.reportBitmapConstructionTime(timeNs);
}
@Override
public QueryMetrics reportSegmentRows(long numRows)
{
return delegateQueryMetrics.reportSegmentRows(numRows);
}
@Override
public QueryMetrics reportPreFilteredRows(long numRows)
{
return delegateQueryMetrics.reportPreFilteredRows(numRows);
}
@Override
public void emit(ServiceEmitter emitter)
{
delegateQueryMetrics.emit(emitter);
}
}

View File

@ -1,51 +0,0 @@
/*
* 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.select;
import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Inject;
import org.apache.druid.query.DefaultGenericQueryMetricsFactory;
import org.apache.druid.query.GenericQueryMetricsFactory;
public class DefaultSelectQueryMetricsFactory implements SelectQueryMetricsFactory
{
private static final SelectQueryMetricsFactory INSTANCE =
new DefaultSelectQueryMetricsFactory(DefaultGenericQueryMetricsFactory.instance());
private final GenericQueryMetricsFactory genericQueryMetricsFactory;
@Inject
public DefaultSelectQueryMetricsFactory(GenericQueryMetricsFactory genericQueryMetricsFactory)
{
this.genericQueryMetricsFactory = genericQueryMetricsFactory;
}
@VisibleForTesting
public static SelectQueryMetricsFactory instance()
{
return INSTANCE;
}
@Override
public SelectQueryMetrics makeMetrics(SelectQuery query)
{
return new DefaultSelectQueryMetrics(genericQueryMetricsFactory.makeMetrics(query));
}
}

View File

@ -1,128 +0,0 @@
/*
* 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.select;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.Maps;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
import org.joda.time.DateTime;
import java.util.Map;
/**
*/
public class EventHolder
{
public static final String TIMESTAMP_KEY = "timestamp";
private final String segmentId;
private final int offset;
private final Map<String, Object> event;
@JsonCreator
public EventHolder(
@JsonProperty("segmentId") String segmentId,
@JsonProperty("offset") int offset,
@JsonProperty("event") Map<String, Object> event
)
{
this.segmentId = segmentId;
this.offset = offset;
this.event = event;
}
public DateTime getTimestamp()
{
Object retVal = event.get(TIMESTAMP_KEY);
if (retVal instanceof Long) {
return DateTimes.utc((Long) retVal);
} else if (retVal instanceof String) {
return DateTimes.of((String) retVal);
} else if (retVal instanceof DateTime) {
return (DateTime) retVal;
} else {
throw new ISE("Do not understand format [%s]", retVal.getClass());
}
}
@JsonProperty
public String getSegmentId()
{
return segmentId;
}
@JsonProperty
public int getOffset()
{
return offset;
}
@JsonProperty
public Map<String, Object> getEvent()
{
return event;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
EventHolder that = (EventHolder) o;
if (offset != that.offset) {
return false;
}
if (!Maps.difference(event, ((EventHolder) o).event).areEqual()) {
return false;
}
if (segmentId != null ? !segmentId.equals(that.segmentId) : that.segmentId != null) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = segmentId != null ? segmentId.hashCode() : 0;
result = 31 * result + offset;
result = 31 * result + (event != null ? event.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "EventHolder{" +
"segmentId='" + segmentId + '\'' +
", offset=" + offset +
", event=" + event +
'}';
}
}

View File

@ -1,117 +0,0 @@
/*
* 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.select;
import com.google.common.annotations.VisibleForTesting;
/**
* offset iterator for select query
*/
public abstract class PagingOffset
{
protected final int startOffset;
protected final int threshold;
protected int counter;
public PagingOffset(int startOffset, int threshold)
{
this.startOffset = startOffset;
this.threshold = threshold;
}
public final int startOffset()
{
return startOffset;
}
public abstract int startDelta();
public final int threshold()
{
return threshold;
}
public final boolean hasNext()
{
return counter < threshold;
}
public final void next()
{
counter++;
}
public abstract int current();
private static class Ascending extends PagingOffset
{
public Ascending(int offset, int threshold)
{
super(offset, threshold);
}
@Override
public final int startDelta()
{
return startOffset;
}
@Override
public final int current()
{
return startOffset + counter;
}
}
private static class Descending extends PagingOffset
{
public Descending(int offset, int threshold)
{
super(offset, threshold);
}
@Override
public final int startDelta()
{
return -startOffset - 1;
}
@Override
public final int current()
{
return startOffset - counter;
}
}
public static PagingOffset of(int startOffset, int threshold)
{
return startOffset < 0 ? new Descending(startOffset, threshold) : new Ascending(startOffset, threshold);
}
@VisibleForTesting
static int toOffset(int delta, boolean descending)
{
if (delta < 0) {
throw new IllegalArgumentException("Delta should not be negative");
}
return descending ? -delta - 1 : delta;
}
}

View File

@ -1,194 +0,0 @@
/*
* 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.select;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.StringUtils;
import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.Map;
/**
*/
public class PagingSpec
{
public static PagingSpec newSpec(int threshold)
{
return new PagingSpec(null, threshold);
}
public static Map<String, Integer> merge(Iterable<Map<String, Integer>> cursors)
{
Map<String, Integer> next = new HashMap<>();
for (Map<String, Integer> cursor : cursors) {
for (Map.Entry<String, Integer> entry : cursor.entrySet()) {
next.put(entry.getKey(), entry.getValue());
}
}
return next;
}
public static Map<String, Integer> next(Map<String, Integer> cursor, boolean descending)
{
for (Map.Entry<String, Integer> entry : cursor.entrySet()) {
entry.setValue(descending ? entry.getValue() - 1 : entry.getValue() + 1);
}
return cursor;
}
private final Map<String, Integer> pagingIdentifiers;
private final int threshold;
private final boolean fromNext;
@JsonCreator
public PagingSpec(
@JsonProperty("pagingIdentifiers") Map<String, Integer> pagingIdentifiers,
@JsonProperty("threshold") int threshold,
@JsonProperty("fromNext") Boolean fromNext,
@JacksonInject SelectQueryConfig config
)
{
this.pagingIdentifiers = pagingIdentifiers == null ? new HashMap<>() : pagingIdentifiers;
this.threshold = threshold;
boolean defaultFromNext = config.getEnableFromNextDefault();
this.fromNext = fromNext == null ? defaultFromNext : fromNext;
}
public PagingSpec(Map<String, Integer> pagingIdentifiers, int threshold)
{
this(pagingIdentifiers, threshold, null, new SelectQueryConfig(true));
}
public PagingSpec(Map<String, Integer> pagingIdentifiers, int threshold, Boolean fromNext)
{
this(pagingIdentifiers, threshold, fromNext, new SelectQueryConfig(true));
}
@JsonProperty
public Map<String, Integer> getPagingIdentifiers()
{
return pagingIdentifiers;
}
@JsonProperty
public int getThreshold()
{
return threshold;
}
@JsonProperty
public boolean isFromNext()
{
return fromNext;
}
public byte[] getCacheKey()
{
final byte[][] pagingKeys = new byte[pagingIdentifiers.size()][];
final byte[][] pagingValues = new byte[pagingIdentifiers.size()][];
int index = 0;
int pagingKeysSize = 0;
int pagingValuesSize = 0;
for (Map.Entry<String, Integer> entry : pagingIdentifiers.entrySet()) {
pagingKeys[index] = StringUtils.toUtf8(entry.getKey());
pagingValues[index] = ByteBuffer.allocate(Integer.BYTES).putInt(entry.getValue()).array();
pagingKeysSize += pagingKeys[index].length;
pagingValuesSize += Integer.BYTES;
index++;
}
final byte[] thresholdBytes = ByteBuffer.allocate(Integer.BYTES).putInt(threshold).array();
final ByteBuffer queryCacheKey = ByteBuffer.allocate(pagingKeysSize + pagingValuesSize + thresholdBytes.length + 1);
for (byte[] pagingKey : pagingKeys) {
queryCacheKey.put(pagingKey);
}
for (byte[] pagingValue : pagingValues) {
queryCacheKey.put(pagingValue);
}
queryCacheKey.put(thresholdBytes);
queryCacheKey.put(isFromNext() ? (byte) 0x01 : 0x00);
return queryCacheKey.array();
}
public PagingOffset getOffset(String identifier, boolean descending)
{
Integer offset = pagingIdentifiers.get(identifier);
if (offset == null) {
offset = PagingOffset.toOffset(0, descending);
} else if (fromNext) {
offset = descending ? offset - 1 : offset + 1;
}
return PagingOffset.of(offset, threshold);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
PagingSpec that = (PagingSpec) o;
if (fromNext != that.fromNext) {
return false;
}
if (threshold != that.threshold) {
return false;
}
if (!pagingIdentifiers.equals(that.pagingIdentifiers)) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = pagingIdentifiers.hashCode();
result = 31 * result + threshold;
result = 31 * result + (fromNext ? 1 : 0);
return result;
}
@Override
public String toString()
{
return "PagingSpec{" +
"pagingIdentifiers=" + pagingIdentifiers +
", threshold=" + threshold +
", fromNext=" + fromNext +
'}';
}
}

View File

@ -1,105 +0,0 @@
/*
* 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.select;
import com.google.common.collect.Sets;
import org.apache.druid.java.util.common.granularity.AllGranularity;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.query.Result;
import org.joda.time.DateTime;
import java.util.List;
import java.util.Set;
import java.util.function.BinaryOperator;
/**
*/
public class SelectBinaryFn implements BinaryOperator<Result<SelectResultValue>>
{
private final Granularity gran;
private final PagingSpec pagingSpec;
private final boolean descending;
public SelectBinaryFn(Granularity granularity, PagingSpec pagingSpec, boolean descending)
{
this.gran = granularity;
this.pagingSpec = pagingSpec;
this.descending = descending;
}
@Override
public Result<SelectResultValue> apply(Result<SelectResultValue> arg1, Result<SelectResultValue> arg2)
{
if (arg1 == null) {
return arg2;
}
if (arg2 == null) {
return arg1;
}
final List<EventHolder> arg1Val = arg1.getValue().getEvents();
final List<EventHolder> arg2Val = arg2.getValue().getEvents();
if (arg1Val == null || arg1Val.isEmpty()) {
return arg2;
}
if (arg2Val == null || arg2Val.isEmpty()) {
return arg1;
}
final DateTime timestamp = (gran instanceof AllGranularity)
? arg1.getTimestamp()
: gran.bucketStart(arg1.getTimestamp());
SelectResultValueBuilder builder = new SelectResultValueBuilder.MergeBuilder(timestamp, pagingSpec, descending);
builder.addDimensions(mergeColumns(arg1.getValue().getDimensions(), arg2.getValue().getDimensions()));
builder.addMetrics(mergeColumns(arg1.getValue().getMetrics(), arg2.getValue().getMetrics()));
for (EventHolder event : arg1Val) {
builder.addEntry(event);
}
for (EventHolder event : arg2Val) {
builder.addEntry(event);
}
return builder.build();
}
private Set<String> mergeColumns(final Set<String> arg1, final Set<String> arg2)
{
if (arg1.isEmpty()) {
return arg2;
}
if (arg2.isEmpty()) {
return arg1;
}
if (arg1.equals(arg2)) {
return arg1;
}
return Sets.union(arg1, arg2);
}
}

View File

@ -20,212 +20,151 @@
package org.apache.druid.query.select;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.granularity.Granularities;
import com.google.common.collect.Ordering;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.query.BaseQuery;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.Druids;
import org.apache.druid.query.Query;
import org.apache.druid.query.Result;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.segment.VirtualColumns;
import org.joda.time.DateTimeZone;
import org.joda.time.Duration;
import org.joda.time.Interval;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
*/
@JsonTypeName("select")
public class SelectQuery extends BaseQuery<Result<SelectResultValue>>
@Deprecated
public class SelectQuery implements Query<Object>
{
private final DimFilter dimFilter;
private final List<DimensionSpec> dimensions;
private final List<String> metrics;
private final VirtualColumns virtualColumns;
private final PagingSpec pagingSpec;
static final String REMOVED_ERROR_MESSAGE =
"The 'select' query has been removed, use 'scan' instead. See "
+ "https://druid.apache.org/docs/latest/querying/select-query.html for more details.";
@JsonCreator
public SelectQuery(
@JsonProperty("dataSource") DataSource dataSource,
@JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
@JsonProperty("descending") boolean descending,
@JsonProperty("filter") DimFilter dimFilter,
@JsonProperty("granularity") Granularity granularity,
@JsonProperty("dimensions") List<DimensionSpec> dimensions,
@JsonProperty("metrics") List<String> metrics,
@JsonProperty("virtualColumns") VirtualColumns virtualColumns,
@JsonProperty("pagingSpec") PagingSpec pagingSpec,
@JsonProperty("context") Map<String, Object> context
)
public SelectQuery()
{
super(dataSource, querySegmentSpec, descending, context, Granularities.nullToAll(granularity));
this.dimFilter = dimFilter;
this.dimensions = dimensions;
this.virtualColumns = VirtualColumns.nullToEmpty(virtualColumns);
this.metrics = metrics;
this.pagingSpec = pagingSpec;
Preconditions.checkNotNull(pagingSpec, "must specify a pagingSpec");
Preconditions.checkArgument(checkPagingSpec(pagingSpec, descending), "invalid pagingSpec");
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
private boolean checkPagingSpec(PagingSpec pagingSpec, boolean descending)
@Override
public DataSource getDataSource()
{
for (Integer value : pagingSpec.getPagingIdentifiers().values()) {
if (descending ^ (value < 0)) {
return false;
}
}
return pagingSpec.getThreshold() >= 0;
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public boolean hasFilters()
{
return dimFilter != null;
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public DimFilter getFilter()
{
return dimFilter;
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public String getType()
{
return Query.SELECT;
}
@JsonProperty("filter")
public DimFilter getDimensionsFilter()
{
return dimFilter;
}
@JsonProperty
public List<DimensionSpec> getDimensions()
{
return dimensions;
}
@JsonProperty
public PagingSpec getPagingSpec()
{
return pagingSpec;
}
@JsonProperty
public List<String> getMetrics()
{
return metrics;
}
@JsonProperty
public VirtualColumns getVirtualColumns()
{
return virtualColumns;
}
public PagingOffset getPagingOffset(String identifier)
{
return pagingSpec.getOffset(identifier, isDescending());
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public SelectQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec)
public QueryRunner<Object> getRunner(QuerySegmentWalker walker)
{
return Druids.SelectQueryBuilder.copy(this).intervals(querySegmentSpec).build();
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public Query<Result<SelectResultValue>> withDataSource(DataSource dataSource)
public List<Interval> getIntervals()
{
return Druids.SelectQueryBuilder.copy(this).dataSource(dataSource).build();
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public SelectQuery withOverriddenContext(Map<String, Object> contextOverrides)
public Duration getDuration()
{
Map<String, Object> newContext = computeOverriddenContext(getContext(), contextOverrides);
return Druids.SelectQueryBuilder.copy(this).context(newContext).build();
}
public SelectQuery withPagingSpec(PagingSpec pagingSpec)
{
return Druids.SelectQueryBuilder.copy(this).pagingSpec(pagingSpec).build();
}
public SelectQuery withDimFilter(DimFilter dimFilter)
{
return Druids.SelectQueryBuilder.copy(this).filters(dimFilter).build();
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public String toString()
public Granularity getGranularity()
{
return "SelectQuery{" +
"dataSource='" + getDataSource() + '\'' +
", querySegmentSpec=" + getQuerySegmentSpec() +
", descending=" + isDescending() +
", dimFilter=" + dimFilter +
", granularity=" + getGranularity() +
", dimensions=" + dimensions +
", metrics=" + metrics +
", virtualColumns=" + virtualColumns +
", pagingSpec=" + pagingSpec +
'}';
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public boolean equals(Object o)
public DateTimeZone getTimezone()
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
if (!super.equals(o)) {
return false;
}
SelectQuery that = (SelectQuery) o;
if (!Objects.equals(dimFilter, that.dimFilter)) {
return false;
}
if (!Objects.equals(dimensions, that.dimensions)) {
return false;
}
if (!Objects.equals(metrics, that.metrics)) {
return false;
}
if (!Objects.equals(virtualColumns, that.virtualColumns)) {
return false;
}
if (!Objects.equals(pagingSpec, that.pagingSpec)) {
return false;
}
return true;
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public int hashCode()
public Map<String, Object> getContext()
{
int result = super.hashCode();
result = 31 * result + (dimFilter != null ? dimFilter.hashCode() : 0);
result = 31 * result + (dimensions != null ? dimensions.hashCode() : 0);
result = 31 * result + (metrics != null ? metrics.hashCode() : 0);
result = 31 * result + (virtualColumns != null ? virtualColumns.hashCode() : 0);
result = 31 * result + (pagingSpec != null ? pagingSpec.hashCode() : 0);
return result;
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public <ContextType> ContextType getContextValue(String key)
{
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public <ContextType> ContextType getContextValue(String key, ContextType defaultValue)
{
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public boolean getContextBoolean(String key, boolean defaultValue)
{
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public boolean isDescending()
{
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public Ordering<Object> getResultOrdering()
{
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public Query<Object> withOverriddenContext(Map<String, Object> contextOverride)
{
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public Query<Object> withQuerySegmentSpec(QuerySegmentSpec spec)
{
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public Query<Object> withId(String id)
{
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public String getId()
{
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
@Override
public Query<Object> withDataSource(DataSource dataSource)
{
throw new RuntimeException(REMOVED_ERROR_MESSAGE);
}
}

View File

@ -1,51 +0,0 @@
/*
* 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.select;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
public class SelectQueryConfig
{
public static String ENABLE_FROM_NEXT_DEFAULT = "enableFromNextDefault";
@JsonProperty
private boolean enableFromNextDefault = true;
@JsonCreator
public SelectQueryConfig(
@JsonProperty("enableFromNextDefault") Boolean enableFromNextDefault
)
{
if (enableFromNextDefault != null) {
this.enableFromNextDefault = enableFromNextDefault.booleanValue();
}
}
public boolean getEnableFromNextDefault()
{
return enableFromNextDefault;
}
public void setEnableFromNextDefault(boolean enableFromNextDefault)
{
this.enableFromNextDefault = enableFromNextDefault;
}
}

View File

@ -1,321 +0,0 @@
/*
* 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.select;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.ColumnSelectorPlus;
import org.apache.druid.query.QueryRunnerHelper;
import org.apache.druid.query.Result;
import org.apache.druid.query.dimension.ColumnSelectorStrategy;
import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.data.IndexedInts;
import org.apache.druid.segment.filter.Filters;
import org.joda.time.Interval;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
*/
public class SelectQueryEngine
{
private static final SelectStrategyFactory STRATEGY_FACTORY = new SelectStrategyFactory();
public static class SelectStrategyFactory implements ColumnSelectorStrategyFactory<SelectColumnSelectorStrategy>
{
@Override
public SelectColumnSelectorStrategy makeColumnSelectorStrategy(
ColumnCapabilities capabilities,
ColumnValueSelector selector
)
{
ValueType type = capabilities.getType();
switch (type) {
case STRING:
return new StringSelectColumnSelectorStrategy();
case LONG:
return new LongSelectColumnSelectorStrategy();
case FLOAT:
return new FloatSelectColumnSelectorStrategy();
case DOUBLE:
return new DoubleSelectColumnSelectorStrategy();
default:
throw new IAE("Cannot create query type helper from invalid type [%s]", type);
}
}
}
public interface SelectColumnSelectorStrategy<ValueSelectorType> extends ColumnSelectorStrategy
{
/**
* Read the current row from selector and add the row values for a dimension to the result map.
*
* Multi-valued rows should be added to the result as a List, single value rows should be added as a single object.
*
* @param outputName Output name for this dimension in the select query being served
* @param selector Dimension value selector
* @param resultMap Row value map for the current row being retrieved by the select query
*/
void addRowValuesToSelectResult(
String outputName,
ValueSelectorType selector,
Map<String, Object> resultMap
);
}
public static class StringSelectColumnSelectorStrategy implements SelectColumnSelectorStrategy<DimensionSelector>
{
@Override
public void addRowValuesToSelectResult(String outputName, DimensionSelector selector, Map<String, Object> resultMap)
{
final IndexedInts row = selector.getRow();
int rowSize = row.size();
if (rowSize == 0) {
resultMap.put(outputName, null);
} else if (rowSize == 1) {
final String dimVal = selector.lookupName(row.get(0));
resultMap.put(outputName, dimVal);
} else {
List<String> dimVals = new ArrayList<>(rowSize);
for (int i = 0; i < rowSize; ++i) {
dimVals.add(selector.lookupName(row.get(i)));
}
resultMap.put(outputName, dimVals);
}
}
}
public static class LongSelectColumnSelectorStrategy
implements SelectColumnSelectorStrategy<BaseLongColumnValueSelector>
{
@Override
public void addRowValuesToSelectResult(
String outputName,
BaseLongColumnValueSelector selector,
Map<String, Object> resultMap
)
{
if (selector == null) {
resultMap.put(outputName, null);
} else {
resultMap.put(outputName, selector.getLong());
}
}
}
public static class FloatSelectColumnSelectorStrategy
implements SelectColumnSelectorStrategy<BaseFloatColumnValueSelector>
{
@Override
public void addRowValuesToSelectResult(
String outputName,
BaseFloatColumnValueSelector selector,
Map<String, Object> resultMap
)
{
if (selector == null) {
resultMap.put(outputName, null);
} else {
resultMap.put(outputName, selector.getFloat());
}
}
}
public static class DoubleSelectColumnSelectorStrategy
implements SelectColumnSelectorStrategy<BaseDoubleColumnValueSelector>
{
@Override
public void addRowValuesToSelectResult(
String outputName,
BaseDoubleColumnValueSelector selector,
Map<String, Object> resultMap
)
{
if (selector == null) {
resultMap.put(outputName, null);
} else {
resultMap.put(outputName, selector.getDouble());
}
}
}
public Sequence<Result<SelectResultValue>> process(final SelectQuery query, final Segment segment)
{
final StorageAdapter adapter = segment.asStorageAdapter();
if (adapter == null) {
throw new ISE(
"Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped."
);
}
Preconditions.checkArgument(
query.getDataSource().getNames().size() == 1,
"At the point where this code is called, only one data source should exist. Data sources: %s",
query.getDataSource().getNames()
);
final Iterable<DimensionSpec> dims;
if (query.getDimensions() == null || query.getDimensions().isEmpty()) {
dims = DefaultDimensionSpec.toSpec(adapter.getAvailableDimensions());
} else {
dims = query.getDimensions();
}
final Iterable<String> metrics;
if (query.getMetrics() == null || query.getMetrics().isEmpty()) {
metrics = adapter.getAvailableMetrics();
} else {
metrics = query.getMetrics();
}
List<Interval> intervals = query.getQuerySegmentSpec().getIntervals();
Preconditions.checkArgument(intervals.size() == 1, "Can only handle a single interval, got[%s]", intervals);
// should be rewritten with given interval
final String segmentId = segment.getId().withInterval(intervals.get(0)).toString();
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimensionsFilter()));
return QueryRunnerHelper.makeCursorBasedQuery(
adapter,
query.getQuerySegmentSpec().getIntervals(),
filter,
query.getVirtualColumns(),
query.isDescending(),
query.getGranularity(),
new Function<Cursor, Result<SelectResultValue>>()
{
@Override
public Result<SelectResultValue> apply(Cursor cursor)
{
final SelectResultValueBuilder builder = new SelectResultValueBuilder(
cursor.getTime(),
query.getPagingSpec(),
query.isDescending()
);
final BaseLongColumnValueSelector timestampColumnSelector =
cursor.getColumnSelectorFactory().makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME);
final List<ColumnSelectorPlus<SelectColumnSelectorStrategy>> selectorPlusList = Arrays.asList(
DimensionHandlerUtils.createColumnSelectorPluses(
STRATEGY_FACTORY,
Lists.newArrayList(dims),
cursor.getColumnSelectorFactory()
)
);
for (DimensionSpec dimSpec : dims) {
builder.addDimension(dimSpec.getOutputName());
}
final Map<String, BaseObjectColumnValueSelector<?>> metSelectors = new HashMap<>();
for (String metric : metrics) {
final BaseObjectColumnValueSelector<?> metricSelector =
cursor.getColumnSelectorFactory().makeColumnValueSelector(metric);
metSelectors.put(metric, metricSelector);
builder.addMetric(metric);
}
final PagingOffset offset = query.getPagingOffset(segmentId);
cursor.advanceTo(offset.startDelta());
int lastOffset = offset.startOffset();
for (; !cursor.isDone() && offset.hasNext(); cursor.advance(), offset.next()) {
final Map<String, Object> theEvent = singleEvent(
EventHolder.TIMESTAMP_KEY,
timestampColumnSelector,
selectorPlusList,
metSelectors
);
builder.addEntry(
new EventHolder(
segmentId,
lastOffset = offset.current(),
theEvent
)
);
}
builder.finished(segmentId, lastOffset);
return builder.build();
}
}
);
}
public static Map<String, Object> singleEvent(
String timestampKey,
BaseLongColumnValueSelector timestampColumnSelector,
List<ColumnSelectorPlus<SelectColumnSelectorStrategy>> selectorPlusList,
Map<String, BaseObjectColumnValueSelector<?>> metSelectors
)
{
final Map<String, Object> theEvent = Maps.newLinkedHashMap();
theEvent.put(timestampKey, DateTimes.utc(timestampColumnSelector.getLong()));
for (ColumnSelectorPlus<SelectColumnSelectorStrategy> selectorPlus : selectorPlusList) {
selectorPlus
.getColumnSelectorStrategy()
.addRowValuesToSelectResult(selectorPlus.getOutputName(), selectorPlus.getSelector(), theEvent);
}
for (Map.Entry<String, BaseObjectColumnValueSelector<?>> metSelector : metSelectors.entrySet()) {
final String metric = metSelector.getKey();
final BaseObjectColumnValueSelector<?> selector = metSelector.getValue();
if (selector == null) {
theEvent.put(metric, null);
} else {
theEvent.put(metric, selector.getObject());
}
}
return theEvent;
}
}

View File

@ -1,34 +0,0 @@
/*
* 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.select;
import org.apache.druid.query.QueryMetrics;
public interface SelectQueryMetrics extends QueryMetrics<SelectQuery>
{
/**
* Sets the granularity of {@link SelectQuery#getGranularity()} of the given query as dimension.
*
* This method is going to be used in "full" metrics impl,
* see https://github.com/apache/incubator-druid/pull/4570#issuecomment-319458229
*/
@SuppressWarnings("unused")
void granularity(SelectQuery query);
}

View File

@ -1,36 +0,0 @@
/*
* 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.select;
/**
* Implementations could be injected using
*
* PolyBind
* .optionBinder(binder, Key.get(SelectQueryMetricsFactory.class))
* .addBinding("myCustomSelectQueryMetricsFactory")
* .to(MyCustomSelectQueryMetricsFactory.class);
*
* And then setting property:
* druid.query.select.queryMetricsFactory=myCustomSelectQueryMetricsFactory
*/
public interface SelectQueryMetricsFactory
{
SelectQueryMetrics makeMetrics(SelectQuery selectQuery);
}

View File

@ -1,403 +0,0 @@
/*
* 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.select;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Functions;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.inject.Inject;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.guava.Comparators;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.CacheStrategy;
import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
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;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.function.BinaryOperator;
/**
*/
public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResultValue>, SelectQuery>
{
private static final byte SELECT_QUERY = 0x16;
private static final TypeReference<Object> OBJECT_TYPE_REFERENCE =
new TypeReference<Object>()
{
};
private static final TypeReference<Result<SelectResultValue>> TYPE_REFERENCE =
new TypeReference<Result<SelectResultValue>>()
{
};
private final ObjectMapper jsonMapper;
@Deprecated
private final IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator;
private final SelectQueryMetricsFactory queryMetricsFactory;
public SelectQueryQueryToolChest(
ObjectMapper jsonMapper,
IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator
)
{
this(jsonMapper, intervalChunkingQueryRunnerDecorator, DefaultSelectQueryMetricsFactory.instance());
}
@Inject
public SelectQueryQueryToolChest(
ObjectMapper jsonMapper,
IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator,
SelectQueryMetricsFactory queryMetricsFactory
)
{
this.jsonMapper = jsonMapper;
this.intervalChunkingQueryRunnerDecorator = intervalChunkingQueryRunnerDecorator;
this.queryMetricsFactory = queryMetricsFactory;
}
@Override
public BinaryOperator<Result<SelectResultValue>> createMergeFn(
Query<Result<SelectResultValue>> query
)
{
final SelectQuery selectQuery = (SelectQuery) query;
return new SelectBinaryFn(selectQuery.getGranularity(), selectQuery.getPagingSpec(), selectQuery.isDescending());
}
@Override
public Comparator<Result<SelectResultValue>> createResultComparator(Query<Result<SelectResultValue>> query)
{
return ResultGranularTimestampComparator.create(query.getGranularity(), query.isDescending());
}
@Override
public SelectQueryMetrics makeMetrics(SelectQuery query)
{
SelectQueryMetrics queryMetrics = queryMetricsFactory.makeMetrics(query);
queryMetrics.query(query);
return queryMetrics;
}
@Override
public Function<Result<SelectResultValue>, Result<SelectResultValue>> makePreComputeManipulatorFn(
final SelectQuery query,
final MetricManipulationFn fn
)
{
return Functions.identity();
}
@Override
public TypeReference<Result<SelectResultValue>> getResultTypeReference()
{
return TYPE_REFERENCE;
}
@Override
public CacheStrategy<Result<SelectResultValue>, Object, SelectQuery> getCacheStrategy(final SelectQuery query)
{
return new CacheStrategy<Result<SelectResultValue>, Object, SelectQuery>()
{
private final List<DimensionSpec> dimensionSpecs =
query.getDimensions() != null ? query.getDimensions() : Collections.emptyList();
private final List<String> dimOutputNames = dimensionSpecs.size() > 0 ?
Lists.transform(dimensionSpecs, DimensionSpec::getOutputName) : Collections.emptyList();
@Override
public boolean isCacheable(SelectQuery query, boolean willMergeRunners)
{
return true;
}
@Override
public byte[] computeCacheKey(SelectQuery query)
{
final DimFilter dimFilter = query.getDimensionsFilter();
final byte[] filterBytes = dimFilter == null ? new byte[]{} : dimFilter.getCacheKey();
final byte[] granularityBytes = query.getGranularity().getCacheKey();
final List<DimensionSpec> dimensionSpecs =
query.getDimensions() != null ? query.getDimensions() : Collections.emptyList();
final byte[][] dimensionsBytes = new byte[dimensionSpecs.size()][];
int dimensionsBytesSize = 0;
int index = 0;
for (DimensionSpec dimension : dimensionSpecs) {
dimensionsBytes[index] = dimension.getCacheKey();
dimensionsBytesSize += dimensionsBytes[index].length;
++index;
}
final Set<String> metrics = new TreeSet<>();
if (query.getMetrics() != null) {
metrics.addAll(query.getMetrics());
}
final byte[][] metricBytes = new byte[metrics.size()][];
int metricBytesSize = 0;
index = 0;
for (String metric : metrics) {
metricBytes[index] = StringUtils.toUtf8(metric);
metricBytesSize += metricBytes[index].length;
++index;
}
final byte[] virtualColumnsCacheKey = query.getVirtualColumns().getCacheKey();
final byte isDescendingByte = query.isDescending() ? (byte) 1 : 0;
final ByteBuffer queryCacheKey = ByteBuffer
.allocate(
2
+ granularityBytes.length
+ filterBytes.length
+ query.getPagingSpec().getCacheKey().length
+ dimensionsBytesSize
+ metricBytesSize
+ virtualColumnsCacheKey.length
)
.put(SELECT_QUERY)
.put(granularityBytes)
.put(filterBytes)
.put(query.getPagingSpec().getCacheKey())
.put(isDescendingByte);
for (byte[] dimensionsByte : dimensionsBytes) {
queryCacheKey.put(dimensionsByte);
}
for (byte[] metricByte : metricBytes) {
queryCacheKey.put(metricByte);
}
queryCacheKey.put(virtualColumnsCacheKey);
return queryCacheKey.array();
}
@Override
public byte[] computeResultLevelCacheKey(SelectQuery query)
{
return computeCacheKey(query);
}
@Override
public TypeReference<Object> getCacheObjectClazz()
{
return OBJECT_TYPE_REFERENCE;
}
@Override
public Function<Result<SelectResultValue>, Object> prepareForCache(boolean isResultLevelCache)
{
return new Function<Result<SelectResultValue>, Object>()
{
@Override
public Object apply(final Result<SelectResultValue> input)
{
if (!dimOutputNames.isEmpty()) {
return Arrays.asList(
input.getTimestamp().getMillis(),
input.getValue().getPagingIdentifiers(),
input.getValue().getDimensions(),
input.getValue().getMetrics(),
input.getValue().getEvents(),
dimOutputNames
);
}
return Arrays.asList(
input.getTimestamp().getMillis(),
input.getValue().getPagingIdentifiers(),
input.getValue().getDimensions(),
input.getValue().getMetrics(),
input.getValue().getEvents()
);
}
};
}
@Override
public Function<Object, Result<SelectResultValue>> pullFromCache(boolean isResultLevelCache)
{
return new Function<Object, Result<SelectResultValue>>()
{
private final Granularity granularity = query.getGranularity();
@Override
public Result<SelectResultValue> apply(Object input)
{
List<Object> results = (List<Object>) input;
Iterator<Object> resultIter = results.iterator();
DateTime timestamp = granularity.toDateTime(((Number) resultIter.next()).longValue());
Map<String, Integer> pageIdentifier = jsonMapper.convertValue(
resultIter.next(),
new TypeReference<Map<String, Integer>>() {}
);
Set<String> dimensionSet = jsonMapper.convertValue(resultIter.next(), new TypeReference<Set<String>>() {});
Set<String> metricSet = jsonMapper.convertValue(resultIter.next(), new TypeReference<Set<String>>() {});
List<EventHolder> eventHolders = jsonMapper.convertValue(
resultIter.next(),
new TypeReference<List<EventHolder>>() {}
);
// check the condition that outputName of cached result should be updated
if (resultIter.hasNext()) {
List<String> cachedOutputNames = (List<String>) resultIter.next();
Preconditions.checkArgument(
cachedOutputNames.size() == dimOutputNames.size(),
"Cache hit but different number of dimensions??"
);
for (int idx = 0; idx < dimOutputNames.size(); idx++) {
if (!cachedOutputNames.get(idx).equals(dimOutputNames.get(idx))) {
// rename outputName in the EventHolder
for (EventHolder eventHolder : eventHolders) {
Object obj = eventHolder.getEvent().remove(cachedOutputNames.get(idx));
if (obj != null) {
eventHolder.getEvent().put(dimOutputNames.get(idx), obj);
}
}
}
}
}
return new Result<>(
timestamp,
new SelectResultValue(pageIdentifier, dimensionSet, metricSet, eventHolders)
);
}
};
}
};
}
@Override
public QueryRunner<Result<SelectResultValue>> preMergeQueryDecoration(final QueryRunner<Result<SelectResultValue>> runner)
{
return intervalChunkingQueryRunnerDecorator.decorate(
new QueryRunner<Result<SelectResultValue>>()
{
@Override
public Sequence<Result<SelectResultValue>> run(
QueryPlus<Result<SelectResultValue>> queryPlus,
ResponseContext responseContext
)
{
SelectQuery selectQuery = (SelectQuery) queryPlus.getQuery();
if (selectQuery.getDimensionsFilter() != null) {
selectQuery = selectQuery.withDimFilter(selectQuery.getDimensionsFilter().optimize());
queryPlus = queryPlus.withQuery(selectQuery);
}
return runner.run(queryPlus, responseContext);
}
}, this);
}
@Override
public <T extends LogicalSegment> List<T> filterSegments(SelectQuery query, List<T> segments)
{
// at the point where this code is called, only one datasource should exist.
final String dataSource = Iterables.getOnlyElement(query.getDataSource().getNames());
PagingSpec pagingSpec = query.getPagingSpec();
Map<String, Integer> paging = pagingSpec.getPagingIdentifiers();
if (paging == null || paging.isEmpty()) {
return segments;
}
final Granularity granularity = query.getGranularity();
TreeMap<Long, Long> granularThresholds = new TreeMap<>();
// A paged select query using a UnionDataSource will return pagingIdentifiers from segments in more than one
// dataSource which confuses subsequent queries and causes a failure. To avoid this, filter only the paging keys
// that are applicable to this dataSource so that each dataSource in a union query gets the appropriate keys.
paging
.keySet()
.stream()
.filter(identifier -> SegmentId.tryParse(dataSource, identifier) != null)
.map(SegmentId.makeIntervalExtractor(dataSource))
.sorted(query.isDescending() ? Comparators.intervalsByEndThenStart()
: Comparators.intervalsByStartThenEnd())
.forEach(interval -> {
if (query.isDescending()) {
long granularEnd = granularity.bucketStart(interval.getEnd()).getMillis();
Long currentEnd = granularThresholds.get(granularEnd);
if (currentEnd == null || interval.getEndMillis() > currentEnd) {
granularThresholds.put(granularEnd, interval.getEndMillis());
}
} else {
long granularStart = granularity.bucketStart(interval.getStart()).getMillis();
Long currentStart = granularThresholds.get(granularStart);
if (currentStart == null || interval.getStartMillis() < currentStart) {
granularThresholds.put(granularStart, interval.getStartMillis());
}
}
});
List<T> queryIntervals = Lists.newArrayList(segments);
Iterator<T> it = queryIntervals.iterator();
if (query.isDescending()) {
while (it.hasNext()) {
Interval interval = it.next().getInterval();
Map.Entry<Long, Long> ceiling = granularThresholds.ceilingEntry(granularity.bucketStart(interval.getEnd()).getMillis());
if (ceiling == null || interval.getStartMillis() >= ceiling.getValue()) {
it.remove();
}
}
} else {
while (it.hasNext()) {
Interval interval = it.next().getInterval();
Map.Entry<Long, Long> floor = granularThresholds.floorEntry(granularity.bucketStart(interval.getStart()).getMillis());
if (floor == null || interval.getEndMillis() <= floor.getValue()) {
it.remove();
}
}
}
return queryIntervals;
}
}

View File

@ -1,105 +0,0 @@
/*
* 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.select;
import com.google.inject.Inject;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.ChainedExecutionQueryRunner;
import org.apache.druid.query.Query;
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.QueryWatcher;
import org.apache.druid.query.Result;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.Segment;
import java.util.concurrent.ExecutorService;
/**
*/
public class SelectQueryRunnerFactory
implements QueryRunnerFactory<Result<SelectResultValue>, SelectQuery>
{
private final SelectQueryQueryToolChest toolChest;
private final SelectQueryEngine engine;
private final QueryWatcher queryWatcher;
@Inject
public SelectQueryRunnerFactory(
SelectQueryQueryToolChest toolChest,
SelectQueryEngine engine,
QueryWatcher queryWatcher
)
{
this.toolChest = toolChest;
this.engine = engine;
this.queryWatcher = queryWatcher;
}
@Override
public QueryRunner<Result<SelectResultValue>> createRunner(final Segment segment)
{
return new SelectQueryRunner(engine, segment);
}
@Override
public QueryRunner<Result<SelectResultValue>> mergeRunners(
ExecutorService queryExecutor,
Iterable<QueryRunner<Result<SelectResultValue>>> queryRunners
)
{
return new ChainedExecutionQueryRunner<>(queryExecutor, queryWatcher, queryRunners);
}
@Override
public QueryToolChest<Result<SelectResultValue>, SelectQuery> getToolchest()
{
return toolChest;
}
private static class SelectQueryRunner implements QueryRunner<Result<SelectResultValue>>
{
private final SelectQueryEngine engine;
private final Segment segment;
private SelectQueryRunner(SelectQueryEngine engine, Segment segment)
{
this.engine = engine;
this.segment = segment;
}
@Override
public Sequence<Result<SelectResultValue>> run(
QueryPlus<Result<SelectResultValue>> queryPlus,
ResponseContext responseContext
)
{
Query<Result<SelectResultValue>> input = queryPlus.getQuery();
if (!(input instanceof SelectQuery)) {
throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SelectQuery.class);
}
return engine.process((SelectQuery) input, segment);
}
}
}

View File

@ -1,136 +0,0 @@
/*
* 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.select;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
*/
public class SelectResultValue implements Iterable<EventHolder>
{
private final Map<String, Integer> pagingIdentifiers;
private final Set<String> dimensions;
private final Set<String> metrics;
private final List<EventHolder> events;
@JsonCreator
public SelectResultValue(
@JsonProperty("pagingIdentifiers") Map<String, Integer> pagingIdentifiers,
@JsonProperty("dimensions") Set<String> dimensions,
@JsonProperty("metrics") Set<String> metrics,
@JsonProperty("events") List<EventHolder> events
)
{
this.pagingIdentifiers = pagingIdentifiers;
this.dimensions = dimensions;
this.metrics = metrics;
this.events = events;
}
@JsonProperty
public Map<String, Integer> getPagingIdentifiers()
{
return pagingIdentifiers;
}
@JsonProperty
public Set<String> getDimensions()
{
return dimensions;
}
@JsonProperty
public Set<String> getMetrics()
{
return metrics;
}
@JsonProperty
public List<EventHolder> getEvents()
{
return events;
}
@Override
public Iterator<EventHolder> iterator()
{
return events.iterator();
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SelectResultValue that = (SelectResultValue) o;
if (events != null ? !events.equals(that.events) : that.events != null) {
return false;
}
if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) {
return false;
}
if (metrics != null ? !metrics.equals(that.metrics) : that.metrics != null) {
return false;
}
if (pagingIdentifiers != null
? !pagingIdentifiers.equals(that.pagingIdentifiers)
: that.pagingIdentifiers != null) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = pagingIdentifiers != null ? pagingIdentifiers.hashCode() : 0;
result = 31 * result + (dimensions != null ? dimensions.hashCode() : 0);
result = 31 * result + (metrics != null ? metrics.hashCode() : 0);
result = 31 * result + (events != null ? events.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "SelectResultValue{" +
"pagingIdentifiers=" + pagingIdentifiers +
", dimensions=" + dimensions +
", metrics=" + metrics +
", events=" + events +
'}';
}
}

View File

@ -1,153 +0,0 @@
/*
* 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.select;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.MinMaxPriorityQueue;
import com.google.common.primitives.Longs;
import org.apache.druid.query.Result;
import org.joda.time.DateTime;
import java.util.ArrayDeque;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
/**
*/
public class SelectResultValueBuilder
{
private static final Comparator<EventHolder> COMPARATOR = new Comparator<EventHolder>()
{
@Override
public int compare(EventHolder o1, EventHolder o2)
{
int retVal = Longs.compare(o1.getTimestamp().getMillis(), o2.getTimestamp().getMillis());
if (retVal == 0) {
retVal = o1.getSegmentId().compareTo(o2.getSegmentId());
}
if (retVal == 0) {
retVal = Integer.compare(o1.getOffset(), o2.getOffset());
}
return retVal;
}
};
protected final DateTime timestamp;
protected final PagingSpec pagingSpec;
protected final boolean descending;
protected Set<String> dimensions;
protected Set<String> metrics;
protected final Queue<EventHolder> pQueue;
protected final Map<String, Integer> pagingIdentifiers;
public SelectResultValueBuilder(DateTime timestamp, PagingSpec pagingSpec, boolean descending)
{
this.timestamp = timestamp;
this.pagingSpec = pagingSpec;
this.descending = descending;
this.dimensions = new HashSet<>();
this.metrics = new HashSet<>();
this.pagingIdentifiers = Maps.newLinkedHashMap();
this.pQueue = instantiatePQueue();
}
public void addEntry(EventHolder event)
{
pQueue.add(event);
}
public void finished(String segmentId, int lastOffset)
{
pagingIdentifiers.put(segmentId, lastOffset);
}
public void addDimension(String dimension)
{
dimensions.add(dimension);
}
public void addDimensions(Set<String> dimensions)
{
this.dimensions.addAll(dimensions);
}
public void addMetric(String metric)
{
metrics.add(metric);
}
public void addMetrics(Set<String> metrics)
{
this.metrics.addAll(metrics);
}
public Result<SelectResultValue> build()
{
return new Result<>(timestamp, new SelectResultValue(pagingIdentifiers, dimensions, metrics, getEventHolders()));
}
protected List<EventHolder> getEventHolders()
{
return Lists.newArrayList(pQueue);
}
protected Queue<EventHolder> instantiatePQueue()
{
return new ArrayDeque<>();
}
public static class MergeBuilder extends SelectResultValueBuilder
{
public MergeBuilder(DateTime timestamp, PagingSpec pagingSpec, boolean descending)
{
super(timestamp, pagingSpec, descending);
}
@Override
protected Queue<EventHolder> instantiatePQueue()
{
int threshold = pagingSpec.getThreshold();
return MinMaxPriorityQueue.orderedBy(descending ? COMPARATOR.reversed() : COMPARATOR)
.maximumSize(threshold > 0 ? threshold : Integer.MAX_VALUE)
.create();
}
@Override
protected List<EventHolder> getEventHolders()
{
final List<EventHolder> values = Lists.newArrayListWithCapacity(pQueue.size());
while (!pQueue.isEmpty()) {
EventHolder event = pQueue.remove();
pagingIdentifiers.put(event.getSegmentId(), event.getOffset());
values.add(event);
}
return values;
}
}
}

View File

@ -39,7 +39,6 @@ public interface Cursor
DateTime getTime();
void advance();
void advanceUninterruptibly();
void advanceTo(int offset);
boolean isDone();
boolean isDoneOrInterrupted();
void reset();

View File

@ -429,16 +429,6 @@ public class QueryableIndexCursorSequenceBuilder
cursorOffset.increment();
}
@Override
public void advanceTo(int offset)
{
int count = 0;
while (count < offset && !isDone()) {
advance();
count++;
}
}
@Override
public boolean isDone()
{

View File

@ -344,16 +344,6 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
done = true;
}
@Override
public void advanceTo(int offset)
{
int count = 0;
while (count < offset && !isDone()) {
advance();
count++;
}
}
@Override
public boolean isDone()
{

View File

@ -23,12 +23,10 @@ import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonToken;
import com.fasterxml.jackson.core.ObjectCodec;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.collect.Lists;
import com.google.common.io.Closeables;
import org.apache.commons.io.IOUtils;
@ -59,10 +57,6 @@ import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
import org.apache.druid.query.groupby.ResultRow;
import org.apache.druid.query.select.SelectQueryConfig;
import org.apache.druid.query.select.SelectQueryEngine;
import org.apache.druid.query.select.SelectQueryQueryToolChest;
import org.apache.druid.query.select.SelectQueryRunnerFactory;
import org.apache.druid.query.timeseries.TimeseriesQueryEngine;
import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory;
@ -176,60 +170,6 @@ public class AggregationTestHelper implements Closeable
);
}
public static AggregationTestHelper createSelectQueryAggregationTestHelper(
List<? extends Module> jsonModulesToRegister,
TemporaryFolder tempFolder
)
{
ObjectMapper mapper = TestHelper.makeJsonMapper();
mapper.setInjectableValues(
new InjectableValues.Std().addValue(
SelectQueryConfig.class,
new SelectQueryConfig(true)
)
);
Supplier<SelectQueryConfig> configSupplier = Suppliers.ofInstance(new SelectQueryConfig(true));
SelectQueryQueryToolChest toolchest = new SelectQueryQueryToolChest(
TestHelper.makeJsonMapper(),
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
SelectQueryRunnerFactory factory = new SelectQueryRunnerFactory(
new SelectQueryQueryToolChest(
TestHelper.makeJsonMapper(),
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
new SelectQueryEngine(
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
IndexIO indexIO = new IndexIO(
mapper,
new ColumnConfig()
{
@Override
public int columnCacheSizeBytes()
{
return 0;
}
}
);
return new AggregationTestHelper(
mapper,
new IndexMergerV9(mapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()),
indexIO,
toolchest,
factory,
tempFolder,
jsonModulesToRegister,
Closer.create()
);
}
public static AggregationTestHelper createTimeseriesQueryAggregationTestHelper(
List<? extends Module> jsonModulesToRegister,
TemporaryFolder tempFolder

View File

@ -25,6 +25,7 @@ import com.google.common.io.CharSource;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang.StringUtils;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.MergeSequence;
@ -38,7 +39,8 @@ 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.query.spec.LegacySegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.TestIndex;
@ -75,6 +77,8 @@ public class MultiSegmentScanQueryTest
new ScanQueryEngine(),
new ScanQueryConfig()
);
private static final Interval I_0112_0114 = Intervals.of("2011-01-12/2011-01-14");
public static final QuerySegmentSpec I_0112_0114_SPEC = new LegacySegmentSpec(I_0112_0114);
// time modified version of druid.sample.numeric.tsv
public static final String[] V_0112 = {
@ -182,7 +186,7 @@ public class MultiSegmentScanQueryTest
{
return Druids.newScanQueryBuilder()
.dataSource(new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE))
.intervals(SelectQueryRunnerTest.I_0112_0114_SPEC)
.intervals(I_0112_0114_SPEC)
.batchSize(batchSize)
.columns(Collections.emptyList())
.legacy(false)

View File

@ -1,99 +0,0 @@
/*
* 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.select;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.query.CachingEmitter;
import org.apache.druid.query.DefaultQueryMetricsTest;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.query.Druids;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
public class DefaultSelectQueryMetricsTest
{
/**
* Tests that passed a query {@link DefaultSelectQueryMetrics} produces events with a certain set of dimensions.
*/
@Test
public void testDefaultSelectQueryMetricsQuery()
{
CachingEmitter cachingEmitter = new CachingEmitter();
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
SelectQuery query = Druids
.newSelectQueryBuilder()
.dataSource(QueryRunnerTestHelper.DATA_SOURCE)
.granularity(QueryRunnerTestHelper.DAY_GRAN)
.intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC)
.descending(true)
.pagingSpec(PagingSpec.newSpec(1))
.build();
SelectQueryMetrics queryMetrics = DefaultSelectQueryMetricsFactory.instance().makeMetrics(query);
queryMetrics.query(query);
queryMetrics.reportQueryTime(0).emit(serviceEmitter);
Map<String, Object> actualEvent = cachingEmitter.getLastEmittedEvent().toMap();
Assert.assertEquals(12, actualEvent.size());
Assert.assertTrue(actualEvent.containsKey("feed"));
Assert.assertTrue(actualEvent.containsKey("timestamp"));
Assert.assertEquals("", actualEvent.get("host"));
Assert.assertEquals("", actualEvent.get("service"));
Assert.assertEquals(QueryRunnerTestHelper.DATA_SOURCE, actualEvent.get(DruidMetrics.DATASOURCE));
Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE));
List<Interval> expectedIntervals = QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC.getIntervals();
List<String> expectedStringIntervals =
expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList());
Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL));
Assert.assertEquals("false", actualEvent.get("hasFilters"));
Assert.assertEquals(expectedIntervals.get(0).toDuration().toString(), actualEvent.get("duration"));
Assert.assertEquals("", actualEvent.get(DruidMetrics.ID));
// Metric
Assert.assertEquals("query/time", actualEvent.get("metric"));
Assert.assertEquals(0L, actualEvent.get("value"));
}
@Test
public void testDefaultSelectQueryMetricsMetricNamesAndUnits()
{
CachingEmitter cachingEmitter = new CachingEmitter();
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
SelectQuery query = Druids
.newSelectQueryBuilder()
.dataSource(QueryRunnerTestHelper.DATA_SOURCE)
.granularity(QueryRunnerTestHelper.DAY_GRAN)
.intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC)
.descending(true)
.pagingSpec(PagingSpec.newSpec(1))
.build();
SelectQueryMetrics queryMetrics = DefaultSelectQueryMetricsFactory.instance().makeMetrics(query);
DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics);
}
}

View File

@ -1,356 +0,0 @@
/*
* 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.select;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList;
import com.google.common.io.CharSource;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang.StringUtils;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.Druids;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
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.UnionDataSource;
import org.apache.druid.query.UnionQueryRunner;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.apache.druid.timeline.partition.SingleElementPartitionChunk;
import org.joda.time.Interval;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
/**
*/
@RunWith(Parameterized.class)
public class MultiSegmentSelectQueryTest
{
private static final Supplier<SelectQueryConfig> CONFIG_SUPPLIER = Suppliers.ofInstance(new SelectQueryConfig(true));
private static final SelectQueryQueryToolChest TOOL_CHEST = new SelectQueryQueryToolChest(
new DefaultObjectMapper(),
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
private static final QueryRunnerFactory FACTORY = new SelectQueryRunnerFactory(
TOOL_CHEST,
new SelectQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
// time modified version of druid.sample.numeric.tsv
public static final String[] V_0112 = {
"2011-01-12T00:00:00.000Z\tspot\tautomotive\t1000\t10000.0\t10000.0\t100000\tpreferred\tapreferred\t100.000000",
"2011-01-12T01:00:00.000Z\tspot\tbusiness\t1100\t11000.0\t11000.0\t110000\tpreferred\tbpreferred\t100.000000",
"2011-01-12T02:00:00.000Z\tspot\tentertainment\t1200\t12000.0\t12000.0\t120000\tpreferred\tepreferred\t100.000000",
"2011-01-12T03:00:00.000Z\tspot\thealth\t1300\t13000.0\t13000.0\t130000\tpreferred\thpreferred\t100.000000",
"2011-01-12T04:00:00.000Z\tspot\tmezzanine\t1400\t14000.0\t14000.0\t140000\tpreferred\tmpreferred\t100.000000",
"2011-01-12T05:00:00.000Z\tspot\tnews\t1500\t15000.0\t15000.0\t150000\tpreferred\tnpreferred\t100.000000",
"2011-01-12T06:00:00.000Z\tspot\tpremium\t1600\t16000.0\t16000.0\t160000\tpreferred\tppreferred\t100.000000",
"2011-01-12T07:00:00.000Z\tspot\ttechnology\t1700\t17000.0\t17000.0\t170000\tpreferred\ttpreferred\t100.000000",
"2011-01-12T08:00:00.000Z\tspot\ttravel\t1800\t18000.0\t18000.0\t180000\tpreferred\ttpreferred\t100.000000",
"2011-01-12T09:00:00.000Z\ttotal_market\tmezzanine\t1400\t14000.0\t14000.0\t140000\tpreferred\tmpreferred\t1000.000000",
"2011-01-12T10:00:00.000Z\ttotal_market\tpremium\t1600\t16000.0\t16000.0\t160000\tpreferred\tppreferred\t1000.000000",
"2011-01-12T11:00:00.000Z\tupfront\tmezzanine\t1400\t14000.0\t14000.0\t140000\tpreferred\tmpreferred\t800.000000\tvalue",
"2011-01-12T12:00:00.000Z\tupfront\tpremium\t1600\t16000.0\t16000.0\t160000\tpreferred\tppreferred\t800.000000\tvalue"
};
public static final String[] V_0113 = {
"2011-01-13T00:00:00.000Z\tspot\tautomotive\t1000\t10000.0\t10000.0\t100000\tpreferred\tapreferred\t94.874713",
"2011-01-13T01:00:00.000Z\tspot\tbusiness\t1100\t11000.0\t11000.0\t110000\tpreferred\tbpreferred\t103.629399",
"2011-01-13T02:00:00.000Z\tspot\tentertainment\t1200\t12000.0\t12000.0\t120000\tpreferred\tepreferred\t110.087299",
"2011-01-13T03:00:00.000Z\tspot\thealth\t1300\t13000.0\t13000.0\t130000\tpreferred\thpreferred\t114.947403",
"2011-01-13T04:00:00.000Z\tspot\tmezzanine\t1400\t14000.0\t14000.0\t140000\tpreferred\tmpreferred\t104.465767",
"2011-01-13T05:00:00.000Z\tspot\tnews\t1500\t15000.0\t15000.0\t150000\tpreferred\tnpreferred\t102.851683",
"2011-01-13T06:00:00.000Z\tspot\tpremium\t1600\t16000.0\t16000.0\t160000\tpreferred\tppreferred\t108.863011",
"2011-01-13T07:00:00.000Z\tspot\ttechnology\t1700\t17000.0\t17000.0\t170000\tpreferred\ttpreferred\t111.356672",
"2011-01-13T08:00:00.000Z\tspot\ttravel\t1800\t18000.0\t18000.0\t180000\tpreferred\ttpreferred\t106.236928",
"2011-01-13T09:00:00.000Z\ttotal_market\tmezzanine\t1400\t14000.0\t14000.0\t140000\tpreferred\tmpreferred\t1040.945505",
"2011-01-13T10:00:00.000Z\ttotal_market\tpremium\t1600\t16000.0\t16000.0\t160000\tpreferred\tppreferred\t1689.012875",
"2011-01-13T11:00:00.000Z\tupfront\tmezzanine\t1400\t14000.0\t14000.0\t140000\tpreferred\tmpreferred\t826.060182\tvalue",
"2011-01-13T12:00:00.000Z\tupfront\tpremium\t1600\t16000.0\t16000.0\t160000\tpreferred\tppreferred\t1564.617729\tvalue"
};
public static final String[] V_OVERRIDE = {
"2011-01-12T04:00:00.000Z\tspot\tautomotive\t1000\t10000.0\t10000.0\t100000\tpreferred\tapreferred\t999.000000",
"2011-01-12T05:00:00.000Z\tspot\tbusiness\t1100\t11000.0\t11000.0\t110000\tpreferred\tbpreferred\t999.000000",
"2011-01-12T06:00:00.000Z\tspot\tentertainment\t1200\t12000.0\t12000.0\t120000\tpreferred\tepreferred\t999.000000",
"2011-01-12T07:00:00.000Z\tspot\thealth\t1300\t13000.0\t13000.0\t130000\tpreferred\thpreferred\t999.000000"
};
private static Segment segment0;
private static Segment segment1;
private static Segment segment_override; // this makes segment0 split into three logical segments
private static List<String> segmentIdentifiers;
private static QueryRunner runner;
@BeforeClass
public static void setup() throws IOException
{
CharSource v_0112 = CharSource.wrap(StringUtils.join(V_0112, "\n"));
CharSource v_0113 = CharSource.wrap(StringUtils.join(V_0113, "\n"));
CharSource v_override = CharSource.wrap(StringUtils.join(V_OVERRIDE, "\n"));
IncrementalIndex index0 = TestIndex.loadIncrementalIndex(newIndex("2011-01-12T00:00:00.000Z"), v_0112);
IncrementalIndex index1 = TestIndex.loadIncrementalIndex(newIndex("2011-01-13T00:00:00.000Z"), v_0113);
IncrementalIndex index2 = TestIndex.loadIncrementalIndex(newIndex("2011-01-12T04:00:00.000Z"), v_override);
segment0 = new IncrementalIndexSegment(index0, makeIdentifier(index0, "v1"));
segment1 = new IncrementalIndexSegment(index1, makeIdentifier(index1, "v1"));
segment_override = new IncrementalIndexSegment(index2, makeIdentifier(index2, "v2"));
VersionedIntervalTimeline<String, ReferenceCountingSegment> timeline = new VersionedIntervalTimeline<>(StringComparators.LEXICOGRAPHIC);
timeline.add(
index0.getInterval(),
"v1",
new SingleElementPartitionChunk<>(ReferenceCountingSegment.wrapRootGenerationSegment(segment0))
);
timeline.add(
index1.getInterval(),
"v1",
new SingleElementPartitionChunk<>(ReferenceCountingSegment.wrapRootGenerationSegment(segment1))
);
timeline.add(
index2.getInterval(),
"v2",
new SingleElementPartitionChunk<>(ReferenceCountingSegment.wrapRootGenerationSegment(segment_override))
);
segmentIdentifiers = new ArrayList<>();
for (TimelineObjectHolder<String, ?> holder : timeline.lookup(Intervals.of("2011-01-12/2011-01-14"))) {
segmentIdentifiers.add(makeIdentifier(holder.getInterval(), holder.getVersion()).toString());
}
runner = QueryRunnerTestHelper.makeFilteringQueryRunner(timeline, FACTORY);
}
private static SegmentId makeIdentifier(IncrementalIndex index, String version)
{
return makeIdentifier(index.getInterval(), version);
}
private static SegmentId makeIdentifier(Interval interval, String version)
{
return SegmentId.of(QueryRunnerTestHelper.DATA_SOURCE, interval, version, NoneShardSpec.instance());
}
private static IncrementalIndex newIndex(String minTimeStamp)
{
return newIndex(minTimeStamp, 10000);
}
private static IncrementalIndex newIndex(String minTimeStamp, int maxRowCount)
{
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
.withMinTimestamp(DateTimes.of(minTimeStamp).getMillis())
.withQueryGranularity(Granularities.HOUR)
.withMetrics(TestIndex.METRIC_AGGS)
.build();
return new IncrementalIndex.Builder()
.setIndexSchema(schema)
.setMaxRowCount(maxRowCount)
.buildOnheap();
}
@AfterClass
public static void clear()
{
IOUtils.closeQuietly(segment0);
IOUtils.closeQuietly(segment1);
IOUtils.closeQuietly(segment_override);
}
@Parameterized.Parameters(name = "fromNext={0}")
public static Iterable<Object[]> constructorFeeder()
{
return QueryRunnerTestHelper.cartesian(Arrays.asList(false, true));
}
private final boolean fromNext;
public MultiSegmentSelectQueryTest(boolean fromNext)
{
this.fromNext = fromNext;
}
private Druids.SelectQueryBuilder newBuilder()
{
return Druids.newSelectQueryBuilder()
.dataSource(new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE))
.intervals(SelectQueryRunnerTest.I_0112_0114_SPEC)
.granularity(QueryRunnerTestHelper.ALL_GRAN)
.dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.DIMENSIONS))
.pagingSpec(PagingSpec.newSpec(3));
}
@Test
public void testAllGranularity()
{
runAllGranularityTest(
newBuilder().build(),
new int[][]{
{2, -1, -1, -1, 3}, {3, 1, -1, -1, 3}, {-1, 3, 0, -1, 3}, {-1, -1, 3, -1, 3}, {-1, -1, 4, 1, 3},
{-1, -1, -1, 4, 3}, {-1, -1, -1, 7, 3}, {-1, -1, -1, 10, 3}, {-1, -1, -1, 12, 2}, {-1, -1, -1, 13, 0}
}
);
runAllGranularityTest(
newBuilder().descending(true).build(),
new int[][]{
{0, 0, 0, -3, 3}, {0, 0, 0, -6, 3}, {0, 0, 0, -9, 3}, {0, 0, 0, -12, 3}, {0, 0, -2, -13, 3},
{0, 0, -5, 0, 3}, {0, -3, 0, 0, 3}, {-2, -4, 0, 0, 3}, {-4, 0, 0, 0, 2}, {-5, 0, 0, 0, 0}
}
);
}
private void runAllGranularityTest(SelectQuery query, int[][] expectedOffsets)
{
for (int[] expected : expectedOffsets) {
List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
Assert.assertEquals(1, results.size());
SelectResultValue value = results.get(0).getValue();
Map<String, Integer> pagingIdentifiers = value.getPagingIdentifiers();
Map<String, Integer> merged = PagingSpec.merge(Collections.singletonList(pagingIdentifiers));
for (int i = 0; i < 4; i++) {
if (query.isDescending() ^ expected[i] >= 0) {
Assert.assertEquals(expected[i], pagingIdentifiers.get(segmentIdentifiers.get(i)).intValue());
}
}
Assert.assertEquals(expected[4], value.getEvents().size());
query = query.withPagingSpec(toNextCursor(merged, query, 3));
}
}
@Test
public void testDayGranularity()
{
runDayGranularityTest(
newBuilder().granularity(QueryRunnerTestHelper.DAY_GRAN).build(),
new int[][]{
{2, -1, -1, 2, 3, 0, 0, 3}, {3, 1, -1, 5, 1, 2, 0, 3}, {-1, 3, 0, 8, 0, 2, 1, 3},
{-1, -1, 3, 11, 0, 0, 3, 3}, {-1, -1, 4, 12, 0, 0, 1, 1}, {-1, -1, 5, 13, 0, 0, 0, 0}
}
);
runDayGranularityTest(
newBuilder().granularity(QueryRunnerTestHelper.DAY_GRAN).descending(true).build(),
new int[][]{
{0, 0, -3, -3, 0, 0, 3, 3}, {0, -1, -5, -6, 0, 1, 2, 3}, {0, -4, 0, -9, 0, 3, 0, 3},
{-3, 0, 0, -12, 3, 0, 0, 3}, {-4, 0, 0, -13, 1, 0, 0, 1}, {-5, 0, 0, -14, 0, 0, 0, 0}
}
);
}
private void runDayGranularityTest(SelectQuery query, int[][] expectedOffsets)
{
for (int[] expected : expectedOffsets) {
List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query)).toList();
Assert.assertEquals(2, results.size());
SelectResultValue value0 = results.get(0).getValue();
SelectResultValue value1 = results.get(1).getValue();
Map<String, Integer> pagingIdentifiers0 = value0.getPagingIdentifiers();
Map<String, Integer> pagingIdentifiers1 = value1.getPagingIdentifiers();
Map<String, Integer> merged = PagingSpec.merge(Arrays.asList(pagingIdentifiers0, pagingIdentifiers1));
for (int i = 0; i < 4; i++) {
if (query.isDescending() ^ expected[i] >= 0) {
Assert.assertEquals(expected[i], merged.get(segmentIdentifiers.get(i)).intValue());
}
}
query = query.withPagingSpec(toNextCursor(merged, query, 3));
}
}
@Test
public void testPagingIdentifiersForUnionDatasource()
{
Druids.SelectQueryBuilder selectQueryBuilder = Druids
.newSelectQueryBuilder()
.dataSource(
new UnionDataSource(
ImmutableList.of(
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
new TableDataSource("testing-2")
)
)
)
.intervals(SelectQueryRunnerTest.I_0112_0114_SPEC)
.granularity(QueryRunnerTestHelper.ALL_GRAN)
.dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.DIMENSIONS))
.pagingSpec(PagingSpec.newSpec(3));
SelectQuery query = selectQueryBuilder.build();
QueryRunner unionQueryRunner = new UnionQueryRunner(runner);
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)).toList();
}
private PagingSpec toNextCursor(Map<String, Integer> merged, SelectQuery query, int threshold)
{
if (!fromNext) {
merged = PagingSpec.next(merged, query.isDescending());
}
return new PagingSpec(merged, threshold, fromNext);
}
}

View File

@ -1,71 +0,0 @@
/*
* 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.select;
import com.google.common.primitives.Ints;
import org.junit.Assert;
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
public class PagingOffsetTest
{
@Test
public void testZeroThreshold()
{
PagingOffset offset = PagingOffset.of(PagingOffset.toOffset(3, false), 0);
Assert.assertEquals(3, offset.startOffset());
Assert.assertEquals(3, offset.startDelta());
Assert.assertArrayEquals(new int[]{}, toArray(offset));
offset = PagingOffset.of(PagingOffset.toOffset(3, true), 0);
Assert.assertEquals(-4, offset.startOffset());
Assert.assertEquals(3, offset.startDelta());
Assert.assertArrayEquals(new int[]{}, toArray(offset));
}
@Test
public void testAscending()
{
PagingOffset offset = PagingOffset.of(PagingOffset.toOffset(3, false), 3);
Assert.assertEquals(3, offset.startOffset());
Assert.assertEquals(3, offset.startDelta());
Assert.assertArrayEquals(new int[]{3, 4, 5}, toArray(offset));
}
@Test
public void testDescending()
{
PagingOffset offset = PagingOffset.of(PagingOffset.toOffset(3, true), 3);
Assert.assertEquals(-4, offset.startOffset());
Assert.assertEquals(3, offset.startDelta());
Assert.assertArrayEquals(new int[]{-4, -5, -6}, toArray(offset));
}
private int[] toArray(PagingOffset offset)
{
List<Integer> ints = new ArrayList<>();
for (; offset.hasNext(); offset.next()) {
ints.add(offset.current());
}
return Ints.toArray(ints);
}
}

View File

@ -1,277 +0,0 @@
/*
* 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.select;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.Result;
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Set;
/**
*/
public class SelectBinaryFnTest
{
private static final String SEGMENT_ID1 = "testSegment1";
private static final String SEGMENT_ID2 = "testSegment2";
@Test
public void testApply()
{
SelectBinaryFn binaryFn = new SelectBinaryFn(Granularities.ALL, new PagingSpec(null, 5), false);
Result<SelectResultValue> res1 = new Result<>(
DateTimes.of("2013-01-01"),
new SelectResultValue(
ImmutableMap.of(),
Sets.newHashSet("first", "fourth"),
Sets.newHashSet("sixth"),
Arrays.asList(
new EventHolder(
SEGMENT_ID1,
0,
ImmutableMap.of(
EventHolder.TIMESTAMP_KEY,
DateTimes.of("2013-01-01T00"),
"dim",
"first"
)
),
new EventHolder(
SEGMENT_ID1,
1,
ImmutableMap.of(
EventHolder.TIMESTAMP_KEY,
DateTimes.of("2013-01-01T03"),
"dim",
"fourth"
)
),
new EventHolder(
SEGMENT_ID1,
2,
ImmutableMap.of(
EventHolder.TIMESTAMP_KEY,
DateTimes.of("2013-01-01T05"),
"dim",
"sixth"
)
)
)
)
);
Result<SelectResultValue> res2 = new Result<>(
DateTimes.of("2013-01-01"),
new SelectResultValue(
ImmutableMap.of(),
Sets.newHashSet("second", "third"),
Sets.newHashSet("fifth"),
Arrays.asList(
new EventHolder(
SEGMENT_ID2,
0,
ImmutableMap.of(
EventHolder.TIMESTAMP_KEY,
DateTimes.of("2013-01-01T00"),
"dim",
"second"
)
),
new EventHolder(
SEGMENT_ID2,
1,
ImmutableMap.of(
EventHolder.TIMESTAMP_KEY,
DateTimes.of("2013-01-01T02"),
"dim",
"third"
)
),
new EventHolder(
SEGMENT_ID2,
2,
ImmutableMap.of(
EventHolder.TIMESTAMP_KEY,
DateTimes.of("2013-01-01T04"),
"dim",
"fifth"
)
)
)
)
);
Result<SelectResultValue> merged = binaryFn.apply(res1, res2);
Assert.assertEquals(res1.getTimestamp(), merged.getTimestamp());
LinkedHashMap<String, Integer> expectedPageIds = Maps.newLinkedHashMap();
expectedPageIds.put(SEGMENT_ID1, 1);
expectedPageIds.put(SEGMENT_ID2, 2);
Iterator<String> exSegmentIter = expectedPageIds.keySet().iterator();
Iterator<String> acSegmentIter = merged.getValue().getPagingIdentifiers().keySet().iterator();
verifyIters(exSegmentIter, acSegmentIter);
Iterator<Integer> exOffsetIter = expectedPageIds.values().iterator();
Iterator<Integer> acOffsetIter = merged.getValue().getPagingIdentifiers().values().iterator();
verifyIters(exOffsetIter, acOffsetIter);
List<EventHolder> exEvents = Arrays.asList(
new EventHolder(
SEGMENT_ID1,
0,
ImmutableMap.of(
EventHolder.TIMESTAMP_KEY,
DateTimes.of("2013-01-01T00"), "dim", "first"
)
),
new EventHolder(
SEGMENT_ID2,
0,
ImmutableMap.of(
EventHolder.TIMESTAMP_KEY,
DateTimes.of("2013-01-01T00"),
"dim",
"second"
)
),
new EventHolder(
SEGMENT_ID2,
1,
ImmutableMap.of(
EventHolder.TIMESTAMP_KEY,
DateTimes.of("2013-01-01T02"),
"dim",
"third"
)
),
new EventHolder(
SEGMENT_ID1,
1,
ImmutableMap.of(
EventHolder.TIMESTAMP_KEY,
DateTimes.of("2013-01-01T03"),
"dim",
"fourth"
)
),
new EventHolder(
SEGMENT_ID2,
2,
ImmutableMap.of(
EventHolder.TIMESTAMP_KEY,
DateTimes.of("2013-01-01T04"),
"dim",
"fifth"
)
)
);
List<EventHolder> acEvents = merged.getValue().getEvents();
verifyEvents(exEvents, acEvents);
}
@Test
public void testColumnMerge()
{
SelectBinaryFn binaryFn = new SelectBinaryFn(Granularities.ALL, new PagingSpec(null, 5), false);
Result<SelectResultValue> res1 = new Result<>(
DateTimes.of("2013-01-01"),
new SelectResultValue(
ImmutableMap.of(),
Sets.newHashSet("first", "second", "fourth"),
Sets.newHashSet("eight", "nineth"),
Collections.singletonList(
new EventHolder(
SEGMENT_ID1,
0,
ImmutableMap.of(EventHolder.TIMESTAMP_KEY, DateTimes.of("2013-01-01T00"), "dim", "first")
)
)
)
);
Result<SelectResultValue> res2 = new Result<>(
DateTimes.of("2013-01-01"),
new SelectResultValue(
ImmutableMap.of(),
Sets.newHashSet("third", "second", "fifth"),
Sets.newHashSet("seventh"),
Collections.singletonList(
new EventHolder(
SEGMENT_ID2,
0,
ImmutableMap.of(EventHolder.TIMESTAMP_KEY, DateTimes.of("2013-01-01T00"), "dim", "second")
)
)
)
);
Result<SelectResultValue> merged = binaryFn.apply(res1, res2);
Set<String> exDimensions = Sets.newHashSet("first", "second", "fourth", "third", "fifth");
Set<String> exMetrics = Sets.newHashSet("eight", "nineth", "seventh");
Set<String> acDimensions = merged.getValue().getDimensions();
Set<String> acMetrics = merged.getValue().getMetrics();
Assert.assertEquals(exDimensions, acDimensions);
Assert.assertEquals(exMetrics, acMetrics);
}
private void verifyIters(Iterator iter1, Iterator iter2)
{
while (iter1.hasNext()) {
Assert.assertEquals(iter1.next(), iter2.next());
}
if (iter2.hasNext()) {
throw new ISE("This should be empty!");
}
}
private void verifyEvents(List<EventHolder> events1, List<EventHolder> events2)
{
Iterator<EventHolder> ex = events1.iterator();
Iterator<EventHolder> ac = events2.iterator();
verifyIters(ex, ac);
}
}

View File

@ -1,58 +0,0 @@
/*
* 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.select;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.junit.Assert;
import org.junit.Test;
public class SelectQueryConfigTest
{
private final ObjectMapper MAPPER = new DefaultObjectMapper();
private final ImmutableMap<String, String> CONFIG_MAP = ImmutableMap
.<String, String>builder()
.put(SelectQueryConfig.ENABLE_FROM_NEXT_DEFAULT, "false")
.build();
private final ImmutableMap<String, String> CONFIG_MAP2 = ImmutableMap
.<String, String>builder()
.put(SelectQueryConfig.ENABLE_FROM_NEXT_DEFAULT, "true")
.build();
private final ImmutableMap<String, String> CONFIG_MAP_EMPTY = ImmutableMap
.<String, String>builder()
.build();
@Test
public void testSerde()
{
final SelectQueryConfig config = MAPPER.convertValue(CONFIG_MAP, SelectQueryConfig.class);
Assert.assertEquals(false, config.getEnableFromNextDefault());
final SelectQueryConfig config2 = MAPPER.convertValue(CONFIG_MAP2, SelectQueryConfig.class);
Assert.assertEquals(true, config2.getEnableFromNextDefault());
final SelectQueryConfig configEmpty = MAPPER.convertValue(CONFIG_MAP_EMPTY, SelectQueryConfig.class);
Assert.assertEquals(true, configEmpty.getEnableFromNextDefault());
}
}

View File

@ -1,72 +0,0 @@
/*
* 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.select;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.query.CacheStrategy;
import org.apache.druid.query.Druids;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.Result;
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
public class SelectQueryQueryToolChestTest
{
private static final Supplier<SelectQueryConfig> CONFIG_SUPPLIER = Suppliers.ofInstance(new SelectQueryConfig(true));
private static final SelectQueryQueryToolChest TOOL_CHEST = new SelectQueryQueryToolChest(
new DefaultObjectMapper(),
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
@Test
public void testComputeCacheKeyWithDifferentSortOrer()
{
final SelectQuery query1 = Druids.newSelectQueryBuilder()
.dataSource("dummy")
.dimensions(Collections.singletonList("testDim"))
.intervals(SelectQueryRunnerTest.I_0112_0114_SPEC)
.granularity(QueryRunnerTestHelper.ALL_GRAN)
.pagingSpec(PagingSpec.newSpec(3))
.descending(false)
.build();
final SelectQuery query2 = Druids.newSelectQueryBuilder()
.dataSource("dummy")
.dimensions(Collections.singletonList("testDim"))
.intervals(SelectQueryRunnerTest.I_0112_0114_SPEC)
.granularity(QueryRunnerTestHelper.ALL_GRAN)
.pagingSpec(PagingSpec.newSpec(3))
.descending(true)
.build();
final CacheStrategy<Result<SelectResultValue>, Object, SelectQuery> strategy1 = TOOL_CHEST.getCacheStrategy(query1);
Assert.assertNotNull(strategy1);
final CacheStrategy<Result<SelectResultValue>, Object, SelectQuery> strategy2 = TOOL_CHEST.getCacheStrategy(query2);
Assert.assertNotNull(strategy2);
Assert.assertFalse(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2)));
}
}

View File

@ -1,155 +0,0 @@
/*
* 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.select;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.spec.LegacySegmentSpec;
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
/**
*/
public class SelectQuerySpecTest
{
private final ObjectMapper objectMapper = new DefaultObjectMapper();
{
objectMapper.setInjectableValues(
new InjectableValues.Std().addValue(SelectQueryConfig.class, new SelectQueryConfig(true))
);
}
@Test
public void testSerializationLegacyString() throws Exception
{
String legacy =
"{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},"
+ "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},"
+ "\"descending\":true,"
+ "\"filter\":null,"
+ "\"granularity\":{\"type\":\"all\"},"
+ "\"dimensions\":[\"market\",\"quality\"],"
+ "\"metrics\":[\"index\"],"
+ "\"virtualColumns\":null,"
+ "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3},"
+ "\"context\":null}";
String current =
"{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},"
+ "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},"
+ "\"descending\":true,"
+ "\"filter\":null,"
+ "\"granularity\":{\"type\":\"all\"},"
+ "\"dimensions\":"
+ "[{\"type\":\"default\",\"dimension\":\"market\",\"outputName\":\"market\",\"outputType\":\"STRING\"},"
+ "{\"type\":\"default\",\"dimension\":\"quality\",\"outputName\":\"quality\",\"outputType\":\"STRING\"}],"
+ "\"metrics\":[\"index\"],"
+ "\"virtualColumns\":[],"
+ "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3,\"fromNext\":true},"
+ "\"context\":null}";
SelectQuery query = new SelectQuery(
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
new LegacySegmentSpec(Intervals.of("2011-01-12/2011-01-14")),
true,
null,
QueryRunnerTestHelper.ALL_GRAN,
DefaultDimensionSpec.toSpec(Arrays.asList("market", "quality")),
Collections.singletonList("index"),
null,
new PagingSpec(null, 3, null),
null
);
String actual = objectMapper.writeValueAsString(query);
Assert.assertEquals(current, actual);
Assert.assertEquals(query, objectMapper.readValue(actual, SelectQuery.class));
Assert.assertEquals(query, objectMapper.readValue(legacy, SelectQuery.class));
}
@Test
public void testPagingSpecFromNext() throws Exception
{
String baseQueryJson =
"{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},"
+ "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},"
+ "\"descending\":true,"
+ "\"filter\":null,"
+ "\"granularity\":{\"type\":\"all\"},"
+ "\"dimensions\":"
+ "[{\"type\":\"default\",\"dimension\":\"market\",\"outputName\":\"market\",\"outputType\":\"STRING\"},"
+ "{\"type\":\"default\",\"dimension\":\"quality\",\"outputName\":\"quality\",\"outputType\":\"STRING\"}],"
+ "\"metrics\":[\"index\"],"
+ "\"virtualColumns\":[],";
String withFalse =
baseQueryJson
+ "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3,\"fromNext\":false},"
+ "\"context\":null}";
String withTrue =
baseQueryJson
+ "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3,\"fromNext\":true},"
+ "\"context\":null}";
SelectQuery queryWithNull = new SelectQuery(
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
new LegacySegmentSpec(Intervals.of("2011-01-12/2011-01-14")),
true,
null,
QueryRunnerTestHelper.ALL_GRAN,
DefaultDimensionSpec.toSpec(Arrays.asList("market", "quality")),
Collections.singletonList("index"),
null,
new PagingSpec(null, 3, null),
null
);
SelectQuery queryWithFalse = queryWithNull.withPagingSpec(
new PagingSpec(null, 3, false)
);
SelectQuery queryWithTrue = queryWithNull.withPagingSpec(
new PagingSpec(null, 3, true)
);
String actualWithNull = objectMapper.writeValueAsString(queryWithNull);
Assert.assertEquals(withTrue, actualWithNull);
String actualWithFalse = objectMapper.writeValueAsString(queryWithFalse);
Assert.assertEquals(withFalse, actualWithFalse);
String actualWithTrue = objectMapper.writeValueAsString(queryWithTrue);
Assert.assertEquals(withTrue, actualWithTrue);
Assert.assertEquals(queryWithNull, objectMapper.readValue(actualWithNull, SelectQuery.class));
Assert.assertEquals(queryWithFalse, objectMapper.readValue(actualWithFalse, SelectQuery.class));
Assert.assertEquals(queryWithTrue, objectMapper.readValue(actualWithTrue, SelectQuery.class));
}
}

View File

@ -0,0 +1,63 @@
/*
* 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.select;
import com.fasterxml.jackson.databind.JsonMappingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.StringUtils;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
public class SelectQueryTest
{
private static final String SOME_QUERY_THAT_IS_DOOMED_TO_FAIL =
"{\n"
+ " \"queryType\": \"select\",\n"
+ " \"dataSource\": \"wikipedia\",\n"
+ " \"descending\": \"false\",\n"
+ " \"dimensions\":[],\n"
+ " \"metrics\":[],\n"
+ " \"granularity\": \"all\",\n"
+ " \"intervals\": [\n"
+ " \"2013-01-01/2013-01-02\"\n"
+ " ],\n"
+ " \"pagingSpec\":{\"pagingIdentifiers\": {}, \"threshold\":5}\n"
+ " }";
private final ObjectMapper objectMapper = new DefaultObjectMapper();
@Rule
public ExpectedException expectedException = ExpectedException.none();
@Test
public void testSerde() throws Exception
{
final String exceptionMessage =
StringUtils.format(
"Instantiation of [simple type, class org.apache.druid.query.select.SelectQuery] value failed: %s",
SelectQuery.REMOVED_ERROR_MESSAGE
);
expectedException.expect(JsonMappingException.class);
expectedException.expectMessage(exceptionMessage);
objectMapper.readValue(SOME_QUERY_THAT_IS_DOOMED_TO_FAIL, SelectQuery.class);
}
}

View File

@ -55,7 +55,7 @@ public class CacheConfig
private int maxEntrySize = 1_000_000;
@JsonProperty
private List<String> unCacheable = ImmutableList.of(Query.SELECT);
private List<String> unCacheable = ImmutableList.of();
@JsonProperty
private int resultLevelCacheLimit = Integer.MAX_VALUE;

View File

@ -36,8 +36,6 @@ import org.apache.druid.query.scan.ScanQuery;
import org.apache.druid.query.scan.ScanQueryRunnerFactory;
import org.apache.druid.query.search.SearchQuery;
import org.apache.druid.query.search.SearchQueryRunnerFactory;
import org.apache.druid.query.select.SelectQuery;
import org.apache.druid.query.select.SelectQueryRunnerFactory;
import org.apache.druid.query.timeboundary.TimeBoundaryQuery;
import org.apache.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory;
import org.apache.druid.query.timeseries.TimeseriesQuery;
@ -60,7 +58,6 @@ public class QueryRunnerFactoryModule extends QueryToolChestModule
.put(SegmentMetadataQuery.class, SegmentMetadataQueryRunnerFactory.class)
.put(GroupByQuery.class, GroupByQueryRunnerFactory.class)
.put(ScanQuery.class, ScanQueryRunnerFactory.class)
.put(SelectQuery.class, SelectQueryRunnerFactory.class)
.put(TopNQuery.class, TopNQueryRunnerFactory.class)
.put(DataSourceMetadataQuery.class, DataSourceMetadataQueryRunnerFactory.class)
.build();

View File

@ -49,11 +49,6 @@ import org.apache.druid.query.search.SearchQuery;
import org.apache.druid.query.search.SearchQueryConfig;
import org.apache.druid.query.search.SearchQueryMetricsFactory;
import org.apache.druid.query.search.SearchQueryQueryToolChest;
import org.apache.druid.query.select.DefaultSelectQueryMetricsFactory;
import org.apache.druid.query.select.SelectQuery;
import org.apache.druid.query.select.SelectQueryConfig;
import org.apache.druid.query.select.SelectQueryMetricsFactory;
import org.apache.druid.query.select.SelectQueryQueryToolChest;
import org.apache.druid.query.timeboundary.TimeBoundaryQuery;
import org.apache.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest;
import org.apache.druid.query.timeseries.DefaultTimeseriesQueryMetricsFactory;
@ -76,7 +71,6 @@ public class QueryToolChestModule implements Module
public static final String GROUPBY_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.groupBy.queryMetricsFactory";
public static final String TIMESERIES_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.timeseries.queryMetricsFactory";
public static final String TOPN_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.topN.queryMetricsFactory";
public static final String SELECT_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.select.queryMetricsFactory";
public static final String SEARCH_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.search.queryMetricsFactory";
public final Map<Class<? extends Query>, Class<? extends QueryToolChest>> mappings =
@ -87,7 +81,6 @@ public class QueryToolChestModule implements Module
.put(SegmentMetadataQuery.class, SegmentMetadataQueryQueryToolChest.class)
.put(GroupByQuery.class, GroupByQueryQueryToolChest.class)
.put(ScanQuery.class, ScanQueryQueryToolChest.class)
.put(SelectQuery.class, SelectQueryQueryToolChest.class)
.put(TopNQuery.class, TopNQueryQueryToolChest.class)
.put(DataSourceMetadataQuery.class, DataSourceQueryQueryToolChest.class)
.build();
@ -109,7 +102,6 @@ public class QueryToolChestModule implements Module
JsonConfigProvider.bind(binder, "druid.query.search", SearchQueryConfig.class);
JsonConfigProvider.bind(binder, "druid.query.topN", TopNQueryConfig.class);
JsonConfigProvider.bind(binder, "druid.query.segmentMetadata", SegmentMetadataQueryConfig.class);
JsonConfigProvider.bind(binder, "druid.query.select", SelectQueryConfig.class);
JsonConfigProvider.bind(binder, "druid.query.scan", ScanQueryConfig.class);
PolyBind.createChoice(
@ -156,17 +148,6 @@ public class QueryToolChestModule implements Module
.addBinding("default")
.to(DefaultTopNQueryMetricsFactory.class);
PolyBind.createChoice(
binder,
SELECT_QUERY_METRICS_FACTORY_PROPERTY,
Key.get(SelectQueryMetricsFactory.class),
Key.get(DefaultSelectQueryMetricsFactory.class)
);
PolyBind
.optionBinder(binder, Key.get(SelectQueryMetricsFactory.class))
.addBinding("default")
.to(DefaultSelectQueryMetricsFactory.class);
PolyBind.createChoice(
binder,
SEARCH_QUERY_METRICS_FACTORY_PROPERTY,

View File

@ -25,6 +25,7 @@ import com.google.common.collect.Maps;
import org.apache.druid.data.input.Firehose;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
@ -33,7 +34,6 @@ import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.java.util.common.guava.Yielders;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.select.EventHolder;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.Cursor;
@ -132,7 +132,7 @@ public class IngestSegmentFirehose implements Firehose
{
final Map<String, Object> theEvent = Maps.newLinkedHashMap();
final long timestamp = timestampColumnSelector.getLong();
theEvent.put(EventHolder.TIMESTAMP_KEY, DateTimes.utc(timestamp));
theEvent.put(TimestampSpec.DEFAULT_COLUMN, DateTimes.utc(timestamp));
for (Map.Entry<String, DimensionSelector> dimSelector :
dimSelectors.entrySet()) {

View File

@ -20,8 +20,6 @@
package org.apache.druid.client;
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.collect.Lists;
import com.google.common.collect.Ordering;
@ -48,7 +46,6 @@ 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;
import org.apache.druid.timeline.VersionedIntervalTimeline;
@ -74,11 +71,8 @@ import java.util.concurrent.Executor;
public class CachingClusteredClientFunctionalityTest
{
private static final ObjectMapper OBJECT_MAPPER = CachingClusteredClientTestUtils.createObjectMapper();
private static final Supplier<SelectQueryConfig> SELECT_CONFIG_SUPPLIER = Suppliers.ofInstance(
new SelectQueryConfig(true)
);
private static final Pair<QueryToolChestWarehouse, Closer> WAREHOUSE_AND_CLOSER = CachingClusteredClientTestUtils
.createWarehouse(OBJECT_MAPPER, SELECT_CONFIG_SUPPLIER);
.createWarehouse(OBJECT_MAPPER);
private static final QueryToolChestWarehouse WAREHOUSE = WAREHOUSE_AND_CLOSER.lhs;
private static final Closer RESOURCE_CLOSER = WAREHOUSE_AND_CLOSER.rhs;

View File

@ -24,14 +24,11 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
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.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing;
import com.google.common.util.concurrent.ForwardingListeningExecutorService;
@ -106,12 +103,6 @@ import org.apache.druid.query.search.SearchQuery;
import org.apache.druid.query.search.SearchQueryConfig;
import org.apache.druid.query.search.SearchQueryQueryToolChest;
import org.apache.druid.query.search.SearchResultValue;
import org.apache.druid.query.select.EventHolder;
import org.apache.druid.query.select.PagingSpec;
import org.apache.druid.query.select.SelectQuery;
import org.apache.druid.query.select.SelectQueryConfig;
import org.apache.druid.query.select.SelectQueryQueryToolChest;
import org.apache.druid.query.select.SelectResultValue;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
import org.apache.druid.query.timeboundary.TimeBoundaryQuery;
@ -158,7 +149,6 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentLinkedDeque;
@ -254,11 +244,8 @@ public class CachingClusteredClientTest
private static final DateTimeZone TIMEZONE = DateTimes.inferTzFromString("America/Los_Angeles");
private static final Granularity PT1H_TZ_GRANULARITY = new PeriodGranularity(new Period("PT1H"), null, TIMEZONE);
private static final String TOP_DIM = "a_dim";
private static final Supplier<SelectQueryConfig> SELECT_CONFIG_SUPPLIER = Suppliers.ofInstance(
new SelectQueryConfig(true)
);
private static final Pair<QueryToolChestWarehouse, Closer> WAREHOUSE_AND_CLOSER = CachingClusteredClientTestUtils
.createWarehouse(JSON_MAPPER, SELECT_CONFIG_SUPPLIER);
.createWarehouse(JSON_MAPPER);
private static final QueryToolChestWarehouse WAREHOUSE = WAREHOUSE_AND_CLOSER.lhs;
private static final Closer RESOURCE_CLOSER = WAREHOUSE_AND_CLOSER.rhs;
@ -1255,183 +1242,6 @@ public class CachingClusteredClientTest
);
}
@Test
public void testSelectCaching()
{
final Set<String> dimensions = Sets.newHashSet("a");
final Set<String> metrics = Sets.newHashSet("rows");
Druids.SelectQueryBuilder builder = Druids.newSelectQueryBuilder()
.dataSource(DATA_SOURCE)
.intervals(SEG_SPEC)
.filters(DIM_FILTER)
.granularity(GRANULARITY)
.dimensions(Collections.singletonList("a"))
.metrics(Collections.singletonList("rows"))
.pagingSpec(new PagingSpec(null, 3))
.context(CONTEXT);
testQueryCaching(
getDefaultQueryRunner(),
builder.build(),
Intervals.of("2011-01-01/2011-01-02"),
makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-01"), ImmutableMap.of("a", "b", "rows", 1)),
Intervals.of("2011-01-02/2011-01-03"),
makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-02"), ImmutableMap.of("a", "c", "rows", 5)),
Intervals.of("2011-01-05/2011-01-10"),
makeSelectResults(
dimensions,
metrics,
DateTimes.of("2011-01-05"),
DateTimes.of("2011-01-06"),
DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"),
DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8),
DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9)
),
Intervals.of("2011-01-05/2011-01-10"),
makeSelectResults(
dimensions,
metrics,
DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5),
DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6),
DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7),
DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8),
DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9)
)
);
QueryRunner runner = new FinalizeResultsQueryRunner(
getDefaultQueryRunner(),
new SelectQueryQueryToolChest(
JSON_MAPPER,
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
);
TestHelper.assertExpectedResults(
makeSelectResults(
dimensions,
metrics,
DateTimes.of("2011-01-01"), ImmutableMap.of("a", "b", "rows", 1),
DateTimes.of("2011-01-02"), ImmutableMap.of("a", "c", "rows", 5),
DateTimes.of("2011-01-05"),
DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5),
DateTimes.of("2011-01-06"),
DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6),
DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"),
DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7),
DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8),
DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8),
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()))
);
}
@Test
public void testSelectCachingRenamedOutputName()
{
final Set<String> dimensions = Sets.newHashSet("a");
final Set<String> metrics = Sets.newHashSet("rows");
Druids.SelectQueryBuilder builder = Druids.newSelectQueryBuilder()
.dataSource(DATA_SOURCE)
.intervals(SEG_SPEC)
.filters(DIM_FILTER)
.granularity(GRANULARITY)
.dimensions(Collections.singletonList("a"))
.metrics(Collections.singletonList("rows"))
.pagingSpec(new PagingSpec(null, 3))
.context(CONTEXT);
testQueryCaching(
getDefaultQueryRunner(),
builder.build(),
Intervals.of("2011-01-01/2011-01-02"),
makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-01"), ImmutableMap.of("a", "b", "rows", 1)),
Intervals.of("2011-01-02/2011-01-03"),
makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-02"), ImmutableMap.of("a", "c", "rows", 5)),
Intervals.of("2011-01-05/2011-01-10"),
makeSelectResults(
dimensions,
metrics,
DateTimes.of("2011-01-05"), ImmutableMap.of("a", "d", "rows", 5),
DateTimes.of("2011-01-06"), ImmutableMap.of("a", "e", "rows", 6),
DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7),
DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8),
DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9)
),
Intervals.of("2011-01-05/2011-01-10"),
makeSelectResults(
dimensions,
metrics,
DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5),
DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6),
DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7),
DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8),
DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9)
)
);
QueryRunner runner = new FinalizeResultsQueryRunner(
getDefaultQueryRunner(),
new SelectQueryQueryToolChest(
JSON_MAPPER,
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
);
ResponseContext context = ResponseContext.createEmpty();
TestHelper.assertExpectedResults(
makeSelectResults(
dimensions,
metrics,
DateTimes.of("2011-01-01"), ImmutableMap.of("a", "b", "rows", 1),
DateTimes.of("2011-01-02"), ImmutableMap.of("a", "c", "rows", 5),
DateTimes.of("2011-01-05"), ImmutableMap.of("a", "d", "rows", 5),
DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5),
DateTimes.of("2011-01-06"), ImmutableMap.of("a", "e", "rows", 6),
DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6),
DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7),
DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7),
DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8),
DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8),
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)
);
SelectQuery query = builder
.intervals("2011-01-01/2011-01-10")
.dimensionSpecs(Collections.singletonList(new DefaultDimensionSpec("a", "a2")))
.build();
TestHelper.assertExpectedResults(
makeSelectResults(
dimensions,
metrics,
DateTimes.of("2011-01-01"), ImmutableMap.of("a2", "b", "rows", 1),
DateTimes.of("2011-01-02"), ImmutableMap.of("a2", "c", "rows", 5),
DateTimes.of("2011-01-05"), ImmutableMap.of("a2", "d", "rows", 5),
DateTimes.of("2011-01-05T01"), ImmutableMap.of("a2", "d", "rows", 5),
DateTimes.of("2011-01-06"), ImmutableMap.of("a2", "e", "rows", 6),
DateTimes.of("2011-01-06T01"), ImmutableMap.of("a2", "e", "rows", 6),
DateTimes.of("2011-01-07"), ImmutableMap.of("a2", "f", "rows", 7),
DateTimes.of("2011-01-07T01"), ImmutableMap.of("a2", "f", "rows", 7),
DateTimes.of("2011-01-08"), ImmutableMap.of("a2", "g", "rows", 8),
DateTimes.of("2011-01-08T01"), ImmutableMap.of("a2", "g", "rows", 8),
DateTimes.of("2011-01-09"), ImmutableMap.of("a2", "h", "rows", 9),
DateTimes.of("2011-01-09T01"), ImmutableMap.of("a2", "h", "rows", 9)
),
runner.run(QueryPlus.wrap(query), context)
);
}
@Test
public void testGroupByCaching()
{
@ -2060,18 +1870,6 @@ public class CachingClusteredClientTest
EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(context)))
.andReturn(toQueryableSearchResults(segmentIds, intervals, results))
.once();
} else if (query instanceof SelectQuery) {
List<SegmentId> segmentIds = new ArrayList<>();
List<Interval> intervals = new ArrayList<>();
List<Iterable<Result<SelectResultValue>>> results = new ArrayList<>();
for (ServerExpectation expectation : expectations) {
segmentIds.add(expectation.getSegmentId());
intervals.add(expectation.getInterval());
results.add(expectation.getResults());
}
EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(context)))
.andReturn(toQueryableSelectResults(segmentIds, intervals, results))
.once();
} else if (query instanceof GroupByQuery) {
List<SegmentId> segmentIds = new ArrayList<>();
List<Interval> intervals = new ArrayList<>();
@ -2351,42 +2149,6 @@ public class CachingClusteredClientTest
);
}
private Sequence<Result<SelectResultValue>> toQueryableSelectResults(
Iterable<SegmentId> segmentIds,
Iterable<Interval> intervals,
Iterable<Iterable<Result<SelectResultValue>>> results
)
{
return Sequences.simple(
FunctionalIterable
.create(segmentIds)
.trinaryTransform(
intervals,
results,
new TrinaryFn<SegmentId, Interval, Iterable<Result<SelectResultValue>>, Result<SelectResultValue>>()
{
@Override
@SuppressWarnings("unchecked")
public Result<SelectResultValue> apply(
final SegmentId segmentId,
final Interval interval,
final Iterable<Result<SelectResultValue>> results
)
{
return new Result(
results.iterator().next().getTimestamp(),
new BySegmentResultValueClass(
Lists.newArrayList(results),
segmentId.toString(),
interval
)
);
}
}
)
);
}
private Sequence<Result> toQueryableGroupByResults(
GroupByQuery query,
Iterable<SegmentId> segmentIds,
@ -2605,32 +2367,6 @@ public class CachingClusteredClientTest
return retVal;
}
private Iterable<Result<SelectResultValue>> makeSelectResults(
Set<String> dimensions,
Set<String> metrics,
Object... objects
)
{
List<Result<SelectResultValue>> retVal = new ArrayList<>();
int index = 0;
while (index < objects.length) {
DateTime timestamp = (DateTime) objects[index++];
List<EventHolder> values = new ArrayList<>();
while (index < objects.length && !(objects[index] instanceof DateTime)) {
values.add(new EventHolder(null, 0, (Map) objects[index++]));
}
Result<SelectResultValue> result = new Result<>(
timestamp,
new SelectResultValue(ImmutableMap.of(timestamp.toString(), 0), dimensions, metrics, values)
);
retVal.add(result);
}
return retVal;
}
private Iterable<ResultRow> makeGroupByResults(GroupByQuery query, Object... objects)
{
List<ResultRow> retVal = new ArrayList<>();

View File

@ -21,7 +21,6 @@ package org.apache.druid.client;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Pair;
@ -38,9 +37,6 @@ import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
import org.apache.druid.query.search.SearchQuery;
import org.apache.druid.query.search.SearchQueryConfig;
import org.apache.druid.query.search.SearchQueryQueryToolChest;
import org.apache.druid.query.select.SelectQuery;
import org.apache.druid.query.select.SelectQueryConfig;
import org.apache.druid.query.select.SelectQueryQueryToolChest;
import org.apache.druid.query.timeboundary.TimeBoundaryQuery;
import org.apache.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest;
import org.apache.druid.query.timeseries.TimeseriesQuery;
@ -56,8 +52,7 @@ public final class CachingClusteredClientTestUtils
* of the test.
*/
public static Pair<QueryToolChestWarehouse, Closer> createWarehouse(
ObjectMapper objectMapper,
Supplier<SelectQueryConfig> selectConfigSupplier
ObjectMapper objectMapper
)
{
final Pair<GroupByQueryRunnerFactory, Closer> factoryCloserPair = GroupByQueryRunnerTest.makeQueryRunnerFactory(
@ -88,13 +83,6 @@ public final class CachingClusteredClientTestUtils
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
)
.put(
SelectQuery.class,
new SelectQueryQueryToolChest(
objectMapper,
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
)
.put(
GroupByQuery.class,
factory.getToolchest()

View File

@ -59,7 +59,6 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
import org.apache.druid.query.ordering.StringComparator;
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.query.scan.ScanQuery;
import org.apache.druid.query.select.SelectQuery;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.topn.DimensionTopNMetricSpec;
import org.apache.druid.query.topn.InvertedTopNMetricSpec;
@ -614,7 +613,7 @@ public class DruidQuery
/**
* Return this query as some kind of Druid query. The returned query will either be {@link TopNQuery},
* {@link TimeseriesQuery}, {@link GroupByQuery}, {@link ScanQuery}, or {@link SelectQuery}.
* {@link TimeseriesQuery}, {@link GroupByQuery}, {@link ScanQuery}
*
* @return Druid query
*/

View File

@ -54,7 +54,6 @@ import org.apache.druid.query.groupby.having.DimFilterHavingSpec;
import org.apache.druid.query.ordering.StringComparator;
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.query.scan.ScanQuery;
import org.apache.druid.query.select.PagingSpec;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.segment.column.ColumnHolder;
@ -214,7 +213,6 @@ public class BaseCalciteQueryTest extends CalciteTestBase
// Matches QUERY_CONTEXT_LOS_ANGELES
public static final Map<String, Object> TIMESERIES_CONTEXT_LOS_ANGELES = new HashMap<>();
public static final PagingSpec FIRST_PAGING_SPEC = new PagingSpec(null, 1000, true);
public static QueryRunnerFactoryConglomerate conglomerate;
public static Closer resourceCloser;

View File

@ -84,10 +84,6 @@ import org.apache.druid.query.scan.ScanQueryConfig;
import org.apache.druid.query.scan.ScanQueryEngine;
import org.apache.druid.query.scan.ScanQueryQueryToolChest;
import org.apache.druid.query.scan.ScanQueryRunnerFactory;
import org.apache.druid.query.select.SelectQuery;
import org.apache.druid.query.select.SelectQueryEngine;
import org.apache.druid.query.select.SelectQueryQueryToolChest;
import org.apache.druid.query.select.SelectQueryRunnerFactory;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.timeseries.TimeseriesQueryEngine;
import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
@ -537,17 +533,6 @@ public class CalciteTests
new ScanQueryConfig()
)
)
.put(
SelectQuery.class,
new SelectQueryRunnerFactory(
new SelectQueryQueryToolChest(
TestHelper.makeJsonMapper(),
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
new SelectQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
)
.put(
TimeseriesQuery.class,
new TimeseriesQueryRunnerFactory(