mirror of
https://github.com/apache/druid.git
synced 2025-02-17 07:25:02 +00:00
Move scan-query from a contrib extension into core. (#4751)
* Move scan-query from a contrib extension into core. Based on a proposal at: https://groups.google.com/d/topic/druid-development/ME_OatUDnbk/discussion This patch also adds support for virtual columns to the Scan query, and updates Druid SQL to use Scan instead of Select. This patch also makes some behavioral changes to handling of the __time column. In particular, it is now is returned as "__time" rather than "timestamp"; it is no longer included if you do not specifically ask for it in your "columns"; and it is returned as a long rather than a string. Users can revert time handling to the legacy extension behavior by setting "legacy" : true in their queries, or setting the property druid.query.scan.legacy = true. This is meant to provide a migration path for users that were formerly using the contrib extension. * Adjustments from review. * Add back Select query. * Adjust SQL docs. * Restore SelectQuery link.
This commit is contained in:
parent
587f1807c2
commit
2ce8123bdb
@ -232,8 +232,6 @@
|
|||||||
<argument>-c</argument>
|
<argument>-c</argument>
|
||||||
<argument>io.druid.extensions.contrib:druid-redis-cache</argument>
|
<argument>io.druid.extensions.contrib:druid-redis-cache</argument>
|
||||||
<argument>-c</argument>
|
<argument>-c</argument>
|
||||||
<argument>io.druid.extensions.contrib:scan-query</argument>
|
|
||||||
<argument>-c</argument>
|
|
||||||
<argument>io.druid.extensions.contrib:sqlserver-metadata-storage</argument>
|
<argument>io.druid.extensions.contrib:sqlserver-metadata-storage</argument>
|
||||||
<argument>-c</argument>
|
<argument>-c</argument>
|
||||||
<argument>io.druid.extensions.contrib:statsd-emitter</argument>
|
<argument>io.druid.extensions.contrib:statsd-emitter</argument>
|
||||||
|
@ -70,7 +70,6 @@ All of these community extensions can be downloaded using *pull-deps* with the c
|
|||||||
|statsd-emitter|StatsD metrics emitter|[link](../development/extensions-contrib/statsd.html)|
|
|statsd-emitter|StatsD metrics emitter|[link](../development/extensions-contrib/statsd.html)|
|
||||||
|kafka-emitter|Kafka metrics emitter|[link](../development/extensions-contrib/kafka-emitter.html)|
|
|kafka-emitter|Kafka metrics emitter|[link](../development/extensions-contrib/kafka-emitter.html)|
|
||||||
|druid-thrift-extensions|Support thrift ingestion |[link](../development/extensions-contrib/thrift.html)|
|
|druid-thrift-extensions|Support thrift ingestion |[link](../development/extensions-contrib/thrift.html)|
|
||||||
|scan-query|Scan query|[link](../development/extensions-contrib/scan-query.html)|
|
|
||||||
|
|
||||||
## Promoting Community Extension to Core Extension
|
## Promoting Community Extension to Core Extension
|
||||||
|
|
||||||
|
@ -31,8 +31,11 @@ There are several main parts to a scan query:
|
|||||||
|columns|A String array of dimensions and metrics to scan. If left empty, all dimensions and metrics are returned.|no|
|
|columns|A String array of dimensions and metrics to scan. If left empty, all dimensions and metrics are returned.|no|
|
||||||
|batchSize|How many rows buffered before return to client. Default is `20480`|no|
|
|batchSize|How many rows buffered before return to client. Default is `20480`|no|
|
||||||
|limit|How many rows to return. If not specified, all rows will be returned.|no|
|
|limit|How many rows to return. If not specified, all rows will be returned.|no|
|
||||||
|
|legacy|Return results consistent with the legacy "scan-query" contrib extension. Defaults to the value set by `druid.query.scan.legacy`, which in turn defaults to false. See [Legacy mode](#legacy-mode) for details.|no|
|
||||||
|context|An additional JSON Object which can be used to specify certain flags.|no|
|
|context|An additional JSON Object which can be used to specify certain flags.|no|
|
||||||
|
|
||||||
|
## Example results
|
||||||
|
|
||||||
The format of the result when resultFormat equals to `list`:
|
The format of the result when resultFormat equals to `list`:
|
||||||
|
|
||||||
```json
|
```json
|
||||||
@ -155,3 +158,18 @@ The biggest difference between select query and scan query is that, scan query d
|
|||||||
It will cause memory pressure if too many rows required by select query.
|
It will cause memory pressure if too many rows required by select query.
|
||||||
Scan query doesn't have this issue.
|
Scan query doesn't have this issue.
|
||||||
Scan query can return all rows without issuing another pagination query, which is extremely useful when query against historical or realtime node directly.
|
Scan query can return all rows without issuing another pagination query, which is extremely useful when query against historical or realtime node directly.
|
||||||
|
|
||||||
|
## Legacy mode
|
||||||
|
|
||||||
|
The Scan query supports a legacy mode designed for protocol compatibility with the former scan-query contrib extension.
|
||||||
|
In legacy mode you can expect the following behavior changes:
|
||||||
|
|
||||||
|
- The __time column is returned as "timestamp" rather than "__time". This will take precedence over any other column
|
||||||
|
you may have that is named "timestamp".
|
||||||
|
- The __time column is included in the list of columns even if you do not specifically ask for it.
|
||||||
|
- Timestamps are returned as ISO8601 time strings rather than integers (milliseconds since 1970-01-01 00:00:00 UTC).
|
||||||
|
|
||||||
|
Legacy mode can be triggered either by passing `"legacy" : true` in your query JSON, or by setting
|
||||||
|
`druid.query.scan.legacy = true` on your Druid nodes. If you were previously using the scan-query contrib extension,
|
||||||
|
the best way to migrate is to activate legacy mode during a rolling upgrade, then switch it off after the upgrade
|
||||||
|
is complete.
|
@ -2,6 +2,7 @@
|
|||||||
layout: doc_page
|
layout: doc_page
|
||||||
---
|
---
|
||||||
# Select Queries
|
# Select Queries
|
||||||
|
|
||||||
Select queries return raw Druid rows and support pagination.
|
Select queries return raw Druid rows and support pagination.
|
||||||
|
|
||||||
```json
|
```json
|
||||||
@ -19,6 +20,13 @@ Select queries return raw Druid rows and support pagination.
|
|||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
|
||||||
|
<div class="note info">
|
||||||
|
Consider using the [Scan query](scan-query.html) instead of the Select query if you don't need pagination, and you
|
||||||
|
don't need the strict time-ascending or time-descending ordering offered by the Select query. The Scan query returns
|
||||||
|
results without pagination, and offers "looser" ordering than Select, but is significantly more efficient in terms of
|
||||||
|
both processing time and memory requirements. It is also capable of returning a virtually unlimited number of results.
|
||||||
|
</div>
|
||||||
|
|
||||||
There are several main parts to a select query:
|
There are several main parts to a select query:
|
||||||
|
|
||||||
|property|description|required?|
|
|property|description|required?|
|
||||||
|
@ -256,7 +256,9 @@ converted to zeroes).
|
|||||||
|
|
||||||
## Query execution
|
## Query execution
|
||||||
|
|
||||||
Queries without aggregations will use Druid's [Select](select-query.html) native query type.
|
Queries without aggregations will use Druid's [Scan](scan-query.html) or [Select](select-query.html) native query types.
|
||||||
|
Scan is used whenever possible, as it is generally higher performance and more efficient than Select. However, Select
|
||||||
|
is used in one case: when the query includes an `ORDER BY __time`, since Scan does not have a sorting feature.
|
||||||
|
|
||||||
Aggregation queries (using GROUP BY, DISTINCT, or any aggregation functions) will use one of Druid's three native
|
Aggregation queries (using GROUP BY, DISTINCT, or any aggregation functions) will use one of Druid's three native
|
||||||
aggregation query types. Two (Timeseries and TopN) are specialized for specific types of aggregations, whereas the other
|
aggregation query types. Two (Timeseries and TopN) are specialized for specific types of aggregations, whereas the other
|
||||||
|
@ -34,6 +34,7 @@ layout: toc
|
|||||||
* [DataSource Metadata](/docs/VERSION/querying/datasourcemetadataquery.html)
|
* [DataSource Metadata](/docs/VERSION/querying/datasourcemetadataquery.html)
|
||||||
* [Search](/docs/VERSION/querying/searchquery.html)
|
* [Search](/docs/VERSION/querying/searchquery.html)
|
||||||
* [Select](/docs/VERSION/querying/select-query.html)
|
* [Select](/docs/VERSION/querying/select-query.html)
|
||||||
|
* [Scan](/docs/VERSION/querying/scan-query.html)
|
||||||
* Components
|
* Components
|
||||||
* [Datasources](/docs/VERSION/querying/datasource.html)
|
* [Datasources](/docs/VERSION/querying/datasource.html)
|
||||||
* [Filters](/docs/VERSION/querying/filters.html)
|
* [Filters](/docs/VERSION/querying/filters.html)
|
||||||
|
@ -1,63 +0,0 @@
|
|||||||
<?xml version="1.0" encoding="UTF-8"?>
|
|
||||||
<!--
|
|
||||||
~ Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
|
||||||
~ or more contributor license agreements. See the NOTICE file
|
|
||||||
~ distributed with this work for additional information
|
|
||||||
~ regarding copyright ownership. Metamarkets licenses this file
|
|
||||||
~ to you under the Apache License, Version 2.0 (the
|
|
||||||
~ "License"); you may not use this file except in compliance
|
|
||||||
~ with the License. You may obtain a copy of the License at
|
|
||||||
~
|
|
||||||
~ http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
~
|
|
||||||
~ Unless required by applicable law or agreed to in writing,
|
|
||||||
~ software distributed under the License is distributed on an
|
|
||||||
~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
||||||
~ KIND, either express or implied. See the License for the
|
|
||||||
~ specific language governing permissions and limitations
|
|
||||||
~ under the License.
|
|
||||||
-->
|
|
||||||
|
|
||||||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
|
||||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
|
||||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
|
||||||
|
|
||||||
<parent>
|
|
||||||
<groupId>io.druid</groupId>
|
|
||||||
<artifactId>druid</artifactId>
|
|
||||||
<version>0.11.0-SNAPSHOT</version>
|
|
||||||
<relativePath>../../pom.xml</relativePath>
|
|
||||||
</parent>
|
|
||||||
<modelVersion>4.0.0</modelVersion>
|
|
||||||
|
|
||||||
<groupId>io.druid.extensions.contrib</groupId>
|
|
||||||
<artifactId>scan-query</artifactId>
|
|
||||||
<name>scan-query</name>
|
|
||||||
<description>streaming version of select query</description>
|
|
||||||
|
|
||||||
<dependencies>
|
|
||||||
<dependency>
|
|
||||||
<groupId>io.druid</groupId>
|
|
||||||
<artifactId>druid-server</artifactId>
|
|
||||||
<version>${project.parent.version}</version>
|
|
||||||
</dependency>
|
|
||||||
<dependency>
|
|
||||||
<groupId>junit</groupId>
|
|
||||||
<artifactId>junit</artifactId>
|
|
||||||
<scope>test</scope>
|
|
||||||
</dependency>
|
|
||||||
<dependency>
|
|
||||||
<groupId>org.easymock</groupId>
|
|
||||||
<artifactId>easymock</artifactId>
|
|
||||||
<scope>test</scope>
|
|
||||||
</dependency>
|
|
||||||
<dependency>
|
|
||||||
<groupId>io.druid</groupId>
|
|
||||||
<artifactId>druid-processing</artifactId>
|
|
||||||
<version>${project.parent.version}</version>
|
|
||||||
<classifier>tests</classifier>
|
|
||||||
<scope>test</scope>
|
|
||||||
</dependency>
|
|
||||||
</dependencies>
|
|
||||||
|
|
||||||
</project>
|
|
@ -1,58 +0,0 @@
|
|||||||
/*
|
|
||||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. Metamarkets licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing,
|
|
||||||
* software distributed under the License is distributed on an
|
|
||||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
||||||
* KIND, either express or implied. See the License for the
|
|
||||||
* specific language governing permissions and limitations
|
|
||||||
* under the License.
|
|
||||||
*/
|
|
||||||
package io.druid.query.scan;
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.Module;
|
|
||||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
|
||||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
|
||||||
import com.google.inject.Binder;
|
|
||||||
import io.druid.guice.DruidBinders;
|
|
||||||
import io.druid.guice.LazySingleton;
|
|
||||||
import io.druid.initialization.DruidModule;
|
|
||||||
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
public class ScanQueryDruidModule implements DruidModule
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public void configure(Binder binder)
|
|
||||||
{
|
|
||||||
DruidBinders.queryToolChestBinder(binder)
|
|
||||||
.addBinding(ScanQuery.class)
|
|
||||||
.to(ScanQueryQueryToolChest.class)
|
|
||||||
.in(LazySingleton.class);
|
|
||||||
|
|
||||||
DruidBinders.queryRunnerFactoryBinder(binder)
|
|
||||||
.addBinding(ScanQuery.class)
|
|
||||||
.to(ScanQueryRunnerFactory.class)
|
|
||||||
.in(LazySingleton.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<? extends Module> getJacksonModules()
|
|
||||||
{
|
|
||||||
return Arrays.<Module>asList(
|
|
||||||
new SimpleModule("ScanQueryDruidModule")
|
|
||||||
.registerSubtypes(
|
|
||||||
new NamedType(ScanQuery.class, ScanQuery.SCAN)
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
|
||||||
}
|
|
@ -1 +0,0 @@
|
|||||||
io.druid.query.scan.ScanQueryDruidModule
|
|
1
pom.xml
1
pom.xml
@ -134,7 +134,6 @@
|
|||||||
<module>extensions-contrib/virtual-columns</module>
|
<module>extensions-contrib/virtual-columns</module>
|
||||||
<module>extensions-contrib/thrift-extensions</module>
|
<module>extensions-contrib/thrift-extensions</module>
|
||||||
<module>extensions-contrib/ambari-metrics-emitter</module>
|
<module>extensions-contrib/ambari-metrics-emitter</module>
|
||||||
<module>extensions-contrib/scan-query</module>
|
|
||||||
<module>extensions-contrib/sqlserver-metadata-storage</module>
|
<module>extensions-contrib/sqlserver-metadata-storage</module>
|
||||||
<module>extensions-contrib/kafka-emitter</module>
|
<module>extensions-contrib/kafka-emitter</module>
|
||||||
<module>extensions-contrib/redis-cache</module>
|
<module>extensions-contrib/redis-cache</module>
|
||||||
|
@ -27,6 +27,7 @@ import io.druid.query.datasourcemetadata.DataSourceMetadataQuery;
|
|||||||
import io.druid.query.filter.DimFilter;
|
import io.druid.query.filter.DimFilter;
|
||||||
import io.druid.query.groupby.GroupByQuery;
|
import io.druid.query.groupby.GroupByQuery;
|
||||||
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||||
|
import io.druid.query.scan.ScanQuery;
|
||||||
import io.druid.query.search.search.SearchQuery;
|
import io.druid.query.search.search.SearchQuery;
|
||||||
import io.druid.query.select.SelectQuery;
|
import io.druid.query.select.SelectQuery;
|
||||||
import io.druid.query.spec.QuerySegmentSpec;
|
import io.druid.query.spec.QuerySegmentSpec;
|
||||||
@ -46,6 +47,7 @@ import java.util.Map;
|
|||||||
@JsonSubTypes.Type(name = Query.SEARCH, value = SearchQuery.class),
|
@JsonSubTypes.Type(name = Query.SEARCH, value = SearchQuery.class),
|
||||||
@JsonSubTypes.Type(name = Query.TIME_BOUNDARY, value = TimeBoundaryQuery.class),
|
@JsonSubTypes.Type(name = Query.TIME_BOUNDARY, value = TimeBoundaryQuery.class),
|
||||||
@JsonSubTypes.Type(name = Query.GROUP_BY, value = GroupByQuery.class),
|
@JsonSubTypes.Type(name = Query.GROUP_BY, value = GroupByQuery.class),
|
||||||
|
@JsonSubTypes.Type(name = Query.SCAN, value = ScanQuery.class),
|
||||||
@JsonSubTypes.Type(name = Query.SEGMENT_METADATA, value = SegmentMetadataQuery.class),
|
@JsonSubTypes.Type(name = Query.SEGMENT_METADATA, value = SegmentMetadataQuery.class),
|
||||||
@JsonSubTypes.Type(name = Query.SELECT, value = SelectQuery.class),
|
@JsonSubTypes.Type(name = Query.SELECT, value = SelectQuery.class),
|
||||||
@JsonSubTypes.Type(name = Query.TOPN, value = TopNQuery.class),
|
@JsonSubTypes.Type(name = Query.TOPN, value = TopNQuery.class),
|
||||||
@ -58,6 +60,7 @@ public interface Query<T>
|
|||||||
String SEARCH = "search";
|
String SEARCH = "search";
|
||||||
String TIME_BOUNDARY = "timeBoundary";
|
String TIME_BOUNDARY = "timeBoundary";
|
||||||
String GROUP_BY = "groupBy";
|
String GROUP_BY = "groupBy";
|
||||||
|
String SCAN = "scan";
|
||||||
String SEGMENT_METADATA = "segmentMetadata";
|
String SEGMENT_METADATA = "segmentMetadata";
|
||||||
String SELECT = "select";
|
String SELECT = "select";
|
||||||
String TOPN = "topN";
|
String TOPN = "topN";
|
||||||
|
@ -22,6 +22,7 @@ package io.druid.query.extraction;
|
|||||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
import io.druid.guice.annotations.ExtensionPoint;
|
import io.druid.guice.annotations.ExtensionPoint;
|
||||||
|
import io.druid.java.util.common.Cacheable;
|
||||||
import io.druid.query.lookup.LookupExtractionFn;
|
import io.druid.query.lookup.LookupExtractionFn;
|
||||||
import io.druid.query.lookup.RegisteredLookupExtractionFn;
|
import io.druid.query.lookup.RegisteredLookupExtractionFn;
|
||||||
|
|
||||||
@ -57,16 +58,8 @@ import javax.annotation.Nullable;
|
|||||||
* regular expression with a capture group. When the regular expression matches the value of a dimension,
|
* regular expression with a capture group. When the regular expression matches the value of a dimension,
|
||||||
* the value captured by the group is used for grouping operations instead of the dimension value.
|
* the value captured by the group is used for grouping operations instead of the dimension value.
|
||||||
*/
|
*/
|
||||||
public interface ExtractionFn
|
public interface ExtractionFn extends Cacheable
|
||||||
{
|
{
|
||||||
/**
|
|
||||||
* Returns a byte[] unique to all concrete implementations of DimExtractionFn. This byte[] is used to
|
|
||||||
* generate a cache key for the specific query.
|
|
||||||
*
|
|
||||||
* @return a byte[] unit to all concrete implements of DimExtractionFn
|
|
||||||
*/
|
|
||||||
public byte[] getCacheKey();
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The "extraction" function. This should map an Object into some String value.
|
* The "extraction" function. This should map an Object into some String value.
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -20,7 +20,6 @@ package io.druid.query.scan;
|
|||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import io.druid.query.BaseQuery;
|
import io.druid.query.BaseQuery;
|
||||||
@ -32,39 +31,45 @@ import io.druid.query.filter.InDimFilter;
|
|||||||
import io.druid.query.filter.SelectorDimFilter;
|
import io.druid.query.filter.SelectorDimFilter;
|
||||||
import io.druid.query.spec.LegacySegmentSpec;
|
import io.druid.query.spec.LegacySegmentSpec;
|
||||||
import io.druid.query.spec.QuerySegmentSpec;
|
import io.druid.query.spec.QuerySegmentSpec;
|
||||||
|
import io.druid.segment.VirtualColumn;
|
||||||
|
import io.druid.segment.VirtualColumns;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
@JsonTypeName("scan")
|
|
||||||
public class ScanQuery extends BaseQuery<ScanResultValue>
|
public class ScanQuery extends BaseQuery<ScanResultValue>
|
||||||
{
|
{
|
||||||
public static final String SCAN = "scan";
|
|
||||||
public static final String RESULT_FORMAT_LIST = "list";
|
public static final String RESULT_FORMAT_LIST = "list";
|
||||||
public static final String RESULT_FORMAT_COMPACTED_LIST = "compactedList";
|
public static final String RESULT_FORMAT_COMPACTED_LIST = "compactedList";
|
||||||
public static final String RESULT_FORMAT_VALUE_VECTOR = "valueVector";
|
public static final String RESULT_FORMAT_VALUE_VECTOR = "valueVector";
|
||||||
|
|
||||||
|
private final VirtualColumns virtualColumns;
|
||||||
private final String resultFormat;
|
private final String resultFormat;
|
||||||
private final int batchSize;
|
private final int batchSize;
|
||||||
private final long limit;
|
private final long limit;
|
||||||
private final DimFilter dimFilter;
|
private final DimFilter dimFilter;
|
||||||
private final List<String> columns;
|
private final List<String> columns;
|
||||||
|
private final Boolean legacy;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public ScanQuery(
|
public ScanQuery(
|
||||||
@JsonProperty("dataSource") DataSource dataSource,
|
@JsonProperty("dataSource") DataSource dataSource,
|
||||||
@JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
|
@JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
|
||||||
|
@JsonProperty("virtualColumns") VirtualColumns virtualColumns,
|
||||||
@JsonProperty("resultFormat") String resultFormat,
|
@JsonProperty("resultFormat") String resultFormat,
|
||||||
@JsonProperty("batchSize") int batchSize,
|
@JsonProperty("batchSize") int batchSize,
|
||||||
@JsonProperty("limit") long limit,
|
@JsonProperty("limit") long limit,
|
||||||
@JsonProperty("filter") DimFilter dimFilter,
|
@JsonProperty("filter") DimFilter dimFilter,
|
||||||
@JsonProperty("columns") List<String> columns,
|
@JsonProperty("columns") List<String> columns,
|
||||||
|
@JsonProperty("legacy") Boolean legacy,
|
||||||
@JsonProperty("context") Map<String, Object> context
|
@JsonProperty("context") Map<String, Object> context
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(dataSource, querySegmentSpec, false, context);
|
super(dataSource, querySegmentSpec, false, context);
|
||||||
|
this.virtualColumns = VirtualColumns.nullToEmpty(virtualColumns);
|
||||||
this.resultFormat = resultFormat == null ? RESULT_FORMAT_LIST : resultFormat;
|
this.resultFormat = resultFormat == null ? RESULT_FORMAT_LIST : resultFormat;
|
||||||
this.batchSize = (batchSize == 0) ? 4096 * 5 : batchSize;
|
this.batchSize = (batchSize == 0) ? 4096 * 5 : batchSize;
|
||||||
this.limit = (limit == 0) ? Long.MAX_VALUE : limit;
|
this.limit = (limit == 0) ? Long.MAX_VALUE : limit;
|
||||||
@ -72,6 +77,13 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||||||
Preconditions.checkArgument(this.limit > 0, "limit must be greater than 0");
|
Preconditions.checkArgument(this.limit > 0, "limit must be greater than 0");
|
||||||
this.dimFilter = dimFilter;
|
this.dimFilter = dimFilter;
|
||||||
this.columns = columns;
|
this.columns = columns;
|
||||||
|
this.legacy = legacy;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public VirtualColumns getVirtualColumns()
|
||||||
|
{
|
||||||
|
return virtualColumns;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
@ -99,6 +111,7 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@JsonProperty
|
||||||
public DimFilter getFilter()
|
public DimFilter getFilter()
|
||||||
{
|
{
|
||||||
return dimFilter;
|
return dimFilter;
|
||||||
@ -110,18 +123,26 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||||||
return SCAN;
|
return SCAN;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty("filter")
|
|
||||||
public DimFilter getDimensionsFilter()
|
|
||||||
{
|
|
||||||
return dimFilter;
|
|
||||||
}
|
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public List<String> getColumns()
|
public List<String> getColumns()
|
||||||
{
|
{
|
||||||
return columns;
|
return columns;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Compatibility mode with the legacy scan-query extension.
|
||||||
|
*/
|
||||||
|
@JsonProperty
|
||||||
|
public Boolean isLegacy()
|
||||||
|
{
|
||||||
|
return legacy;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ScanQuery withNonNullLegacy(final ScanQueryConfig scanQueryConfig)
|
||||||
|
{
|
||||||
|
return ScanQueryBuilder.copy(this).legacy(legacy != null ? legacy : scanQueryConfig.isLegacy()).build();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Query<ScanResultValue> withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec)
|
public Query<ScanResultValue> withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec)
|
||||||
{
|
{
|
||||||
@ -146,7 +167,7 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(Object o)
|
public boolean equals(final Object o)
|
||||||
{
|
{
|
||||||
if (this == o) {
|
if (this == o) {
|
||||||
return true;
|
return true;
|
||||||
@ -157,34 +178,20 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||||||
if (!super.equals(o)) {
|
if (!super.equals(o)) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
final ScanQuery scanQuery = (ScanQuery) o;
|
||||||
ScanQuery that = (ScanQuery) o;
|
return batchSize == scanQuery.batchSize &&
|
||||||
|
limit == scanQuery.limit &&
|
||||||
if (batchSize != that.batchSize) {
|
legacy == scanQuery.legacy &&
|
||||||
return false;
|
Objects.equals(virtualColumns, scanQuery.virtualColumns) &&
|
||||||
}
|
Objects.equals(resultFormat, scanQuery.resultFormat) &&
|
||||||
if (limit != that.limit) {
|
Objects.equals(dimFilter, scanQuery.dimFilter) &&
|
||||||
return false;
|
Objects.equals(columns, scanQuery.columns);
|
||||||
}
|
|
||||||
if (resultFormat != null ? !resultFormat.equals(that.resultFormat) : that.resultFormat != null) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
return columns != null ? columns.equals(that.columns) : that.columns == null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
int result = super.hashCode();
|
return Objects.hash(super.hashCode(), virtualColumns, resultFormat, batchSize, limit, dimFilter, columns, legacy);
|
||||||
result = 31 * result + (resultFormat != null ? resultFormat.hashCode() : 0);
|
|
||||||
result = 31 * result + batchSize;
|
|
||||||
result = 31 * result + (int) (limit ^ (limit >>> 32));
|
|
||||||
result = 31 * result + (dimFilter != null ? dimFilter.hashCode() : 0);
|
|
||||||
result = 31 * result + (columns != null ? columns.hashCode() : 0);
|
|
||||||
return result;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -193,12 +200,13 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||||||
return "ScanQuery{" +
|
return "ScanQuery{" +
|
||||||
"dataSource='" + getDataSource() + '\'' +
|
"dataSource='" + getDataSource() + '\'' +
|
||||||
", querySegmentSpec=" + getQuerySegmentSpec() +
|
", querySegmentSpec=" + getQuerySegmentSpec() +
|
||||||
", descending=" + isDescending() +
|
", virtualColumns=" + getVirtualColumns() +
|
||||||
", resultFormat='" + resultFormat + '\'' +
|
", resultFormat='" + resultFormat + '\'' +
|
||||||
", batchSize=" + batchSize +
|
", batchSize=" + batchSize +
|
||||||
", limit=" + limit +
|
", limit=" + limit +
|
||||||
", dimFilter=" + dimFilter +
|
", dimFilter=" + dimFilter +
|
||||||
", columns=" + columns +
|
", columns=" + columns +
|
||||||
|
", legacy=" + legacy +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -221,23 +229,27 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||||||
{
|
{
|
||||||
private DataSource dataSource;
|
private DataSource dataSource;
|
||||||
private QuerySegmentSpec querySegmentSpec;
|
private QuerySegmentSpec querySegmentSpec;
|
||||||
|
private VirtualColumns virtualColumns;
|
||||||
private Map<String, Object> context;
|
private Map<String, Object> context;
|
||||||
private String resultFormat;
|
private String resultFormat;
|
||||||
private int batchSize;
|
private int batchSize;
|
||||||
private long limit;
|
private long limit;
|
||||||
private DimFilter dimFilter;
|
private DimFilter dimFilter;
|
||||||
private List<String> columns;
|
private List<String> columns;
|
||||||
|
private Boolean legacy;
|
||||||
|
|
||||||
public ScanQueryBuilder()
|
public ScanQueryBuilder()
|
||||||
{
|
{
|
||||||
dataSource = null;
|
dataSource = null;
|
||||||
querySegmentSpec = null;
|
querySegmentSpec = null;
|
||||||
|
virtualColumns = null;
|
||||||
context = null;
|
context = null;
|
||||||
resultFormat = null;
|
resultFormat = null;
|
||||||
batchSize = 0;
|
batchSize = 0;
|
||||||
limit = 0;
|
limit = 0;
|
||||||
dimFilter = null;
|
dimFilter = null;
|
||||||
columns = Lists.newArrayList();
|
columns = Lists.newArrayList();
|
||||||
|
legacy = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ScanQuery build()
|
public ScanQuery build()
|
||||||
@ -245,11 +257,13 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||||||
return new ScanQuery(
|
return new ScanQuery(
|
||||||
dataSource,
|
dataSource,
|
||||||
querySegmentSpec,
|
querySegmentSpec,
|
||||||
|
virtualColumns,
|
||||||
resultFormat,
|
resultFormat,
|
||||||
batchSize,
|
batchSize,
|
||||||
limit,
|
limit,
|
||||||
dimFilter,
|
dimFilter,
|
||||||
columns,
|
columns,
|
||||||
|
legacy,
|
||||||
context
|
context
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
@ -259,11 +273,13 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||||||
return new ScanQueryBuilder()
|
return new ScanQueryBuilder()
|
||||||
.dataSource(query.getDataSource())
|
.dataSource(query.getDataSource())
|
||||||
.intervals(query.getQuerySegmentSpec())
|
.intervals(query.getQuerySegmentSpec())
|
||||||
|
.virtualColumns(query.getVirtualColumns())
|
||||||
.resultFormat(query.getResultFormat())
|
.resultFormat(query.getResultFormat())
|
||||||
.batchSize(query.getBatchSize())
|
.batchSize(query.getBatchSize())
|
||||||
.limit(query.getLimit())
|
.limit(query.getLimit())
|
||||||
.filters(query.getFilter())
|
.filters(query.getFilter())
|
||||||
.columns(query.getColumns())
|
.columns(query.getColumns())
|
||||||
|
.legacy(query.isLegacy())
|
||||||
.context(query.getContext());
|
.context(query.getContext());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -297,6 +313,22 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public ScanQueryBuilder virtualColumns(VirtualColumns virtualColumns)
|
||||||
|
{
|
||||||
|
this.virtualColumns = virtualColumns;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ScanQueryBuilder virtualColumns(List<VirtualColumn> virtualColumns)
|
||||||
|
{
|
||||||
|
return virtualColumns(VirtualColumns.create(virtualColumns));
|
||||||
|
}
|
||||||
|
|
||||||
|
public ScanQueryBuilder virtualColumns(VirtualColumn... virtualColumns)
|
||||||
|
{
|
||||||
|
return virtualColumns(VirtualColumns.create(Arrays.asList(virtualColumns)));
|
||||||
|
}
|
||||||
|
|
||||||
public ScanQueryBuilder context(Map<String, Object> c)
|
public ScanQueryBuilder context(Map<String, Object> c)
|
||||||
{
|
{
|
||||||
context = c;
|
context = c;
|
||||||
@ -350,6 +382,12 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||||||
columns = Arrays.asList(c);
|
columns = Arrays.asList(c);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public ScanQueryBuilder legacy(Boolean legacy)
|
||||||
|
{
|
||||||
|
this.legacy = legacy;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static ScanQueryBuilder newScanQueryBuilder()
|
public static ScanQueryBuilder newScanQueryBuilder()
|
@ -0,0 +1,68 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.scan;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class ScanQueryConfig
|
||||||
|
{
|
||||||
|
@JsonProperty
|
||||||
|
private boolean legacy = false;
|
||||||
|
|
||||||
|
public boolean isLegacy()
|
||||||
|
{
|
||||||
|
return legacy;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ScanQueryConfig setLegacy(final boolean legacy)
|
||||||
|
{
|
||||||
|
this.legacy = legacy;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(final Object o)
|
||||||
|
{
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
final ScanQueryConfig that = (ScanQueryConfig) o;
|
||||||
|
return legacy == that.legacy;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(legacy);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "ScanQueryConfig{" +
|
||||||
|
"legacy=" + legacy +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
|
}
|
@ -20,46 +20,51 @@ package io.druid.query.scan;
|
|||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Sets;
|
||||||
|
import io.druid.java.util.common.DateTimes;
|
||||||
import io.druid.java.util.common.ISE;
|
import io.druid.java.util.common.ISE;
|
||||||
|
import io.druid.java.util.common.UOE;
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
import io.druid.java.util.common.guava.BaseSequence;
|
import io.druid.java.util.common.guava.BaseSequence;
|
||||||
import io.druid.java.util.common.guava.Sequence;
|
import io.druid.java.util.common.guava.Sequence;
|
||||||
import io.druid.java.util.common.guava.Sequences;
|
import io.druid.java.util.common.guava.Sequences;
|
||||||
import io.druid.query.ColumnSelectorPlus;
|
|
||||||
import io.druid.query.QueryContexts;
|
import io.druid.query.QueryContexts;
|
||||||
import io.druid.query.QueryInterruptedException;
|
import io.druid.query.QueryInterruptedException;
|
||||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
|
||||||
import io.druid.query.filter.Filter;
|
import io.druid.query.filter.Filter;
|
||||||
import io.druid.query.select.SelectQueryEngine;
|
|
||||||
import io.druid.segment.Cursor;
|
import io.druid.segment.Cursor;
|
||||||
import io.druid.segment.DimensionHandlerUtils;
|
|
||||||
import io.druid.segment.LongColumnSelector;
|
|
||||||
import io.druid.segment.ObjectColumnSelector;
|
import io.druid.segment.ObjectColumnSelector;
|
||||||
import io.druid.segment.Segment;
|
import io.druid.segment.Segment;
|
||||||
import io.druid.segment.StorageAdapter;
|
import io.druid.segment.StorageAdapter;
|
||||||
import io.druid.segment.VirtualColumns;
|
import io.druid.segment.VirtualColumn;
|
||||||
import io.druid.segment.column.Column;
|
import io.druid.segment.column.Column;
|
||||||
import io.druid.segment.filter.Filters;
|
import io.druid.segment.filter.Filters;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
public class ScanQueryEngine
|
public class ScanQueryEngine
|
||||||
{
|
{
|
||||||
private static final SelectQueryEngine.SelectStrategyFactory STRATEGY_FACTORY = new SelectQueryEngine.SelectStrategyFactory();
|
private static final String LEGACY_TIMESTAMP_KEY = "timestamp";
|
||||||
|
|
||||||
public Sequence<ScanResultValue> process(
|
public Sequence<ScanResultValue> process(
|
||||||
final ScanQuery query,
|
final ScanQuery query,
|
||||||
final Segment segment,
|
final Segment segment,
|
||||||
final Map<String, Object> responseContext
|
final Map<String, Object> responseContext
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
// "legacy" should be non-null due to toolChest.mergeResults
|
||||||
|
final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "WTF?! Expected non-null legacy");
|
||||||
|
|
||||||
if (responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) != null) {
|
if (responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) != null) {
|
||||||
long count = (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT);
|
long count = (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT);
|
||||||
if (count >= query.getLimit()) {
|
if (count >= query.getLimit()) {
|
||||||
@ -77,32 +82,42 @@ public class ScanQueryEngine
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
List<String> allDims = Lists.newLinkedList(adapter.getAvailableDimensions());
|
final List<String> allColumns = new ArrayList<>();
|
||||||
List<String> allMetrics = Lists.newLinkedList(adapter.getAvailableMetrics());
|
|
||||||
final List<String> allColumns = Lists.newLinkedList();
|
|
||||||
if (query.getColumns() != null && !query.getColumns().isEmpty()) {
|
if (query.getColumns() != null && !query.getColumns().isEmpty()) {
|
||||||
if (!query.getColumns().contains(ScanResultValue.timestampKey)) {
|
if (legacy && !query.getColumns().contains(LEGACY_TIMESTAMP_KEY)) {
|
||||||
allColumns.add(ScanResultValue.timestampKey);
|
allColumns.add(LEGACY_TIMESTAMP_KEY);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Unless we're in legacy mode, allColumns equals query.getColumns() exactly. This is nice since it makes
|
||||||
|
// the compactedList form easier to use.
|
||||||
allColumns.addAll(query.getColumns());
|
allColumns.addAll(query.getColumns());
|
||||||
allDims.retainAll(query.getColumns());
|
|
||||||
allMetrics.retainAll(query.getColumns());
|
|
||||||
} else {
|
} else {
|
||||||
if (!allDims.contains(ScanResultValue.timestampKey)) {
|
final Set<String> availableColumns = Sets.newLinkedHashSet(
|
||||||
allColumns.add(ScanResultValue.timestampKey);
|
Iterables.concat(
|
||||||
|
Collections.singleton(legacy ? LEGACY_TIMESTAMP_KEY : Column.TIME_COLUMN_NAME),
|
||||||
|
Iterables.transform(
|
||||||
|
Arrays.asList(query.getVirtualColumns().getVirtualColumns()),
|
||||||
|
VirtualColumn::getOutputName
|
||||||
|
),
|
||||||
|
adapter.getAvailableDimensions(),
|
||||||
|
adapter.getAvailableMetrics()
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
allColumns.addAll(availableColumns);
|
||||||
|
|
||||||
|
if (legacy) {
|
||||||
|
allColumns.remove(Column.TIME_COLUMN_NAME);
|
||||||
}
|
}
|
||||||
allColumns.addAll(allDims);
|
|
||||||
allColumns.addAll(allMetrics);
|
|
||||||
}
|
}
|
||||||
final List<DimensionSpec> dims = DefaultDimensionSpec.toSpec(allDims);
|
|
||||||
final List<String> metrics = allMetrics;
|
|
||||||
|
|
||||||
final List<Interval> intervals = query.getQuerySegmentSpec().getIntervals();
|
final List<Interval> intervals = query.getQuerySegmentSpec().getIntervals();
|
||||||
Preconditions.checkArgument(intervals.size() == 1, "Can only handle a single interval, got[%s]", intervals);
|
Preconditions.checkArgument(intervals.size() == 1, "Can only handle a single interval, got[%s]", intervals);
|
||||||
|
|
||||||
final String segmentId = segment.getIdentifier();
|
final String segmentId = segment.getIdentifier();
|
||||||
|
|
||||||
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimensionsFilter()));
|
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()));
|
||||||
|
|
||||||
if (responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) == null) {
|
if (responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) == null) {
|
||||||
responseContext.put(ScanQueryRunnerFactory.CTX_COUNT, 0L);
|
responseContext.put(ScanQueryRunnerFactory.CTX_COUNT, 0L);
|
||||||
@ -113,7 +128,7 @@ public class ScanQueryEngine
|
|||||||
adapter.makeCursors(
|
adapter.makeCursors(
|
||||||
filter,
|
filter,
|
||||||
intervals.get(0),
|
intervals.get(0),
|
||||||
VirtualColumns.EMPTY,
|
query.getVirtualColumns(),
|
||||||
Granularities.ALL,
|
Granularities.ALL,
|
||||||
query.isDescending(),
|
query.isDescending(),
|
||||||
null
|
null
|
||||||
@ -129,23 +144,21 @@ public class ScanQueryEngine
|
|||||||
@Override
|
@Override
|
||||||
public Iterator<ScanResultValue> make()
|
public Iterator<ScanResultValue> make()
|
||||||
{
|
{
|
||||||
final LongColumnSelector timestampColumnSelector =
|
final List<ObjectColumnSelector> columnSelectors = new ArrayList<>(allColumns.size());
|
||||||
cursor.getColumnSelectorFactory().makeLongColumnSelector(Column.TIME_COLUMN_NAME);
|
|
||||||
|
|
||||||
final List<ColumnSelectorPlus<SelectQueryEngine.SelectColumnSelectorStrategy>> selectorPlusList = Arrays.asList(
|
for (String column : allColumns) {
|
||||||
DimensionHandlerUtils.createColumnSelectorPluses(
|
final ObjectColumnSelector selector;
|
||||||
STRATEGY_FACTORY,
|
|
||||||
Lists.newArrayList(dims),
|
|
||||||
cursor.getColumnSelectorFactory()
|
|
||||||
)
|
|
||||||
);
|
|
||||||
|
|
||||||
final Map<String, ObjectColumnSelector> metSelectors = Maps.newHashMap();
|
if (legacy && column.equals(LEGACY_TIMESTAMP_KEY)) {
|
||||||
for (String metric : metrics) {
|
selector = cursor.getColumnSelectorFactory()
|
||||||
final ObjectColumnSelector metricSelector =
|
.makeObjectColumnSelector(Column.TIME_COLUMN_NAME);
|
||||||
cursor.getColumnSelectorFactory().makeObjectColumnSelector(metric);
|
} else {
|
||||||
metSelectors.put(metric, metricSelector);
|
selector = cursor.getColumnSelectorFactory().makeObjectColumnSelector(column);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
columnSelectors.add(selector);
|
||||||
|
}
|
||||||
|
|
||||||
final int batchSize = query.getBatchSize();
|
final int batchSize = query.getBatchSize();
|
||||||
return new Iterator<ScanResultValue>()
|
return new Iterator<ScanResultValue>()
|
||||||
{
|
{
|
||||||
@ -163,15 +176,15 @@ public class ScanQueryEngine
|
|||||||
if (hasTimeout && System.currentTimeMillis() >= timeoutAt) {
|
if (hasTimeout && System.currentTimeMillis() >= timeoutAt) {
|
||||||
throw new QueryInterruptedException(new TimeoutException());
|
throw new QueryInterruptedException(new TimeoutException());
|
||||||
}
|
}
|
||||||
long lastOffset = offset;
|
final long lastOffset = offset;
|
||||||
Object events = null;
|
final Object events;
|
||||||
String resultFormat = query.getResultFormat();
|
final String resultFormat = query.getResultFormat();
|
||||||
if (ScanQuery.RESULT_FORMAT_VALUE_VECTOR.equals(resultFormat)) {
|
if (ScanQuery.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat)) {
|
||||||
throw new UnsupportedOperationException("valueVector is not supported now");
|
|
||||||
} else if (ScanQuery.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat)) {
|
|
||||||
events = rowsToCompactedList();
|
events = rowsToCompactedList();
|
||||||
} else {
|
} else if (ScanQuery.RESULT_FORMAT_LIST.equals(resultFormat)) {
|
||||||
events = rowsToList();
|
events = rowsToList();
|
||||||
|
} else {
|
||||||
|
throw new UOE("resultFormat[%s] is not supported", resultFormat);
|
||||||
}
|
}
|
||||||
responseContext.put(
|
responseContext.put(
|
||||||
ScanQueryRunnerFactory.CTX_COUNT,
|
ScanQueryRunnerFactory.CTX_COUNT,
|
||||||
@ -192,46 +205,48 @@ public class ScanQueryEngine
|
|||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Object rowsToCompactedList()
|
private List<Object> rowsToCompactedList()
|
||||||
{
|
{
|
||||||
return Lists.transform(
|
final List<Object> events = new ArrayList<>(batchSize);
|
||||||
(List<Map<String, Object>>) rowsToList(),
|
|
||||||
new Function<Map<String, Object>, Object>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Object apply(Map<String, Object> input)
|
|
||||||
{
|
|
||||||
List eventValues = Lists.newArrayListWithExpectedSize(allColumns.size());
|
|
||||||
for (String expectedColumn : allColumns) {
|
|
||||||
eventValues.add(input.get(expectedColumn));
|
|
||||||
}
|
|
||||||
return eventValues;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
private Object rowsToList()
|
|
||||||
{
|
|
||||||
List<Map<String, Object>> events = Lists.newArrayListWithCapacity(batchSize);
|
|
||||||
for (int i = 0; !cursor.isDone()
|
for (int i = 0; !cursor.isDone()
|
||||||
&& i < batchSize
|
&& i < batchSize
|
||||||
&& offset < limit; cursor.advance(), i++, offset++) {
|
&& offset < limit; cursor.advance(), i++, offset++) {
|
||||||
final Map<String, Object> theEvent = SelectQueryEngine.singleEvent(
|
final List<Object> theEvent = new ArrayList<>(allColumns.size());
|
||||||
ScanResultValue.timestampKey,
|
for (int j = 0; j < allColumns.size(); j++) {
|
||||||
timestampColumnSelector,
|
theEvent.add(getColumnValue(j));
|
||||||
selectorPlusList,
|
}
|
||||||
metSelectors
|
|
||||||
);
|
|
||||||
events.add(theEvent);
|
events.add(theEvent);
|
||||||
}
|
}
|
||||||
return events;
|
return events;
|
||||||
}
|
}
|
||||||
|
|
||||||
private Object rowsToValueVector()
|
private List<Map<String, Object>> rowsToList()
|
||||||
{
|
{
|
||||||
// only support list now, we can support ValueVector or Arrow in future
|
List<Map<String, Object>> events = Lists.newArrayListWithCapacity(batchSize);
|
||||||
return rowsToList();
|
for (int i = 0; !cursor.isDone()
|
||||||
|
&& i < batchSize
|
||||||
|
&& offset < limit; cursor.advance(), i++, offset++) {
|
||||||
|
final Map<String, Object> theEvent = new LinkedHashMap<>();
|
||||||
|
for (int j = 0; j < allColumns.size(); j++) {
|
||||||
|
theEvent.put(allColumns.get(j), getColumnValue(j));
|
||||||
|
}
|
||||||
|
events.add(theEvent);
|
||||||
|
}
|
||||||
|
return events;
|
||||||
|
}
|
||||||
|
|
||||||
|
private Object getColumnValue(int i)
|
||||||
|
{
|
||||||
|
final ObjectColumnSelector selector = columnSelectors.get(i);
|
||||||
|
final Object value;
|
||||||
|
|
||||||
|
if (legacy && allColumns.get(i).equals(LEGACY_TIMESTAMP_KEY)) {
|
||||||
|
value = DateTimes.utc((long) selector.get());
|
||||||
|
} else {
|
||||||
|
value = selector == null ? null : selector.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
return value;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
@ -16,6 +16,7 @@
|
|||||||
* specific language governing permissions and limitations
|
* specific language governing permissions and limitations
|
||||||
* under the License.
|
* under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package io.druid.query.scan;
|
package io.druid.query.scan;
|
||||||
|
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
@ -41,11 +42,16 @@ public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, Sca
|
|||||||
{
|
{
|
||||||
};
|
};
|
||||||
|
|
||||||
|
private final ScanQueryConfig scanQueryConfig;
|
||||||
private final GenericQueryMetricsFactory queryMetricsFactory;
|
private final GenericQueryMetricsFactory queryMetricsFactory;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public ScanQueryQueryToolChest(GenericQueryMetricsFactory queryMetricsFactory)
|
public ScanQueryQueryToolChest(
|
||||||
|
final ScanQueryConfig scanQueryConfig,
|
||||||
|
final GenericQueryMetricsFactory queryMetricsFactory
|
||||||
|
)
|
||||||
{
|
{
|
||||||
|
this.scanQueryConfig = scanQueryConfig;
|
||||||
this.queryMetricsFactory = queryMetricsFactory;
|
this.queryMetricsFactory = queryMetricsFactory;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -59,9 +65,13 @@ public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, Sca
|
|||||||
final QueryPlus<ScanResultValue> queryPlus, final Map<String, Object> responseContext
|
final QueryPlus<ScanResultValue> queryPlus, final Map<String, Object> responseContext
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
ScanQuery scanQuery = (ScanQuery) queryPlus.getQuery();
|
// Ensure "legacy" is a non-null value, such that all other nodes this query is forwarded to will treat it
|
||||||
|
// the same way, even if they have different default legacy values.
|
||||||
|
final ScanQuery scanQuery = ((ScanQuery) queryPlus.getQuery()).withNonNullLegacy(scanQueryConfig);
|
||||||
|
final QueryPlus<ScanResultValue> queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery);
|
||||||
|
|
||||||
if (scanQuery.getLimit() == Long.MAX_VALUE) {
|
if (scanQuery.getLimit() == Long.MAX_VALUE) {
|
||||||
return runner.run(queryPlus, responseContext);
|
return runner.run(queryPlusWithNonNullLegacy, responseContext);
|
||||||
}
|
}
|
||||||
return new BaseSequence<>(
|
return new BaseSequence<>(
|
||||||
new BaseSequence.IteratorMaker<ScanResultValue, ScanQueryLimitRowIterator>()
|
new BaseSequence.IteratorMaker<ScanResultValue, ScanQueryLimitRowIterator>()
|
||||||
@ -69,7 +79,7 @@ public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, Sca
|
|||||||
@Override
|
@Override
|
||||||
public ScanQueryLimitRowIterator make()
|
public ScanQueryLimitRowIterator make()
|
||||||
{
|
{
|
||||||
return new ScanQueryLimitRowIterator(runner, queryPlus, responseContext);
|
return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -114,8 +124,8 @@ public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, Sca
|
|||||||
)
|
)
|
||||||
{
|
{
|
||||||
ScanQuery scanQuery = (ScanQuery) queryPlus.getQuery();
|
ScanQuery scanQuery = (ScanQuery) queryPlus.getQuery();
|
||||||
if (scanQuery.getDimensionsFilter() != null) {
|
if (scanQuery.getFilter() != null) {
|
||||||
scanQuery = scanQuery.withDimFilter(scanQuery.getDimensionsFilter().optimize());
|
scanQuery = scanQuery.withDimFilter(scanQuery.getFilter().optimize());
|
||||||
queryPlus = queryPlus.withQuery(scanQuery);
|
queryPlus = queryPlus.withQuery(scanQuery);
|
||||||
}
|
}
|
||||||
return runner.run(queryPlus, responseContext);
|
return runner.run(queryPlus, responseContext);
|
@ -25,8 +25,6 @@ import java.util.List;
|
|||||||
|
|
||||||
public class ScanResultValue implements Comparable<ScanResultValue>
|
public class ScanResultValue implements Comparable<ScanResultValue>
|
||||||
{
|
{
|
||||||
public static final String timestampKey = "timestamp";
|
|
||||||
|
|
||||||
private final String segmentId;
|
private final String segmentId;
|
||||||
private final List<String> columns;
|
private final List<String> columns;
|
||||||
private final Object events;
|
private final Object events;
|
@ -64,6 +64,7 @@ import java.util.Map;
|
|||||||
public class MultiSegmentScanQueryTest
|
public class MultiSegmentScanQueryTest
|
||||||
{
|
{
|
||||||
private static final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest(
|
private static final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest(
|
||||||
|
new ScanQueryConfig(),
|
||||||
DefaultGenericQueryMetricsFactory.instance()
|
DefaultGenericQueryMetricsFactory.instance()
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -184,6 +185,7 @@ public class MultiSegmentScanQueryTest
|
|||||||
.intervals(SelectQueryRunnerTest.I_0112_0114)
|
.intervals(SelectQueryRunnerTest.I_0112_0114)
|
||||||
.batchSize(batchSize)
|
.batchSize(batchSize)
|
||||||
.columns(Arrays.<String>asList())
|
.columns(Arrays.<String>asList())
|
||||||
|
.legacy(false)
|
||||||
.limit(limit);
|
.limit(limit);
|
||||||
}
|
}
|
||||||
|
|
@ -20,20 +20,25 @@
|
|||||||
package io.druid.query.scan;
|
package io.druid.query.scan;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.ObjectArrays;
|
import com.google.common.collect.ObjectArrays;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
|
import com.google.common.hash.Hashing;
|
||||||
|
import io.druid.hll.HyperLogLogCollector;
|
||||||
import io.druid.java.util.common.DateTimes;
|
import io.druid.java.util.common.DateTimes;
|
||||||
import io.druid.java.util.common.ISE;
|
import io.druid.java.util.common.ISE;
|
||||||
import io.druid.java.util.common.Intervals;
|
import io.druid.java.util.common.Intervals;
|
||||||
|
import io.druid.java.util.common.StringUtils;
|
||||||
import io.druid.java.util.common.guava.Sequences;
|
import io.druid.java.util.common.guava.Sequences;
|
||||||
import io.druid.query.DefaultGenericQueryMetricsFactory;
|
import io.druid.query.DefaultGenericQueryMetricsFactory;
|
||||||
import io.druid.query.QueryPlus;
|
import io.druid.query.QueryPlus;
|
||||||
import io.druid.query.QueryRunner;
|
import io.druid.query.QueryRunner;
|
||||||
import io.druid.query.QueryRunnerTestHelper;
|
import io.druid.query.QueryRunnerTestHelper;
|
||||||
import io.druid.query.TableDataSource;
|
import io.druid.query.TableDataSource;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.query.extraction.MapLookupExtractor;
|
import io.druid.query.extraction.MapLookupExtractor;
|
||||||
import io.druid.query.filter.AndDimFilter;
|
import io.druid.query.filter.AndDimFilter;
|
||||||
import io.druid.query.filter.DimFilter;
|
import io.druid.query.filter.DimFilter;
|
||||||
@ -41,6 +46,11 @@ import io.druid.query.filter.SelectorDimFilter;
|
|||||||
import io.druid.query.lookup.LookupExtractionFn;
|
import io.druid.query.lookup.LookupExtractionFn;
|
||||||
import io.druid.query.spec.LegacySegmentSpec;
|
import io.druid.query.spec.LegacySegmentSpec;
|
||||||
import io.druid.query.spec.QuerySegmentSpec;
|
import io.druid.query.spec.QuerySegmentSpec;
|
||||||
|
import io.druid.segment.VirtualColumn;
|
||||||
|
import io.druid.segment.column.Column;
|
||||||
|
import io.druid.segment.column.ValueType;
|
||||||
|
import io.druid.segment.virtual.ExpressionVirtualColumn;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.runner.RunWith;
|
import org.junit.runner.RunWith;
|
||||||
@ -61,6 +71,9 @@ import java.util.Set;
|
|||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public class ScanQueryRunnerTest
|
public class ScanQueryRunnerTest
|
||||||
{
|
{
|
||||||
|
private static final VirtualColumn EXPR_COLUMN =
|
||||||
|
new ExpressionVirtualColumn("expr", "index * 2", ValueType.LONG, TestExprMacroTable.INSTANCE);
|
||||||
|
|
||||||
// copied from druid.sample.numeric.tsv
|
// copied from druid.sample.numeric.tsv
|
||||||
public static final String[] V_0112 = {
|
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-12T00:00:00.000Z\tspot\tautomotive\t1000\t10000.0\t10000.0\t100000\tpreferred\tapreferred\t100.000000",
|
||||||
@ -99,10 +112,11 @@ public class ScanQueryRunnerTest
|
|||||||
public static final String[] V_0112_0114 = ObjectArrays.concat(V_0112, V_0113, String.class);
|
public static final String[] V_0112_0114 = ObjectArrays.concat(V_0112, V_0113, String.class);
|
||||||
|
|
||||||
private static final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest(
|
private static final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest(
|
||||||
|
new ScanQueryConfig(),
|
||||||
DefaultGenericQueryMetricsFactory.instance()
|
DefaultGenericQueryMetricsFactory.instance()
|
||||||
);
|
);
|
||||||
|
|
||||||
@Parameterized.Parameters(name = "{0}")
|
@Parameterized.Parameters(name = "{0}, legacy = {1}")
|
||||||
public static Iterable<Object[]> constructorFeeder() throws IOException
|
public static Iterable<Object[]> constructorFeeder() throws IOException
|
||||||
{
|
{
|
||||||
return QueryRunnerTestHelper.cartesian(
|
return QueryRunnerTestHelper.cartesian(
|
||||||
@ -111,15 +125,18 @@ public class ScanQueryRunnerTest
|
|||||||
toolChest,
|
toolChest,
|
||||||
new ScanQueryEngine()
|
new ScanQueryEngine()
|
||||||
)
|
)
|
||||||
)
|
),
|
||||||
|
ImmutableList.of(false, true)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
private final QueryRunner runner;
|
private final QueryRunner runner;
|
||||||
|
private final boolean legacy;
|
||||||
|
|
||||||
public ScanQueryRunnerTest(QueryRunner runner)
|
public ScanQueryRunnerTest(final QueryRunner runner, final boolean legacy)
|
||||||
{
|
{
|
||||||
this.runner = runner;
|
this.runner = runner;
|
||||||
|
this.legacy = legacy;
|
||||||
}
|
}
|
||||||
|
|
||||||
private ScanQuery.ScanQueryBuilder newTestQuery()
|
private ScanQuery.ScanQueryBuilder newTestQuery()
|
||||||
@ -128,14 +145,16 @@ public class ScanQueryRunnerTest
|
|||||||
.dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource))
|
.dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource))
|
||||||
.columns(Arrays.<String>asList())
|
.columns(Arrays.<String>asList())
|
||||||
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
.limit(3);
|
.limit(3)
|
||||||
|
.legacy(legacy);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testFullOnSelect()
|
public void testFullOnSelect()
|
||||||
{
|
{
|
||||||
List<String> columns = Lists.newArrayList(
|
List<String> columns = Lists.newArrayList(
|
||||||
ScanResultValue.timestampKey,
|
getTimestampName(),
|
||||||
|
"expr",
|
||||||
"market",
|
"market",
|
||||||
"quality",
|
"quality",
|
||||||
"qualityLong",
|
"qualityLong",
|
||||||
@ -156,6 +175,7 @@ public class ScanQueryRunnerTest
|
|||||||
);
|
);
|
||||||
ScanQuery query = newTestQuery()
|
ScanQuery query = newTestQuery()
|
||||||
.intervals(I_0112_0114)
|
.intervals(I_0112_0114)
|
||||||
|
.virtualColumns(EXPR_COLUMN)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||||
@ -177,7 +197,8 @@ public class ScanQueryRunnerTest
|
|||||||
public void testFullOnSelectAsCompactedList()
|
public void testFullOnSelectAsCompactedList()
|
||||||
{
|
{
|
||||||
final List<String> columns = Lists.newArrayList(
|
final List<String> columns = Lists.newArrayList(
|
||||||
ScanResultValue.timestampKey,
|
getTimestampName(),
|
||||||
|
"expr",
|
||||||
"market",
|
"market",
|
||||||
"quality",
|
"quality",
|
||||||
"qualityLong",
|
"qualityLong",
|
||||||
@ -198,6 +219,7 @@ public class ScanQueryRunnerTest
|
|||||||
);
|
);
|
||||||
ScanQuery query = newTestQuery()
|
ScanQuery query = newTestQuery()
|
||||||
.intervals(I_0112_0114)
|
.intervals(I_0112_0114)
|
||||||
|
.virtualColumns(EXPR_COLUMN)
|
||||||
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
|
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
@ -216,6 +238,56 @@ public class ScanQueryRunnerTest
|
|||||||
verify(expectedResults, populateNullColumnAtLastForQueryableIndexCase(compactedListToRow(results), "null_column"));
|
verify(expectedResults, populateNullColumnAtLastForQueryableIndexCase(compactedListToRow(results), "null_column"));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSelectWithUnderscoreUnderscoreTime()
|
||||||
|
{
|
||||||
|
ScanQuery query = newTestQuery()
|
||||||
|
.intervals(I_0112_0114)
|
||||||
|
.columns(Column.TIME_COLUMN_NAME, QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.indexMetric)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||||
|
Iterable<ScanResultValue> results = Sequences.toList(
|
||||||
|
runner.run(QueryPlus.wrap(query), context),
|
||||||
|
Lists.<ScanResultValue>newArrayList()
|
||||||
|
);
|
||||||
|
|
||||||
|
final List<List<Map<String, Object>>> expectedEvents = toEvents(
|
||||||
|
new String[]{
|
||||||
|
getTimestampName() + ":TIME",
|
||||||
|
QueryRunnerTestHelper.marketDimension + ":STRING",
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
QueryRunnerTestHelper.indexMetric + ":DOUBLE"
|
||||||
|
},
|
||||||
|
V_0112_0114
|
||||||
|
);
|
||||||
|
|
||||||
|
// Add "__time" to all the expected events in legacy mode
|
||||||
|
if (legacy) {
|
||||||
|
for (List<Map<String, Object>> batch : expectedEvents) {
|
||||||
|
for (Map<String, Object> event : batch) {
|
||||||
|
event.put("__time", ((DateTime) event.get("timestamp")).getMillis());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
List<ScanResultValue> expectedResults = toExpected(
|
||||||
|
expectedEvents,
|
||||||
|
legacy
|
||||||
|
? Lists.newArrayList(getTimestampName(), "__time", "market", "index")
|
||||||
|
: Lists.newArrayList("__time", "market", "index"),
|
||||||
|
0,
|
||||||
|
3
|
||||||
|
);
|
||||||
|
verify(expectedResults, results);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSelectWithDimsAndMets()
|
public void testSelectWithDimsAndMets()
|
||||||
{
|
{
|
||||||
@ -233,7 +305,7 @@ public class ScanQueryRunnerTest
|
|||||||
List<ScanResultValue> expectedResults = toExpected(
|
List<ScanResultValue> expectedResults = toExpected(
|
||||||
toEvents(
|
toEvents(
|
||||||
new String[]{
|
new String[]{
|
||||||
ScanResultValue.timestampKey + ":TIME",
|
legacy ? getTimestampName() + ":TIME" : null,
|
||||||
QueryRunnerTestHelper.marketDimension + ":STRING",
|
QueryRunnerTestHelper.marketDimension + ":STRING",
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
@ -246,7 +318,7 @@ public class ScanQueryRunnerTest
|
|||||||
},
|
},
|
||||||
V_0112_0114
|
V_0112_0114
|
||||||
),
|
),
|
||||||
Lists.newArrayList(ScanResultValue.timestampKey, "market", "index"),
|
legacy ? Lists.newArrayList(getTimestampName(), "market", "index") : Lists.newArrayList("market", "index"),
|
||||||
0,
|
0,
|
||||||
3
|
3
|
||||||
);
|
);
|
||||||
@ -271,7 +343,7 @@ public class ScanQueryRunnerTest
|
|||||||
List<ScanResultValue> expectedResults = toExpected(
|
List<ScanResultValue> expectedResults = toExpected(
|
||||||
toEvents(
|
toEvents(
|
||||||
new String[]{
|
new String[]{
|
||||||
ScanResultValue.timestampKey + ":TIME",
|
legacy ? getTimestampName() + ":TIME" : null,
|
||||||
QueryRunnerTestHelper.marketDimension + ":STRING",
|
QueryRunnerTestHelper.marketDimension + ":STRING",
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
@ -284,7 +356,7 @@ public class ScanQueryRunnerTest
|
|||||||
},
|
},
|
||||||
V_0112_0114
|
V_0112_0114
|
||||||
),
|
),
|
||||||
Lists.newArrayList(ScanResultValue.timestampKey, "market", "index"),
|
legacy ? Lists.newArrayList(getTimestampName(), "market", "index") : Lists.newArrayList("market", "index"),
|
||||||
0,
|
0,
|
||||||
3
|
3
|
||||||
);
|
);
|
||||||
@ -311,7 +383,7 @@ public class ScanQueryRunnerTest
|
|||||||
|
|
||||||
final List<List<Map<String, Object>>> events = toEvents(
|
final List<List<Map<String, Object>>> events = toEvents(
|
||||||
new String[]{
|
new String[]{
|
||||||
ScanResultValue.timestampKey + ":TIME",
|
legacy ? getTimestampName() + ":TIME" : null,
|
||||||
null,
|
null,
|
||||||
QueryRunnerTestHelper.qualityDimension + ":STRING",
|
QueryRunnerTestHelper.qualityDimension + ":STRING",
|
||||||
null,
|
null,
|
||||||
@ -345,7 +417,7 @@ public class ScanQueryRunnerTest
|
|||||||
|
|
||||||
List<ScanResultValue> expectedResults = toExpected(
|
List<ScanResultValue> expectedResults = toExpected(
|
||||||
events,
|
events,
|
||||||
Lists.newArrayList(ScanResultValue.timestampKey, "quality", "index"),
|
legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"),
|
||||||
0,
|
0,
|
||||||
limit
|
limit
|
||||||
);
|
);
|
||||||
@ -378,7 +450,7 @@ public class ScanQueryRunnerTest
|
|||||||
|
|
||||||
final List<List<Map<String, Object>>> events = toEvents(
|
final List<List<Map<String, Object>>> events = toEvents(
|
||||||
new String[]{
|
new String[]{
|
||||||
ScanResultValue.timestampKey + ":TIME",
|
legacy ? getTimestampName() + ":TIME" : null,
|
||||||
null,
|
null,
|
||||||
QueryRunnerTestHelper.qualityDimension + ":STRING",
|
QueryRunnerTestHelper.qualityDimension + ":STRING",
|
||||||
null,
|
null,
|
||||||
@ -398,7 +470,14 @@ public class ScanQueryRunnerTest
|
|||||||
|
|
||||||
List<ScanResultValue> expectedResults = toExpected(
|
List<ScanResultValue> expectedResults = toExpected(
|
||||||
events,
|
events,
|
||||||
Lists.newArrayList(ScanResultValue.timestampKey, QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric),
|
legacy ? Lists.newArrayList(
|
||||||
|
getTimestampName(),
|
||||||
|
QueryRunnerTestHelper.qualityDimension,
|
||||||
|
QueryRunnerTestHelper.indexMetric
|
||||||
|
) : Lists.newArrayList(
|
||||||
|
QueryRunnerTestHelper.qualityDimension,
|
||||||
|
QueryRunnerTestHelper.indexMetric
|
||||||
|
),
|
||||||
0,
|
0,
|
||||||
3
|
3
|
||||||
);
|
);
|
||||||
@ -446,18 +525,17 @@ public class ScanQueryRunnerTest
|
|||||||
);
|
);
|
||||||
|
|
||||||
final List<List<Map<String, Object>>> events = toEvents(
|
final List<List<Map<String, Object>>> events = toEvents(
|
||||||
new String[]{
|
legacy ? new String[]{getTimestampName() + ":TIME"} : new String[0],
|
||||||
ScanResultValue.timestampKey + ":TIME"
|
|
||||||
},
|
|
||||||
V_0112_0114
|
V_0112_0114
|
||||||
);
|
);
|
||||||
|
|
||||||
List<ScanResultValue> expectedResults = toExpected(
|
List<ScanResultValue> expectedResults = toExpected(
|
||||||
events,
|
events,
|
||||||
Lists.<String>newArrayList(ScanResultValue.timestampKey, "foo", "foo2"),
|
legacy ? Lists.newArrayList(getTimestampName(), "foo", "foo2") : Lists.newArrayList("foo", "foo2"),
|
||||||
0,
|
0,
|
||||||
3
|
3
|
||||||
);
|
);
|
||||||
|
|
||||||
verify(expectedResults, results);
|
verify(expectedResults, results);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -465,7 +543,7 @@ public class ScanQueryRunnerTest
|
|||||||
{
|
{
|
||||||
return toEvents(
|
return toEvents(
|
||||||
new String[]{
|
new String[]{
|
||||||
ScanResultValue.timestampKey + ":TIME",
|
getTimestampName() + ":TIME",
|
||||||
QueryRunnerTestHelper.marketDimension + ":STRING",
|
QueryRunnerTestHelper.marketDimension + ":STRING",
|
||||||
QueryRunnerTestHelper.qualityDimension + ":STRING",
|
QueryRunnerTestHelper.qualityDimension + ":STRING",
|
||||||
"qualityLong" + ":LONG",
|
"qualityLong" + ":LONG",
|
||||||
@ -475,7 +553,14 @@ public class ScanQueryRunnerTest
|
|||||||
QueryRunnerTestHelper.placementDimension + ":STRING",
|
QueryRunnerTestHelper.placementDimension + ":STRING",
|
||||||
QueryRunnerTestHelper.placementishDimension + ":STRINGS",
|
QueryRunnerTestHelper.placementishDimension + ":STRINGS",
|
||||||
QueryRunnerTestHelper.indexMetric + ":DOUBLE",
|
QueryRunnerTestHelper.indexMetric + ":DOUBLE",
|
||||||
QueryRunnerTestHelper.partialNullDimension + ":STRING"
|
QueryRunnerTestHelper.partialNullDimension + ":STRING",
|
||||||
|
"expr",
|
||||||
|
"indexMin",
|
||||||
|
"indexFloat",
|
||||||
|
"indexMaxPlusTen",
|
||||||
|
"indexMinFloat",
|
||||||
|
"indexMaxFloat",
|
||||||
|
"quality_uniques"
|
||||||
},
|
},
|
||||||
valueSet
|
valueSet
|
||||||
);
|
);
|
||||||
@ -499,14 +584,53 @@ public class ScanQueryRunnerTest
|
|||||||
Map<String, Object> event = Maps.newHashMap();
|
Map<String, Object> event = Maps.newHashMap();
|
||||||
String[] values = input.split("\\t");
|
String[] values = input.split("\\t");
|
||||||
for (int i = 0; i < dimSpecs.length; i++) {
|
for (int i = 0; i < dimSpecs.length; i++) {
|
||||||
if (dimSpecs[i] == null || i >= dimSpecs.length || i >= values.length) {
|
if (dimSpecs[i] == null || i >= dimSpecs.length) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// For testing metrics and virtual columns we have some special handling here, since
|
||||||
|
// they don't appear in the source data.
|
||||||
|
if (dimSpecs[i].equals(EXPR_COLUMN.getOutputName())) {
|
||||||
|
event.put(
|
||||||
|
EXPR_COLUMN.getOutputName(),
|
||||||
|
(double) event.get(QueryRunnerTestHelper.indexMetric) * 2
|
||||||
|
);
|
||||||
|
continue;
|
||||||
|
} else if (dimSpecs[i].equals("indexMin")) {
|
||||||
|
event.put("indexMin", (double) event.get(QueryRunnerTestHelper.indexMetric));
|
||||||
|
continue;
|
||||||
|
} else if (dimSpecs[i].equals("indexFloat")) {
|
||||||
|
event.put("indexFloat", (float) (double) event.get(QueryRunnerTestHelper.indexMetric));
|
||||||
|
continue;
|
||||||
|
} else if (dimSpecs[i].equals("indexMaxPlusTen")) {
|
||||||
|
event.put("indexMaxPlusTen", (double) event.get(QueryRunnerTestHelper.indexMetric) + 10);
|
||||||
|
continue;
|
||||||
|
} else if (dimSpecs[i].equals("indexMinFloat")) {
|
||||||
|
event.put("indexMinFloat", (float) (double) event.get(QueryRunnerTestHelper.indexMetric));
|
||||||
|
continue;
|
||||||
|
} else if (dimSpecs[i].equals("indexMaxFloat")) {
|
||||||
|
event.put("indexMaxFloat", (float) (double) event.get(QueryRunnerTestHelper.indexMetric));
|
||||||
|
continue;
|
||||||
|
} else if (dimSpecs[i].equals("quality_uniques")) {
|
||||||
|
final HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector();
|
||||||
|
collector.add(
|
||||||
|
Hashing.murmur3_128()
|
||||||
|
.hashBytes(StringUtils.toUtf8((String) event.get("quality")))
|
||||||
|
.asBytes()
|
||||||
|
);
|
||||||
|
event.put("quality_uniques", collector);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (i >= values.length) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
String[] specs = dimSpecs[i].split(":");
|
String[] specs = dimSpecs[i].split(":");
|
||||||
|
|
||||||
event.put(
|
event.put(
|
||||||
specs[0],
|
specs[0],
|
||||||
specs.length == 1 || specs[1].equals("STRING") ? values[i] :
|
specs.length == 1 || specs[1].equals("STRING") ? values[i] :
|
||||||
specs[1].equals("TIME") ? DateTimes.of(values[i]) :
|
specs[1].equals("TIME") ? toTimestamp(values[i]) :
|
||||||
specs[1].equals("FLOAT") ? Float.valueOf(values[i]) :
|
specs[1].equals("FLOAT") ? Float.valueOf(values[i]) :
|
||||||
specs[1].equals("DOUBLE") ? Double.valueOf(values[i]) :
|
specs[1].equals("DOUBLE") ? Double.valueOf(values[i]) :
|
||||||
specs[1].equals("LONG") ? Long.valueOf(values[i]) :
|
specs[1].equals("LONG") ? Long.valueOf(values[i]) :
|
||||||
@ -524,6 +648,20 @@ public class ScanQueryRunnerTest
|
|||||||
return events;
|
return events;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Object toTimestamp(final String value)
|
||||||
|
{
|
||||||
|
if (legacy) {
|
||||||
|
return DateTimes.of(value);
|
||||||
|
} else {
|
||||||
|
return DateTimes.of(value).getMillis();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private String getTimestampName()
|
||||||
|
{
|
||||||
|
return legacy ? "timestamp" : Column.TIME_COLUMN_NAME;
|
||||||
|
}
|
||||||
|
|
||||||
private List<ScanResultValue> toExpected(
|
private List<ScanResultValue> toExpected(
|
||||||
List<List<Map<String, Object>>> targets,
|
List<List<Map<String, Object>>> targets,
|
||||||
List<String> columns,
|
List<String> columns,
|
||||||
@ -578,9 +716,23 @@ public class ScanQueryRunnerTest
|
|||||||
for (Map.Entry<String, Object> ex : exHolder.entrySet()) {
|
for (Map.Entry<String, Object> ex : exHolder.entrySet()) {
|
||||||
Object actVal = acHolder.get(ex.getKey());
|
Object actVal = acHolder.get(ex.getKey());
|
||||||
|
|
||||||
// work around for current II limitations
|
if (actVal instanceof String[]) {
|
||||||
|
actVal = Arrays.asList((String[]) actVal);
|
||||||
|
}
|
||||||
|
|
||||||
Assert.assertEquals("invalid value for " + ex.getKey(), ex.getValue(), actVal);
|
Assert.assertEquals("invalid value for " + ex.getKey(), ex.getValue(), actVal);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
for (Map.Entry<String, Object> ac : acHolder.entrySet()) {
|
||||||
|
Object exVal = exHolder.get(ac.getKey());
|
||||||
|
Object actVal = ac.getValue();
|
||||||
|
|
||||||
|
if (actVal instanceof String[]) {
|
||||||
|
actVal = Arrays.asList((String[]) actVal);
|
||||||
|
}
|
||||||
|
|
||||||
|
Assert.assertEquals("invalid value for " + ac.getKey(), exVal, actVal);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (actualEvts.hasNext()) {
|
if (actualEvts.hasNext()) {
|
||||||
@ -612,7 +764,7 @@ public class ScanQueryRunnerTest
|
|||||||
|
|
||||||
private Iterable<ScanResultValue> compactedListToRow(Iterable<ScanResultValue> results)
|
private Iterable<ScanResultValue> compactedListToRow(Iterable<ScanResultValue> results)
|
||||||
{
|
{
|
||||||
return Iterables.transform(results, new Function<ScanResultValue, ScanResultValue>()
|
return Lists.newArrayList(Iterables.transform(results, new Function<ScanResultValue, ScanResultValue>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public ScanResultValue apply(ScanResultValue input)
|
public ScanResultValue apply(ScanResultValue input)
|
||||||
@ -629,6 +781,6 @@ public class ScanQueryRunnerTest
|
|||||||
}
|
}
|
||||||
return new ScanResultValue(input.getSegmentId(), input.getColumns(), mapEvents);
|
return new ScanResultValue(input.getSegmentId(), input.getColumns(), mapEvents);
|
||||||
}
|
}
|
||||||
});
|
}));
|
||||||
}
|
}
|
||||||
}
|
}
|
@ -25,6 +25,7 @@ import io.druid.java.util.common.Intervals;
|
|||||||
import io.druid.query.QueryRunnerTestHelper;
|
import io.druid.query.QueryRunnerTestHelper;
|
||||||
import io.druid.query.TableDataSource;
|
import io.druid.query.TableDataSource;
|
||||||
import io.druid.query.spec.LegacySegmentSpec;
|
import io.druid.query.spec.LegacySegmentSpec;
|
||||||
|
import io.druid.segment.VirtualColumns;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
@ -48,22 +49,26 @@ public class ScanQuerySpecTest
|
|||||||
String current =
|
String current =
|
||||||
"{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},"
|
"{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},"
|
||||||
+ "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},"
|
+ "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},"
|
||||||
|
+ "\"virtualColumns\":[],"
|
||||||
+ "\"resultFormat\":\"list\","
|
+ "\"resultFormat\":\"list\","
|
||||||
+ "\"batchSize\":20480,"
|
+ "\"batchSize\":20480,"
|
||||||
+ "\"limit\":3,"
|
+ "\"limit\":3,"
|
||||||
+ "\"filter\":null,"
|
+ "\"filter\":null,"
|
||||||
+ "\"columns\":[\"market\",\"quality\",\"index\"],"
|
+ "\"columns\":[\"market\",\"quality\",\"index\"],"
|
||||||
|
+ "\"legacy\":null,"
|
||||||
+ "\"context\":null,"
|
+ "\"context\":null,"
|
||||||
+ "\"descending\":false}";
|
+ "\"descending\":false}";
|
||||||
|
|
||||||
ScanQuery query = new ScanQuery(
|
ScanQuery query = new ScanQuery(
|
||||||
new TableDataSource(QueryRunnerTestHelper.dataSource),
|
new TableDataSource(QueryRunnerTestHelper.dataSource),
|
||||||
new LegacySegmentSpec(Intervals.of("2011-01-12/2011-01-14")),
|
new LegacySegmentSpec(Intervals.of("2011-01-12/2011-01-14")),
|
||||||
|
VirtualColumns.EMPTY,
|
||||||
null,
|
null,
|
||||||
0,
|
0,
|
||||||
3,
|
3,
|
||||||
null,
|
null,
|
||||||
Arrays.<String>asList("market", "quality", "index"),
|
Arrays.<String>asList("market", "quality", "index"),
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
@ -32,6 +32,8 @@ import io.druid.query.groupby.GroupByQueryEngine;
|
|||||||
import io.druid.query.groupby.GroupByQueryRunnerFactory;
|
import io.druid.query.groupby.GroupByQueryRunnerFactory;
|
||||||
import io.druid.query.metadata.SegmentMetadataQueryRunnerFactory;
|
import io.druid.query.metadata.SegmentMetadataQueryRunnerFactory;
|
||||||
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||||
|
import io.druid.query.scan.ScanQuery;
|
||||||
|
import io.druid.query.scan.ScanQueryRunnerFactory;
|
||||||
import io.druid.query.search.SearchQueryRunnerFactory;
|
import io.druid.query.search.SearchQueryRunnerFactory;
|
||||||
import io.druid.query.search.search.SearchQuery;
|
import io.druid.query.search.search.SearchQuery;
|
||||||
import io.druid.query.select.SelectQuery;
|
import io.druid.query.select.SelectQuery;
|
||||||
@ -57,6 +59,7 @@ public class QueryRunnerFactoryModule extends QueryToolChestModule
|
|||||||
.put(TimeBoundaryQuery.class, TimeBoundaryQueryRunnerFactory.class)
|
.put(TimeBoundaryQuery.class, TimeBoundaryQueryRunnerFactory.class)
|
||||||
.put(SegmentMetadataQuery.class, SegmentMetadataQueryRunnerFactory.class)
|
.put(SegmentMetadataQuery.class, SegmentMetadataQueryRunnerFactory.class)
|
||||||
.put(GroupByQuery.class, GroupByQueryRunnerFactory.class)
|
.put(GroupByQuery.class, GroupByQueryRunnerFactory.class)
|
||||||
|
.put(ScanQuery.class, ScanQueryRunnerFactory.class)
|
||||||
.put(SelectQuery.class, SelectQueryRunnerFactory.class)
|
.put(SelectQuery.class, SelectQueryRunnerFactory.class)
|
||||||
.put(TopNQuery.class, TopNQueryRunnerFactory.class)
|
.put(TopNQuery.class, TopNQueryRunnerFactory.class)
|
||||||
.put(DataSourceMetadataQuery.class, DataSourceMetadataQueryRunnerFactory.class)
|
.put(DataSourceMetadataQuery.class, DataSourceMetadataQueryRunnerFactory.class)
|
||||||
|
@ -40,6 +40,9 @@ import io.druid.query.groupby.GroupByQueryQueryToolChest;
|
|||||||
import io.druid.query.metadata.SegmentMetadataQueryConfig;
|
import io.druid.query.metadata.SegmentMetadataQueryConfig;
|
||||||
import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest;
|
import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest;
|
||||||
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||||
|
import io.druid.query.scan.ScanQuery;
|
||||||
|
import io.druid.query.scan.ScanQueryConfig;
|
||||||
|
import io.druid.query.scan.ScanQueryQueryToolChest;
|
||||||
import io.druid.query.search.SearchQueryQueryToolChest;
|
import io.druid.query.search.SearchQueryQueryToolChest;
|
||||||
import io.druid.query.search.search.SearchQuery;
|
import io.druid.query.search.search.SearchQuery;
|
||||||
import io.druid.query.search.search.SearchQueryConfig;
|
import io.druid.query.search.search.SearchQueryConfig;
|
||||||
@ -76,6 +79,7 @@ public class QueryToolChestModule implements Module
|
|||||||
.put(TimeBoundaryQuery.class, TimeBoundaryQueryQueryToolChest.class)
|
.put(TimeBoundaryQuery.class, TimeBoundaryQueryQueryToolChest.class)
|
||||||
.put(SegmentMetadataQuery.class, SegmentMetadataQueryQueryToolChest.class)
|
.put(SegmentMetadataQuery.class, SegmentMetadataQueryQueryToolChest.class)
|
||||||
.put(GroupByQuery.class, GroupByQueryQueryToolChest.class)
|
.put(GroupByQuery.class, GroupByQueryQueryToolChest.class)
|
||||||
|
.put(ScanQuery.class, ScanQueryQueryToolChest.class)
|
||||||
.put(SelectQuery.class, SelectQueryQueryToolChest.class)
|
.put(SelectQuery.class, SelectQueryQueryToolChest.class)
|
||||||
.put(TopNQuery.class, TopNQueryQueryToolChest.class)
|
.put(TopNQuery.class, TopNQueryQueryToolChest.class)
|
||||||
.put(DataSourceMetadataQuery.class, DataSourceQueryQueryToolChest.class)
|
.put(DataSourceMetadataQuery.class, DataSourceQueryQueryToolChest.class)
|
||||||
@ -98,6 +102,7 @@ public class QueryToolChestModule implements Module
|
|||||||
JsonConfigProvider.bind(binder, "druid.query.topN", TopNQueryConfig.class);
|
JsonConfigProvider.bind(binder, "druid.query.topN", TopNQueryConfig.class);
|
||||||
JsonConfigProvider.bind(binder, "druid.query.segmentMetadata", SegmentMetadataQueryConfig.class);
|
JsonConfigProvider.bind(binder, "druid.query.segmentMetadata", SegmentMetadataQueryConfig.class);
|
||||||
JsonConfigProvider.bind(binder, "druid.query.select", SelectQueryConfig.class);
|
JsonConfigProvider.bind(binder, "druid.query.select", SelectQueryConfig.class);
|
||||||
|
JsonConfigProvider.bind(binder, "druid.query.scan", ScanQueryConfig.class);
|
||||||
|
|
||||||
PolyBind.createChoice(
|
PolyBind.createChoice(
|
||||||
binder,
|
binder,
|
||||||
|
@ -21,9 +21,11 @@ package io.druid.sql.calcite.rel;
|
|||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
|
import com.google.common.collect.Ordering;
|
||||||
import io.druid.java.util.common.ISE;
|
import io.druid.java.util.common.ISE;
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
import io.druid.java.util.common.granularity.Granularity;
|
import io.druid.java.util.common.granularity.Granularity;
|
||||||
@ -38,6 +40,7 @@ import io.druid.query.groupby.GroupByQuery;
|
|||||||
import io.druid.query.groupby.having.DimFilterHavingSpec;
|
import io.druid.query.groupby.having.DimFilterHavingSpec;
|
||||||
import io.druid.query.groupby.orderby.DefaultLimitSpec;
|
import io.druid.query.groupby.orderby.DefaultLimitSpec;
|
||||||
import io.druid.query.groupby.orderby.OrderByColumnSpec;
|
import io.druid.query.groupby.orderby.OrderByColumnSpec;
|
||||||
|
import io.druid.query.scan.ScanQuery;
|
||||||
import io.druid.query.select.PagingSpec;
|
import io.druid.query.select.PagingSpec;
|
||||||
import io.druid.query.select.SelectQuery;
|
import io.druid.query.select.SelectQuery;
|
||||||
import io.druid.query.timeseries.TimeseriesQuery;
|
import io.druid.query.timeseries.TimeseriesQuery;
|
||||||
@ -341,7 +344,7 @@ public class DruidQueryBuilder
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* Return this query as some kind of Druid query. The returned query will either be {@link TopNQuery},
|
* Return this query as some kind of Druid query. The returned query will either be {@link TopNQuery},
|
||||||
* {@link TimeseriesQuery}, {@link GroupByQuery}, or {@link SelectQuery}.
|
* {@link TimeseriesQuery}, {@link GroupByQuery}, {@link ScanQuery}, or {@link SelectQuery}.
|
||||||
*
|
*
|
||||||
* @param dataSource data source to query
|
* @param dataSource data source to query
|
||||||
* @param plannerContext planner context
|
* @param plannerContext planner context
|
||||||
@ -380,6 +383,11 @@ public class DruidQueryBuilder
|
|||||||
return groupByQuery;
|
return groupByQuery;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final ScanQuery scanQuery = toScanQuery(dataSource, plannerContext);
|
||||||
|
if (scanQuery != null) {
|
||||||
|
return scanQuery;
|
||||||
|
}
|
||||||
|
|
||||||
final SelectQuery selectQuery = toSelectQuery(dataSource, plannerContext);
|
final SelectQuery selectQuery = toSelectQuery(dataSource, plannerContext);
|
||||||
if (selectQuery != null) {
|
if (selectQuery != null) {
|
||||||
return selectQuery;
|
return selectQuery;
|
||||||
@ -583,6 +591,56 @@ public class DruidQueryBuilder
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return this query as a Scan query, or null if this query is not compatible with Scan.
|
||||||
|
*
|
||||||
|
* @param dataSource data source to query
|
||||||
|
* @param plannerContext planner context
|
||||||
|
*
|
||||||
|
* @return query or null
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
public ScanQuery toScanQuery(
|
||||||
|
final DataSource dataSource,
|
||||||
|
final PlannerContext plannerContext
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (grouping != null) {
|
||||||
|
// Scan cannot GROUP BY.
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (limitSpec != null && limitSpec.getColumns().size() > 0) {
|
||||||
|
// Scan cannot ORDER BY.
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (getRowOrder().isEmpty()) {
|
||||||
|
// Should never do a scan query without any columns that we're interested in. This is probably a planner bug.
|
||||||
|
throw new ISE("WTF?! Attempting to convert to Scan query without any columns?");
|
||||||
|
}
|
||||||
|
|
||||||
|
final Filtration filtration = Filtration.create(filter).optimize(sourceRowSignature);
|
||||||
|
|
||||||
|
// DefaultLimitSpec (which we use to "remember" limits) is int typed, and Integer.MAX_VALUE means "no limit".
|
||||||
|
final long scanLimit = limitSpec == null || limitSpec.getLimit() == Integer.MAX_VALUE
|
||||||
|
? 0L
|
||||||
|
: (long) limitSpec.getLimit();
|
||||||
|
|
||||||
|
return new ScanQuery(
|
||||||
|
dataSource,
|
||||||
|
filtration.getQuerySegmentSpec(),
|
||||||
|
selectProjection != null ? VirtualColumns.create(selectProjection.getVirtualColumns()) : VirtualColumns.EMPTY,
|
||||||
|
ScanQuery.RESULT_FORMAT_COMPACTED_LIST,
|
||||||
|
0,
|
||||||
|
scanLimit,
|
||||||
|
filtration.getDimFilter(),
|
||||||
|
Ordering.natural().sortedCopy(ImmutableSet.copyOf(getRowOrder())),
|
||||||
|
false,
|
||||||
|
plannerContext.getQueryContext()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return this query as a Select query, or null if this query is not compatible with Select.
|
* Return this query as a Select query, or null if this query is not compatible with Select.
|
||||||
*
|
*
|
||||||
|
@ -24,6 +24,7 @@ import com.google.common.base.Function;
|
|||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import io.druid.data.input.Row;
|
import io.druid.data.input.Row;
|
||||||
import io.druid.java.util.common.DateTimes;
|
import io.druid.java.util.common.DateTimes;
|
||||||
@ -35,6 +36,7 @@ import io.druid.query.DataSource;
|
|||||||
import io.druid.query.Query;
|
import io.druid.query.Query;
|
||||||
import io.druid.query.Result;
|
import io.druid.query.Result;
|
||||||
import io.druid.query.groupby.GroupByQuery;
|
import io.druid.query.groupby.GroupByQuery;
|
||||||
|
import io.druid.query.scan.ScanQuery;
|
||||||
import io.druid.query.select.EventHolder;
|
import io.druid.query.select.EventHolder;
|
||||||
import io.druid.query.select.PagingSpec;
|
import io.druid.query.select.PagingSpec;
|
||||||
import io.druid.query.select.SelectQuery;
|
import io.druid.query.select.SelectQuery;
|
||||||
@ -104,6 +106,8 @@ public class QueryMaker
|
|||||||
return executeTopN(queryBuilder, (TopNQuery) query);
|
return executeTopN(queryBuilder, (TopNQuery) query);
|
||||||
} else if (query instanceof GroupByQuery) {
|
} else if (query instanceof GroupByQuery) {
|
||||||
return executeGroupBy(queryBuilder, (GroupByQuery) query);
|
return executeGroupBy(queryBuilder, (GroupByQuery) query);
|
||||||
|
} else if (query instanceof ScanQuery) {
|
||||||
|
return executeScan(queryBuilder, (ScanQuery) query);
|
||||||
} else if (query instanceof SelectQuery) {
|
} else if (query instanceof SelectQuery) {
|
||||||
return executeSelect(queryBuilder, (SelectQuery) query);
|
return executeSelect(queryBuilder, (SelectQuery) query);
|
||||||
} else {
|
} else {
|
||||||
@ -111,6 +115,50 @@ public class QueryMaker
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Sequence<Object[]> executeScan(
|
||||||
|
final DruidQueryBuilder queryBuilder,
|
||||||
|
final ScanQuery query
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
|
||||||
|
|
||||||
|
// SQL row column index -> Scan query column index
|
||||||
|
final int[] columnMapping = new int[queryBuilder.getRowOrder().size()];
|
||||||
|
final Map<String, Integer> scanColumnOrder = Maps.newHashMap();
|
||||||
|
|
||||||
|
for (int i = 0; i < query.getColumns().size(); i++) {
|
||||||
|
scanColumnOrder.put(query.getColumns().get(i), i);
|
||||||
|
}
|
||||||
|
|
||||||
|
for (int i = 0; i < queryBuilder.getRowOrder().size(); i++) {
|
||||||
|
final Integer index = scanColumnOrder.get(queryBuilder.getRowOrder().get(i));
|
||||||
|
columnMapping[i] = index == null ? -1 : index;
|
||||||
|
}
|
||||||
|
|
||||||
|
return Sequences.concat(
|
||||||
|
Sequences.map(
|
||||||
|
runQuery(query),
|
||||||
|
scanResult -> {
|
||||||
|
final List<Object[]> retVals = new ArrayList<>();
|
||||||
|
final List<List<Object>> rows = (List<List<Object>>) scanResult.getEvents();
|
||||||
|
|
||||||
|
for (List<Object> row : rows) {
|
||||||
|
final Object[] retVal = new Object[fieldList.size()];
|
||||||
|
for (RelDataTypeField field : fieldList) {
|
||||||
|
retVal[field.getIndex()] = coerce(
|
||||||
|
row.get(columnMapping[field.getIndex()]),
|
||||||
|
field.getType().getSqlTypeName()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
retVals.add(retVal);
|
||||||
|
}
|
||||||
|
|
||||||
|
return Sequences.simple(retVals);
|
||||||
|
}
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
private Sequence<Object[]> executeSelect(
|
private Sequence<Object[]> executeSelect(
|
||||||
final DruidQueryBuilder queryBuilder,
|
final DruidQueryBuilder queryBuilder,
|
||||||
final SelectQuery baseQuery
|
final SelectQuery baseQuery
|
||||||
@ -169,7 +217,6 @@ public class QueryMaker
|
|||||||
|
|
||||||
pagingIdentifiers.set(result.getValue().getPagingIdentifiers());
|
pagingIdentifiers.set(result.getValue().getPagingIdentifiers());
|
||||||
final List<Object[]> retVals = new ArrayList<>();
|
final List<Object[]> retVals = new ArrayList<>();
|
||||||
|
|
||||||
for (EventHolder holder : result.getValue().getEvents()) {
|
for (EventHolder holder : result.getValue().getEvents()) {
|
||||||
morePages.set(true);
|
morePages.set(true);
|
||||||
final Map<String, Object> map = holder.getEvent();
|
final Map<String, Object> map = holder.getEvent();
|
||||||
|
@ -76,7 +76,7 @@ public class SelectRules
|
|||||||
final Project project = call.rel(0);
|
final Project project = call.rel(0);
|
||||||
final DruidRel druidRel = call.rel(1);
|
final DruidRel druidRel = call.rel(1);
|
||||||
|
|
||||||
// Only push in projections that can be used by the Select query.
|
// Only push in projections that can be used by the Scan or Select queries.
|
||||||
// Leave anything more complicated to DruidAggregateProjectRule for possible handling in a GroupBy query.
|
// Leave anything more complicated to DruidAggregateProjectRule for possible handling in a GroupBy query.
|
||||||
|
|
||||||
final RowSignature sourceRowSignature = druidRel.getSourceRowSignature();
|
final RowSignature sourceRowSignature = druidRel.getSourceRowSignature();
|
||||||
|
@ -75,6 +75,7 @@ import io.druid.query.groupby.orderby.OrderByColumnSpec;
|
|||||||
import io.druid.query.lookup.RegisteredLookupExtractionFn;
|
import io.druid.query.lookup.RegisteredLookupExtractionFn;
|
||||||
import io.druid.query.ordering.StringComparator;
|
import io.druid.query.ordering.StringComparator;
|
||||||
import io.druid.query.ordering.StringComparators;
|
import io.druid.query.ordering.StringComparators;
|
||||||
|
import io.druid.query.scan.ScanQuery;
|
||||||
import io.druid.query.select.PagingSpec;
|
import io.druid.query.select.PagingSpec;
|
||||||
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||||
import io.druid.query.spec.QuerySegmentSpec;
|
import io.druid.query.spec.QuerySegmentSpec;
|
||||||
@ -137,14 +138,6 @@ public class CalciteQueryTest
|
|||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
private static final PlannerConfig PLANNER_CONFIG_SELECT_PAGING = new PlannerConfig()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public int getSelectThreshold()
|
|
||||||
{
|
|
||||||
return 2;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
private static final PlannerConfig PLANNER_CONFIG_FALLBACK = new PlannerConfig()
|
private static final PlannerConfig PLANNER_CONFIG_FALLBACK = new PlannerConfig()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
@ -404,29 +397,12 @@ public class CalciteQueryTest
|
|||||||
{
|
{
|
||||||
testQuery(
|
testQuery(
|
||||||
"SELECT * FROM druid.foo",
|
"SELECT * FROM druid.foo",
|
||||||
ImmutableList.of(
|
ImmutableList.<Query>of(
|
||||||
Druids.newSelectQueryBuilder()
|
newScanQueryBuilder()
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
.dataSource(CalciteTests.DATASOURCE1)
|
||||||
.intervals(QSS(Filtration.eternity()))
|
.intervals(QSS(Filtration.eternity()))
|
||||||
.granularity(Granularities.ALL)
|
.columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1")
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
|
||||||
.pagingSpec(FIRST_PAGING_SPEC)
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
|
||||||
.build(),
|
|
||||||
Druids.newSelectQueryBuilder()
|
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
|
||||||
.intervals(QSS(Filtration.eternity()))
|
|
||||||
.granularity(Granularities.ALL)
|
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
|
||||||
.pagingSpec(
|
|
||||||
new PagingSpec(
|
|
||||||
ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 5),
|
|
||||||
1000,
|
|
||||||
true
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
.context(QUERY_CONTEXT_DEFAULT)
|
||||||
.build()
|
.build()
|
||||||
),
|
),
|
||||||
@ -469,7 +445,7 @@ public class CalciteQueryTest
|
|||||||
ImmutableList.of(),
|
ImmutableList.of(),
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new Object[]{
|
new Object[]{
|
||||||
"DruidQueryRel(query=[{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dummy\",\"outputName\":\"dummy\",\"outputType\":\"STRING\"}],\"metrics\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"virtualColumns\":[],\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":0,\"fromNext\":true},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807}}])\n"
|
"DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n"
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
@ -481,13 +457,12 @@ public class CalciteQueryTest
|
|||||||
testQuery(
|
testQuery(
|
||||||
"SELECT * FROM druid.foo LIMIT 2",
|
"SELECT * FROM druid.foo LIMIT 2",
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
Druids.newSelectQueryBuilder()
|
newScanQueryBuilder()
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
.dataSource(CalciteTests.DATASOURCE1)
|
||||||
.intervals(QSS(Filtration.eternity()))
|
.intervals(QSS(Filtration.eternity()))
|
||||||
.granularity(Granularities.ALL)
|
.columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1")
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
.limit(2)
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
.pagingSpec(FIRST_PAGING_SPEC)
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
.context(QUERY_CONTEXT_DEFAULT)
|
||||||
.build()
|
.build()
|
||||||
),
|
),
|
||||||
@ -499,7 +474,32 @@ public class CalciteQueryTest
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSelectStarWithLimitDescending() throws Exception
|
public void testSelectWithProjection() throws Exception
|
||||||
|
{
|
||||||
|
testQuery(
|
||||||
|
"SELECT SUBSTRING(dim2, 1, 1) FROM druid.foo LIMIT 2",
|
||||||
|
ImmutableList.of(
|
||||||
|
newScanQueryBuilder()
|
||||||
|
.dataSource(CalciteTests.DATASOURCE1)
|
||||||
|
.intervals(QSS(Filtration.eternity()))
|
||||||
|
.virtualColumns(
|
||||||
|
EXPRESSION_VIRTUAL_COLUMN("v0", "substring(\"dim2\", 0, 1)", ValueType.STRING)
|
||||||
|
)
|
||||||
|
.columns("v0")
|
||||||
|
.limit(2)
|
||||||
|
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
|
.context(QUERY_CONTEXT_DEFAULT)
|
||||||
|
.build()
|
||||||
|
),
|
||||||
|
ImmutableList.of(
|
||||||
|
new Object[]{"a"},
|
||||||
|
new Object[]{""}
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSelectStarWithLimitTimeDescending() throws Exception
|
||||||
{
|
{
|
||||||
testQuery(
|
testQuery(
|
||||||
"SELECT * FROM druid.foo ORDER BY __time DESC LIMIT 2",
|
"SELECT * FROM druid.foo ORDER BY __time DESC LIMIT 2",
|
||||||
@ -523,20 +523,62 @@ public class CalciteQueryTest
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSelectSingleColumnTwice() throws Exception
|
public void testSelectStarWithoutLimitTimeAscending() throws Exception
|
||||||
{
|
{
|
||||||
testQuery(
|
testQuery(
|
||||||
"SELECT dim2 x, dim2 y FROM druid.foo LIMIT 2",
|
"SELECT * FROM druid.foo ORDER BY __time",
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
Druids.newSelectQueryBuilder()
|
Druids.newSelectQueryBuilder()
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
.dataSource(CalciteTests.DATASOURCE1)
|
||||||
.intervals(QSS(Filtration.eternity()))
|
.intervals(QSS(Filtration.eternity()))
|
||||||
.granularity(Granularities.ALL)
|
.granularity(Granularities.ALL)
|
||||||
.descending(false)
|
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
.dimensions(ImmutableList.of("dummy"))
|
||||||
.metrics(ImmutableList.of("dim2"))
|
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
||||||
|
.descending(false)
|
||||||
.pagingSpec(FIRST_PAGING_SPEC)
|
.pagingSpec(FIRST_PAGING_SPEC)
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
.context(QUERY_CONTEXT_DEFAULT)
|
||||||
|
.build(),
|
||||||
|
Druids.newSelectQueryBuilder()
|
||||||
|
.dataSource(CalciteTests.DATASOURCE1)
|
||||||
|
.intervals(QSS(Filtration.eternity()))
|
||||||
|
.granularity(Granularities.ALL)
|
||||||
|
.dimensions(ImmutableList.of("dummy"))
|
||||||
|
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
||||||
|
.descending(false)
|
||||||
|
.pagingSpec(
|
||||||
|
new PagingSpec(
|
||||||
|
ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 5),
|
||||||
|
1000,
|
||||||
|
true
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.context(QUERY_CONTEXT_DEFAULT)
|
||||||
|
.build()
|
||||||
|
),
|
||||||
|
ImmutableList.of(
|
||||||
|
new Object[]{T("2000-01-01"), 1L, "", "a", 1f, 1.0, HLLCV1.class.getName()},
|
||||||
|
new Object[]{T("2000-01-02"), 1L, "10.1", "", 2f, 2.0, HLLCV1.class.getName()},
|
||||||
|
new Object[]{T("2000-01-03"), 1L, "2", "", 3f, 3.0, HLLCV1.class.getName()},
|
||||||
|
new Object[]{T("2001-01-01"), 1L, "1", "a", 4f, 4.0, HLLCV1.class.getName()},
|
||||||
|
new Object[]{T("2001-01-02"), 1L, "def", "abc", 5f, 5.0, HLLCV1.class.getName()},
|
||||||
|
new Object[]{T("2001-01-03"), 1L, "abc", "", 6f, 6.0, HLLCV1.class.getName()}
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSelectSingleColumnTwice() throws Exception
|
||||||
|
{
|
||||||
|
testQuery(
|
||||||
|
"SELECT dim2 x, dim2 y FROM druid.foo LIMIT 2",
|
||||||
|
ImmutableList.of(
|
||||||
|
newScanQueryBuilder()
|
||||||
|
.dataSource(CalciteTests.DATASOURCE1)
|
||||||
|
.intervals(QSS(Filtration.eternity()))
|
||||||
|
.columns("dim2")
|
||||||
|
.limit(2)
|
||||||
|
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
|
.context(QUERY_CONTEXT_DEFAULT)
|
||||||
.build()
|
.build()
|
||||||
),
|
),
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
@ -619,55 +661,20 @@ public class CalciteQueryTest
|
|||||||
+ " druid.foo x INNER JOIN druid.foo y ON x.dim1 = y.dim2\n"
|
+ " druid.foo x INNER JOIN druid.foo y ON x.dim1 = y.dim2\n"
|
||||||
+ "WHERE\n"
|
+ "WHERE\n"
|
||||||
+ " x.dim1 <> ''",
|
+ " x.dim1 <> ''",
|
||||||
ImmutableList.of(
|
ImmutableList.<Query>of(
|
||||||
Druids.newSelectQueryBuilder()
|
newScanQueryBuilder()
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
.dataSource(CalciteTests.DATASOURCE1)
|
||||||
.intervals(QSS(Filtration.eternity()))
|
.intervals(QSS(Filtration.eternity()))
|
||||||
.granularity(Granularities.ALL)
|
.columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1")
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
|
||||||
.pagingSpec(FIRST_PAGING_SPEC)
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
.context(QUERY_CONTEXT_DEFAULT)
|
||||||
.build(),
|
.build(),
|
||||||
Druids.newSelectQueryBuilder()
|
newScanQueryBuilder()
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
.dataSource(CalciteTests.DATASOURCE1)
|
||||||
.intervals(QSS(Filtration.eternity()))
|
.intervals(QSS(Filtration.eternity()))
|
||||||
.granularity(Granularities.ALL)
|
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
|
||||||
.pagingSpec(
|
|
||||||
new PagingSpec(
|
|
||||||
ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 5),
|
|
||||||
1000,
|
|
||||||
true
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
|
||||||
.build(),
|
|
||||||
Druids.newSelectQueryBuilder()
|
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
|
||||||
.intervals(QSS(Filtration.eternity()))
|
|
||||||
.granularity(Granularities.ALL)
|
|
||||||
.filters(NOT(SELECTOR("dim1", "", null)))
|
.filters(NOT(SELECTOR("dim1", "", null)))
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
.columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1")
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
.pagingSpec(FIRST_PAGING_SPEC)
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
|
||||||
.build(),
|
|
||||||
Druids.newSelectQueryBuilder()
|
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
|
||||||
.intervals(QSS(Filtration.eternity()))
|
|
||||||
.granularity(Granularities.ALL)
|
|
||||||
.filters(NOT(SELECTOR("dim1", "", null)))
|
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
|
||||||
.pagingSpec(
|
|
||||||
new PagingSpec(
|
|
||||||
ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 4),
|
|
||||||
1000,
|
|
||||||
true
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
.context(QUERY_CONTEXT_DEFAULT)
|
||||||
.build()
|
.build()
|
||||||
),
|
),
|
||||||
@ -693,8 +700,8 @@ public class CalciteQueryTest
|
|||||||
new Object[]{
|
new Object[]{
|
||||||
"BindableProject(dim1=[$9], dim10=[$2], dim2=[$3])\n"
|
"BindableProject(dim1=[$9], dim10=[$2], dim2=[$3])\n"
|
||||||
+ " BindableJoin(condition=[=($9, $3)], joinType=[inner])\n"
|
+ " BindableJoin(condition=[=($9, $3)], joinType=[inner])\n"
|
||||||
+ " DruidQueryRel(query=[{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dummy\",\"outputName\":\"dummy\",\"outputType\":\"STRING\"}],\"metrics\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"virtualColumns\":[],\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":0,\"fromNext\":true},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807}}])\n"
|
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n"
|
||||||
+ " DruidQueryRel(query=[{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dummy\",\"outputName\":\"dummy\",\"outputType\":\"STRING\"}],\"metrics\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"virtualColumns\":[],\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":0,\"fromNext\":true},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807}}])\n"
|
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n"
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
@ -1315,114 +1322,18 @@ public class CalciteQueryTest
|
|||||||
{
|
{
|
||||||
testQuery(
|
testQuery(
|
||||||
"SELECT * FROM druid.foo WHERE dim1 > 'd' OR dim2 = 'a'",
|
"SELECT * FROM druid.foo WHERE dim1 > 'd' OR dim2 = 'a'",
|
||||||
ImmutableList.of(
|
ImmutableList.<Query>of(
|
||||||
Druids.newSelectQueryBuilder()
|
newScanQueryBuilder()
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
.dataSource(CalciteTests.DATASOURCE1)
|
||||||
.intervals(QSS(Filtration.eternity()))
|
.intervals(QSS(Filtration.eternity()))
|
||||||
.granularity(Granularities.ALL)
|
|
||||||
.pagingSpec(FIRST_PAGING_SPEC)
|
|
||||||
.filters(
|
.filters(
|
||||||
OR(
|
OR(
|
||||||
BOUND("dim1", "d", null, true, false, null, StringComparators.LEXICOGRAPHIC),
|
BOUND("dim1", "d", null, true, false, null, StringComparators.LEXICOGRAPHIC),
|
||||||
SELECTOR("dim2", "a", null)
|
SELECTOR("dim2", "a", null)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
.columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1")
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
|
||||||
.build(),
|
|
||||||
Druids.newSelectQueryBuilder()
|
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
|
||||||
.intervals(QSS(Filtration.eternity()))
|
|
||||||
.granularity(Granularities.ALL)
|
|
||||||
.pagingSpec(
|
|
||||||
new PagingSpec(
|
|
||||||
ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 2),
|
|
||||||
1000,
|
|
||||||
true
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.filters(
|
|
||||||
OR(
|
|
||||||
BOUND("dim1", "d", null, true, false, null, StringComparators.LEXICOGRAPHIC),
|
|
||||||
SELECTOR("dim2", "a", null)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
|
||||||
.build()
|
|
||||||
),
|
|
||||||
ImmutableList.of(
|
|
||||||
new Object[]{T("2000-01-01"), 1L, "", "a", 1.0f, 1d, HLLCV1.class.getName()},
|
|
||||||
new Object[]{T("2001-01-01"), 1L, "1", "a", 4.0f, 4d, HLLCV1.class.getName()},
|
|
||||||
new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0f, 5d, HLLCV1.class.getName()}
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testSelectStarWithDimFilterAndPaging() throws Exception
|
|
||||||
{
|
|
||||||
testQuery(
|
|
||||||
PLANNER_CONFIG_SELECT_PAGING,
|
|
||||||
"SELECT * FROM druid.foo WHERE dim1 > 'd' OR dim2 = 'a'",
|
|
||||||
ImmutableList.of(
|
|
||||||
Druids.newSelectQueryBuilder()
|
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
|
||||||
.intervals(QSS(Filtration.eternity()))
|
|
||||||
.granularity(Granularities.ALL)
|
|
||||||
.pagingSpec(new PagingSpec(null, 2, true))
|
|
||||||
.filters(
|
|
||||||
OR(
|
|
||||||
BOUND("dim1", "d", null, true, false, null, StringComparators.LEXICOGRAPHIC),
|
|
||||||
SELECTOR("dim2", "a", null)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
|
||||||
.build(),
|
|
||||||
Druids.newSelectQueryBuilder()
|
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
|
||||||
.intervals(QSS(Filtration.eternity()))
|
|
||||||
.granularity(Granularities.ALL)
|
|
||||||
.pagingSpec(
|
|
||||||
new PagingSpec(
|
|
||||||
ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 1),
|
|
||||||
2,
|
|
||||||
true
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.filters(
|
|
||||||
OR(
|
|
||||||
BOUND("dim1", "d", null, true, false, null, StringComparators.LEXICOGRAPHIC),
|
|
||||||
SELECTOR("dim2", "a", null)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
|
||||||
.build(),
|
|
||||||
Druids.newSelectQueryBuilder()
|
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
|
||||||
.intervals(QSS(Filtration.eternity()))
|
|
||||||
.granularity(Granularities.ALL)
|
|
||||||
.pagingSpec(
|
|
||||||
new PagingSpec(
|
|
||||||
ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 2),
|
|
||||||
2,
|
|
||||||
true
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.filters(
|
|
||||||
OR(
|
|
||||||
BOUND("dim1", "d", null, true, false, null, StringComparators.LEXICOGRAPHIC),
|
|
||||||
SELECTOR("dim2", "a", null)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
.context(QUERY_CONTEXT_DEFAULT)
|
||||||
.build()
|
.build()
|
||||||
),
|
),
|
||||||
@ -5458,40 +5369,12 @@ public class CalciteQueryTest
|
|||||||
.setHavingSpec(new DimFilterHavingSpec(NUMERIC_SELECTOR("a0", "1", null)))
|
.setHavingSpec(new DimFilterHavingSpec(NUMERIC_SELECTOR("a0", "1", null)))
|
||||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||||
.build(),
|
.build(),
|
||||||
Druids.newSelectQueryBuilder()
|
newScanQueryBuilder()
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
.dataSource(CalciteTests.DATASOURCE1)
|
||||||
.dimensionSpecs(DIMS(
|
|
||||||
new DefaultDimensionSpec("dim1", "d1"),
|
|
||||||
new DefaultDimensionSpec("dim2", "d2")
|
|
||||||
))
|
|
||||||
.metrics(ImmutableList.of("cnt"))
|
|
||||||
.intervals(QSS(Filtration.eternity()))
|
.intervals(QSS(Filtration.eternity()))
|
||||||
.granularity(Granularities.ALL)
|
|
||||||
.filters(AND(SELECTOR("dim1", "def", null), SELECTOR("dim2", "abc", null)))
|
.filters(AND(SELECTOR("dim1", "def", null), SELECTOR("dim2", "abc", null)))
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
.columns("__time", "cnt", "dim1", "dim2")
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2"))
|
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
.pagingSpec(FIRST_PAGING_SPEC)
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
|
||||||
.build(),
|
|
||||||
Druids.newSelectQueryBuilder()
|
|
||||||
.dataSource(CalciteTests.DATASOURCE1)
|
|
||||||
.dimensionSpecs(DIMS(
|
|
||||||
new DefaultDimensionSpec("dim1", "d1"),
|
|
||||||
new DefaultDimensionSpec("dim2", "d2")
|
|
||||||
))
|
|
||||||
.metrics(ImmutableList.of("cnt"))
|
|
||||||
.intervals(QSS(Filtration.eternity()))
|
|
||||||
.granularity(Granularities.ALL)
|
|
||||||
.filters(AND(SELECTOR("dim1", "def", null), SELECTOR("dim2", "abc", null)))
|
|
||||||
.dimensions(ImmutableList.of("dummy"))
|
|
||||||
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2"))
|
|
||||||
.pagingSpec(
|
|
||||||
new PagingSpec(
|
|
||||||
ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 0),
|
|
||||||
1000,
|
|
||||||
true
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.context(QUERY_CONTEXT_DEFAULT)
|
.context(QUERY_CONTEXT_DEFAULT)
|
||||||
.build()
|
.build()
|
||||||
),
|
),
|
||||||
@ -5813,4 +5696,9 @@ public class CalciteQueryTest
|
|||||||
{
|
{
|
||||||
return new ExpressionPostAggregator(name, expression, null, CalciteTests.createExprMacroTable());
|
return new ExpressionPostAggregator(name, expression, null, CalciteTests.createExprMacroTable());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static ScanQuery.ScanQueryBuilder newScanQueryBuilder()
|
||||||
|
{
|
||||||
|
return new ScanQuery.ScanQueryBuilder().legacy(false);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -67,6 +67,11 @@ import io.druid.query.metadata.SegmentMetadataQueryConfig;
|
|||||||
import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest;
|
import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest;
|
||||||
import io.druid.query.metadata.SegmentMetadataQueryRunnerFactory;
|
import io.druid.query.metadata.SegmentMetadataQueryRunnerFactory;
|
||||||
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||||
|
import io.druid.query.scan.ScanQuery;
|
||||||
|
import io.druid.query.scan.ScanQueryConfig;
|
||||||
|
import io.druid.query.scan.ScanQueryEngine;
|
||||||
|
import io.druid.query.scan.ScanQueryQueryToolChest;
|
||||||
|
import io.druid.query.scan.ScanQueryRunnerFactory;
|
||||||
import io.druid.query.select.SelectQuery;
|
import io.druid.query.select.SelectQuery;
|
||||||
import io.druid.query.select.SelectQueryConfig;
|
import io.druid.query.select.SelectQueryConfig;
|
||||||
import io.druid.query.select.SelectQueryEngine;
|
import io.druid.query.select.SelectQueryEngine;
|
||||||
@ -156,6 +161,16 @@ public class CalciteTests
|
|||||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
.put(
|
||||||
|
ScanQuery.class,
|
||||||
|
new ScanQueryRunnerFactory(
|
||||||
|
new ScanQueryQueryToolChest(
|
||||||
|
new ScanQueryConfig(),
|
||||||
|
new DefaultGenericQueryMetricsFactory(TestHelper.getJsonMapper())
|
||||||
|
),
|
||||||
|
new ScanQueryEngine()
|
||||||
|
)
|
||||||
|
)
|
||||||
.put(
|
.put(
|
||||||
SelectQuery.class,
|
SelectQuery.class,
|
||||||
new SelectQueryRunnerFactory(
|
new SelectQueryRunnerFactory(
|
||||||
|
Loading…
x
Reference in New Issue
Block a user