mirror of https://github.com/apache/druid.git
SQL: Ditch CalciteConnection layer and add DruidMeta, extension aggregators. (#3852)
* SQL: Ditch CalciteConnection layer and add DruidMeta, extension aggregators. Switched from CalciteConnection to Planner, bringing benefits: - CalciteConnection's JDBC interface no longer sits between the SQL server (HTTP/Avatica) and Druid's query layer. Instead, the SQL servers can use Druid Sequence objects directly, reducing overhead in the query return path. - Implemented our own Planner-based Avatica Meta, letting us control connection timeouts and connection / statement limits. The previous CalciteConnection-based implementation didn't have any limits or timeouts. - The Planner interface lets us override the operator table, opening up SQL language extensions. This patch includes two: APPROX_COUNT_DISTINCT in core, and a QUANTILE aggregator in the druid-histogram extension. Also: - Added INFORMATION_SCHEMA metadata schema. - Added tests for Unicode literals and escapes. * Verify statement is actually open before closing it. * More detailed INFORMATION_SCHEMA docs.
This commit is contained in:
parent
33ae9dd485
commit
d51f5e058d
|
@ -34,6 +34,7 @@ import io.druid.granularity.QueryGranularities;
|
|||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
|
@ -41,18 +42,25 @@ import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
|||
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.query.groupby.GroupByQuery;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.TestHelper;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.segment.serde.ComplexMetrics;
|
||||
import io.druid.sql.calcite.planner.Calcites;
|
||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||
import io.druid.sql.calcite.planner.PlannerFactory;
|
||||
import io.druid.sql.calcite.planner.PlannerResult;
|
||||
import io.druid.sql.calcite.rel.QueryMaker;
|
||||
import io.druid.sql.calcite.table.DruidTable;
|
||||
import io.druid.sql.calcite.util.CalciteTests;
|
||||
import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
import org.apache.calcite.jdbc.CalciteConnection;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.partition.LinearShardSpec;
|
||||
import org.apache.calcite.schema.Schema;
|
||||
import org.apache.calcite.schema.Table;
|
||||
import org.apache.calcite.schema.impl.AbstractSchema;
|
||||
import org.apache.calcite.tools.Planner;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.joda.time.Interval;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
|
@ -71,8 +79,6 @@ import org.openjdk.jmh.annotations.Warmup;
|
|||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import java.io.File;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.ResultSetMetaData;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
@ -96,7 +102,7 @@ public class SqlBenchmark
|
|||
|
||||
private File tmpDir;
|
||||
private SpecificSegmentsQuerySegmentWalker walker;
|
||||
private CalciteConnection calciteConnection;
|
||||
private PlannerFactory plannerFactory;
|
||||
private GroupByQuery groupByQuery;
|
||||
private String sqlQuery;
|
||||
|
||||
|
@ -130,7 +136,23 @@ public class SqlBenchmark
|
|||
log.info("%,d/%,d rows generated.", rows.size(), rowsPerSegment);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
walker = CalciteTests.createWalker(tmpDir, rows);
|
||||
final QueryRunnerFactoryConglomerate conglomerate = CalciteTests.queryRunnerFactoryConglomerate();
|
||||
final QueryableIndex index = IndexBuilder.create()
|
||||
.tmpDir(new File(tmpDir, "1"))
|
||||
.indexMerger(TestHelper.getTestIndexMergerV9())
|
||||
.rows(rows)
|
||||
.buildMMappedIndex();
|
||||
|
||||
this.walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
|
||||
DataSegment.builder()
|
||||
.dataSource("foo")
|
||||
.interval(index.getDataInterval())
|
||||
.version("1")
|
||||
.shardSpec(new LinearShardSpec(0))
|
||||
.build(),
|
||||
index
|
||||
);
|
||||
|
||||
final Map<String, Table> tableMap = ImmutableMap.<String, Table>of(
|
||||
"foo",
|
||||
new DruidTable(
|
||||
|
@ -152,7 +174,11 @@ public class SqlBenchmark
|
|||
return tableMap;
|
||||
}
|
||||
};
|
||||
calciteConnection = Calcites.jdbc(druidSchema, plannerConfig);
|
||||
plannerFactory = new PlannerFactory(
|
||||
Calcites.createRootSchema(druidSchema),
|
||||
CalciteTests.createOperatorTable(),
|
||||
plannerConfig
|
||||
);
|
||||
groupByQuery = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource("foo")
|
||||
|
@ -160,7 +186,7 @@ public class SqlBenchmark
|
|||
.setDimensions(
|
||||
Arrays.<DimensionSpec>asList(
|
||||
new DefaultDimensionSpec("dimZipf", "d0"),
|
||||
new DefaultDimensionSpec("dimSequential", "d1")
|
||||
new DefaultDimensionSpec("dimSequential", "d1")
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(Arrays.<AggregatorFactory>asList(new CountAggregatorFactory("c")))
|
||||
|
@ -204,15 +230,12 @@ public class SqlBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MILLISECONDS)
|
||||
public void querySql(Blackhole blackhole) throws Exception
|
||||
public void queryPlanner(Blackhole blackhole) throws Exception
|
||||
{
|
||||
final ResultSet resultSet = calciteConnection.createStatement().executeQuery(sqlQuery);
|
||||
final ResultSetMetaData metaData = resultSet.getMetaData();
|
||||
|
||||
while (resultSet.next()) {
|
||||
for (int i = 0; i < metaData.getColumnCount(); i++) {
|
||||
blackhole.consume(resultSet.getObject(i + 1));
|
||||
}
|
||||
try (final Planner planner = plannerFactory.createPlanner()) {
|
||||
final PlannerResult plannerResult = Calcites.plan(planner, sqlQuery);
|
||||
final ArrayList<Object[]> results = Sequences.toList(plannerResult.run(), Lists.<Object[]>newArrayList());
|
||||
blackhole.consume(results);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -92,8 +92,11 @@ The broker's [built-in SQL server](../querying/sql.html) can be configured throu
|
|||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.sql.enable`|Whether to enable SQL at all, including background metadata fetching. If false, this overrides all other SQL-related properties and disables SQL metadata, serving, and planning completely.|false|
|
||||
|`druid.sql.server.enableAvatica`|Whether to enable an Avatica server at `/druid/v2/sql/avatica/`.|true|
|
||||
|`druid.sql.server.enableJsonOverHttp`|Whether to enable a simple JSON over HTTP route at `/druid/v2/sql/`.|true|
|
||||
|`druid.sql.avatica.enable`|Whether to enable an Avatica server at `/druid/v2/sql/avatica/`.|true|
|
||||
|`druid.sql.avatica.connectionIdleTimeout`|Avatica client connection idle timeout.|PT30M|
|
||||
|`druid.sql.avatica.maxConnections`|Maximum number of open connections for the Avatica server. These are not HTTP connections, but are logical client connections that may span multiple HTTP connections.|25|
|
||||
|`druid.sql.avatica.maxStatementsPerConnection`|Maximum number of simultaneous open statements per Avatica client connection.|4|
|
||||
|`druid.sql.http.enable`|Whether to enable a simple JSON over HTTP route at `/druid/v2/sql/`.|true|
|
||||
|
||||
#### SQL Planner Configuration
|
||||
|
||||
|
|
|
@ -12,7 +12,10 @@ subject to change.
|
|||
|
||||
Druid includes a native SQL layer with an [Apache Calcite](https://calcite.apache.org/)-based parser and planner. All
|
||||
parsing and planning takes place on the Broker, where SQL is converted to native Druid queries. Those native Druid
|
||||
queries are then passed down to data nodes. Each Druid dataSource appears as a table in the "druid" schema.
|
||||
queries are then passed down to data nodes. Each Druid datasource appears as a table in the "druid" schema. Datasource
|
||||
and column names are both case-sensitive and can optionally be quoted using double quotes. Literal strings should be
|
||||
quoted with single quotes, like `'foo'`. Literal strings with Unicode escapes can be written like `U&'fo\00F6'`, where
|
||||
character codes in hex are prefixed by a backslash.
|
||||
|
||||
Add "EXPLAIN PLAN FOR" to the beginning of any query to see how Druid will plan that query.
|
||||
|
||||
|
@ -29,7 +32,7 @@ Example code:
|
|||
|
||||
```java
|
||||
Connection connection = DriverManager.getConnection("jdbc:avatica:remote:url=http://localhost:8082/druid/v2/sql/avatica/");
|
||||
ResultSet resultSet = connection.createStatement().executeQuery("SELECT COUNT(*) AS cnt FROM druid.foo");
|
||||
ResultSet resultSet = connection.createStatement().executeQuery("SELECT COUNT(*) AS cnt FROM data_source");
|
||||
while (resultSet.next()) {
|
||||
// Do something
|
||||
}
|
||||
|
@ -46,17 +49,17 @@ is:
|
|||
|
||||
```json
|
||||
{
|
||||
"query" : "SELECT COUNT(*) FROM druid.ds WHERE foo = ?"
|
||||
"query" : "SELECT COUNT(*) FROM data_source WHERE foo = 'bar'"
|
||||
}
|
||||
```
|
||||
|
||||
You can use _curl_ to send these queries from the command-line:
|
||||
|
||||
```bash
|
||||
curl -XPOST -H'Content-Type: application/json' http://BROKER:8082/druid/v2/sql/ -d '{"query":"SELECT COUNT(*) FROM druid.ds"}'
|
||||
curl -XPOST -H'Content-Type: application/json' http://BROKER:8082/druid/v2/sql/ -d '{"query":"SELECT COUNT(*) FROM data_source"}'
|
||||
```
|
||||
|
||||
Metadata is not available over the HTTP API.
|
||||
Metadata is only available over the HTTP API by querying the "INFORMATION_SCHEMA" tables (see below).
|
||||
|
||||
### Metadata
|
||||
|
||||
|
@ -65,28 +68,28 @@ on broker startup and also periodically in the background through
|
|||
[SegmentMetadata queries](../querying/segmentmetadataquery.html). Background metadata refreshing is triggered by
|
||||
segments entering and exiting the cluster, and can also be throttled through configuration.
|
||||
|
||||
This cached metadata is queryable through the "metadata.COLUMNS" and "metadata.TABLES" tables. When
|
||||
`druid.sql.planner.useFallback` is disabled (the default), only full scans of this table are possible. For example, to
|
||||
retrieve column metadata, use the query:
|
||||
This cached metadata is queryable through "INFORMATION_SCHEMA" tables. For example, to retrieve metadata for the Druid
|
||||
datasource "foo", use the query:
|
||||
|
||||
```sql
|
||||
SELECT * FROM metadata.COLUMNS
|
||||
SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE SCHEMA_NAME = 'druid' AND TABLE_NAME = 'foo'
|
||||
```
|
||||
|
||||
If `druid.sql.planner.useFallback` is enabled, full SQL is possible on metadata tables. However, useFallback is not
|
||||
recommended in production since it can generate unscalable query plans. The JDBC driver allows accessing
|
||||
table and column metadata through `connection.getMetaData()` even if useFallback is off.
|
||||
See the [INFORMATION_SCHEMA tables](#information_schema-tables) section below for details on the available metadata.
|
||||
|
||||
You can also access table and column metadata through JDBC using `connection.getMetaData()`.
|
||||
|
||||
### Approximate queries
|
||||
|
||||
The following SQL queries and features may be executed using approximate algorithms:
|
||||
|
||||
- `COUNT(DISTINCT col)` aggregations use [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf), a
|
||||
fast approximate distinct counting algorithm. If you need exact distinct counts, you can instead use
|
||||
`SELECT COUNT(*) FROM (SELECT DISTINCT col FROM druid.foo)`, which will use a slower and more resource intensive exact
|
||||
- `COUNT(DISTINCT col)` and `APPROX_COUNT_DISTINCT(col)` aggregations use
|
||||
[HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf), a fast approximate distinct counting
|
||||
algorithm. If you need exact distinct counts, you can instead use
|
||||
`SELECT COUNT(*) FROM (SELECT DISTINCT col FROM data_source)`, which will use a slower and more resource intensive exact
|
||||
algorithm.
|
||||
- TopN-style queries with a single grouping column, like
|
||||
`SELECT col1, SUM(col2) FROM druid.foo GROUP BY col1 ORDER BY SUM(col2) DESC LIMIT 100`, by default will be executed
|
||||
`SELECT col1, SUM(col2) FROM data_source GROUP BY col1 ORDER BY SUM(col2) DESC LIMIT 100`, by default will be executed
|
||||
as [TopN queries](topnquery.html), which use an approximate algorithm. To disable this behavior, and use exact
|
||||
algorithms for topN-style queries, set
|
||||
[druid.sql.planner.useApproximateTopN](../configuration/broker.html#sql-planner-configuration) to "false".
|
||||
|
@ -95,9 +98,9 @@ algorithms for topN-style queries, set
|
|||
|
||||
Druid's SQL language supports a number of time operations, including:
|
||||
|
||||
- `FLOOR(__time TO <granularity>)` for grouping or filtering on time buckets, like `SELECT FLOOR(__time TO MONTH), SUM(cnt) FROM druid.foo GROUP BY FLOOR(__time TO MONTH)`
|
||||
- `EXTRACT(<granularity> FROM __time)` for grouping or filtering on time parts, like `SELECT EXTRACT(HOUR FROM __time), SUM(cnt) FROM druid.foo GROUP BY EXTRACT(HOUR FROM __time)`
|
||||
- Comparisons to `TIMESTAMP '<time string>'` for time filters, like `SELECT COUNT(*) FROM druid.foo WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2001-01-01 00:00:00'`
|
||||
- `FLOOR(__time TO <granularity>)` for grouping or filtering on time buckets, like `SELECT FLOOR(__time TO MONTH), SUM(cnt) FROM data_source GROUP BY FLOOR(__time TO MONTH)`
|
||||
- `EXTRACT(<granularity> FROM __time)` for grouping or filtering on time parts, like `SELECT EXTRACT(HOUR FROM __time), SUM(cnt) FROM data_source GROUP BY EXTRACT(HOUR FROM __time)`
|
||||
- Comparisons to `TIMESTAMP '<time string>'` for time filters, like `SELECT COUNT(*) FROM data_source WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2001-01-01 00:00:00'`
|
||||
|
||||
### Subqueries
|
||||
|
||||
|
@ -110,7 +113,7 @@ Subqueries involving `FROM (SELECT ... GROUP BY ...)` may be executed as
|
|||
exact distinct count using a nested groupBy.
|
||||
|
||||
```sql
|
||||
SELECT COUNT(*) FROM (SELECT DISTINCT col FROM druid.foo)
|
||||
SELECT COUNT(*) FROM (SELECT DISTINCT col FROM data_source)
|
||||
```
|
||||
|
||||
Note that groupBys require a separate merge buffer on the broker for each layer beyond the first layer of the groupBy.
|
||||
|
@ -126,29 +129,38 @@ Semi-join subqueries involving `WHERE ... IN (SELECT ...)`, like the following,
|
|||
|
||||
```sql
|
||||
SELECT x, COUNT(*)
|
||||
FROM druid.foo
|
||||
WHERE x IN (SELECT x FROM druid.bar WHERE y = 'baz')
|
||||
FROM data_source_1
|
||||
WHERE x IN (SELECT x FROM data_source_2 WHERE y = 'baz')
|
||||
GROUP BY x
|
||||
```
|
||||
|
||||
For this query, the broker will first translate the inner select on dataSource `bar` into a groupBy to find distinct
|
||||
`x` values. Then it'll use those distinct values to build an "in" filter on dataSource `foo` for the outer query. The
|
||||
For this query, the broker will first translate the inner select on data_source_2 into a groupBy to find distinct
|
||||
`x` values. Then it'll use those distinct values to build an "in" filter on data_source_1 for the outer query. The
|
||||
configuration parameter `druid.sql.planner.maxSemiJoinRowsInMemory` controls the maximum number of values that will be
|
||||
materialized for this kind of plan.
|
||||
|
||||
### Configuration
|
||||
|
||||
Druid's SQL planner can be configured on the [Broker node](../configuration/broker.html#sql-planner-configuration).
|
||||
Druid's SQL layer can be configured on the [Broker node](../configuration/broker.html#sql-planner-configuration).
|
||||
|
||||
### Extensions
|
||||
|
||||
Some Druid extensions also include SQL language extensions.
|
||||
|
||||
If the [approximate histogram extension](../development/extensions-core/approximate-histograms.html) is loaded:
|
||||
|
||||
- `QUANTILE(column, probability)` on numeric or approximate histogram columns computes approximate quantiles. The
|
||||
"probability" should be between 0 and 1 (exclusive).
|
||||
|
||||
### Unsupported features
|
||||
|
||||
Druid does not support all SQL features. Most of these are due to missing features in Druid's native JSON-based query
|
||||
language. Some unsupported SQL features include:
|
||||
|
||||
- Grouping on functions of multiple columns, like concatenation: `SELECT COUNT(*) FROM druid.foo GROUP BY dim1 || ' ' || dim2`
|
||||
- Grouping on functions of multiple columns, like concatenation: `SELECT COUNT(*) FROM data_source GROUP BY dim1 || ' ' || dim2`
|
||||
- Grouping on long and float columns.
|
||||
- Filtering on float columns.
|
||||
- Filtering on non-boolean interactions between columns, like two columns equaling each other: `SELECT COUNT(*) FROM druid.foo WHERE dim1 = dim2`.
|
||||
- Filtering on non-boolean interactions between columns, like two columns equaling each other: `SELECT COUNT(*) FROM data_source WHERE dim1 = dim2`.
|
||||
- A number of miscellaneous functions, like `TRIM`.
|
||||
- Joins, other than semi-joins as described above.
|
||||
|
||||
|
@ -156,10 +168,56 @@ Additionally, some Druid features are not supported by the SQL language. Some un
|
|||
|
||||
- [Multi-value dimensions](multi-value-dimensions.html).
|
||||
- [Query-time lookups](lookups.html).
|
||||
- Extensions, including [approximate histograms](../development/extensions-core/approximate-histograms.html) and
|
||||
[DataSketches](../development/extensions-core/datasketches-aggregators.html).
|
||||
- [DataSketches](../development/extensions-core/datasketches-aggregators.html).
|
||||
|
||||
## Third-party SQL libraries
|
||||
|
||||
A number of third parties have also released SQL libraries for Druid. Links to popular options can be found on
|
||||
our [libraries](/libraries.html) page. These libraries make native Druid JSON queries and do not use Druid's SQL layer.
|
||||
|
||||
## INFORMATION_SCHEMA tables
|
||||
|
||||
Druid metadata is queryable through "INFORMATION_SCHEMA" tables described below.
|
||||
|
||||
### SCHEMATA table
|
||||
|
||||
|Column|Notes|
|
||||
|------|-----|
|
||||
|CATALOG_NAME|Unused|
|
||||
|SCHEMA_NAME||
|
||||
|SCHEMA_OWNER|Unused|
|
||||
|DEFAULT_CHARACTER_SET_CATALOG|Unused|
|
||||
|DEFAULT_CHARACTER_SET_SCHEMA|Unused|
|
||||
|DEFAULT_CHARACTER_SET_NAME|Unused|
|
||||
|SQL_PATH|Unused|
|
||||
|
||||
### TABLES table
|
||||
|
||||
|Column|Notes|
|
||||
|------|-----|
|
||||
|TABLE_CATALOG|Unused|
|
||||
|TABLE_SCHEMA||
|
||||
|TABLE_NAME||
|
||||
|TABLE_TYPE|"TABLE" or "SYSTEM_TABLE"|
|
||||
|
||||
### COLUMNS table
|
||||
|
||||
|Column|Notes|
|
||||
|------|-----|
|
||||
|TABLE_CATALOG|Unused|
|
||||
|TABLE_SCHEMA||
|
||||
|TABLE_NAME||
|
||||
|COLUMN_NAME||
|
||||
|ORDINAL_POSITION||
|
||||
|COLUMN_DEFAULT|Unused|
|
||||
|IS_NULLABLE||
|
||||
|DATA_TYPE||
|
||||
|CHARACTER_MAXIMUM_LENGTH|Unused|
|
||||
|CHARACTER_OCTET_LENGTH|Unused|
|
||||
|NUMERIC_PRECISION||
|
||||
|NUMERIC_PRECISION_RADIX||
|
||||
|NUMERIC_SCALE||
|
||||
|DATETIME_PRECISION||
|
||||
|CHARACTER_SET_NAME||
|
||||
|COLLATION_NAME||
|
||||
|JDBC_TYPE|Type code from java.sql.Types (Druid extension)|
|
||||
|
|
|
@ -37,6 +37,12 @@
|
|||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-sql</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- Tests -->
|
||||
<dependency>
|
||||
|
@ -46,6 +52,13 @@
|
|||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-sql</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
|
|
|
@ -24,7 +24,9 @@ import com.fasterxml.jackson.databind.module.SimpleModule;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import io.druid.initialization.DruidModule;
|
||||
import io.druid.query.aggregation.histogram.sql.QuantileSqlAggregator;
|
||||
import io.druid.segment.serde.ComplexMetrics;
|
||||
import io.druid.sql.guice.SqlBindings;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
|
@ -56,5 +58,10 @@ public class ApproximateHistogramDruidModule implements DruidModule
|
|||
if (ComplexMetrics.getSerdeForType("approximateHistogram") == null) {
|
||||
ComplexMetrics.registerSerde("approximateHistogram", new ApproximateHistogramFoldingSerde());
|
||||
}
|
||||
|
||||
if (binder != null) {
|
||||
// Binder is null in some tests.
|
||||
SqlBindings.addAggregator(binder, QuantileSqlAggregator.class);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -86,6 +86,32 @@ public class QuantilePostAggregator extends ApproximateHistogramPostAggregator
|
|||
return probability;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(final Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final QuantilePostAggregator that = (QuantilePostAggregator) o;
|
||||
|
||||
if (Float.compare(that.probability, probability) != 0) {
|
||||
return false;
|
||||
}
|
||||
return fieldName != null ? fieldName.equals(that.fieldName) : that.fieldName == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = (probability != +0.0f ? Float.floatToIntBits(probability) : 0);
|
||||
result = 31 * result + (fieldName != null ? fieldName.hashCode() : 0);
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -0,0 +1,163 @@
|
|||
/*
|
||||
* 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.aggregation.histogram.sql;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.histogram.ApproximateHistogram;
|
||||
import io.druid.query.aggregation.histogram.ApproximateHistogramAggregatorFactory;
|
||||
import io.druid.query.aggregation.histogram.ApproximateHistogramFoldingAggregatorFactory;
|
||||
import io.druid.query.aggregation.histogram.QuantilePostAggregator;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.sql.calcite.aggregation.Aggregation;
|
||||
import io.druid.sql.calcite.aggregation.SqlAggregator;
|
||||
import io.druid.sql.calcite.expression.Expressions;
|
||||
import io.druid.sql.calcite.expression.RowExtraction;
|
||||
import io.druid.sql.calcite.table.RowSignature;
|
||||
import org.apache.calcite.rel.core.AggregateCall;
|
||||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.sql.SqlAggFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.ReturnTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class QuantileSqlAggregator implements SqlAggregator
|
||||
{
|
||||
private static final SqlAggFunction FUNCTION_INSTANCE = new QuantileSqlAggFunction();
|
||||
private static final String NAME = "QUANTILE";
|
||||
|
||||
@Override
|
||||
public SqlAggFunction calciteFunction()
|
||||
{
|
||||
return FUNCTION_INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Aggregation toDruidAggregation(
|
||||
final String name,
|
||||
final RowSignature rowSignature,
|
||||
final List<Aggregation> existingAggregations,
|
||||
final Project project,
|
||||
final AggregateCall aggregateCall
|
||||
)
|
||||
{
|
||||
final RowExtraction rex = Expressions.toRowExtraction(
|
||||
rowSignature.getRowOrder(),
|
||||
Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(0)
|
||||
)
|
||||
);
|
||||
if (rex == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final RexNode probabilityArg = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(1)
|
||||
);
|
||||
final float probability = ((Number) RexLiteral.value(probabilityArg)).floatValue();
|
||||
|
||||
final AggregatorFactory aggregatorFactory;
|
||||
final String histogramName = String.format("%s:agg", name);
|
||||
|
||||
final int resolution = ApproximateHistogram.DEFAULT_HISTOGRAM_SIZE;
|
||||
final int numBuckets = ApproximateHistogram.DEFAULT_BUCKET_SIZE;
|
||||
final float lowerLimit = Float.NEGATIVE_INFINITY;
|
||||
final float upperLimit = Float.POSITIVE_INFINITY;
|
||||
|
||||
// Look for existing matching aggregatorFactory.
|
||||
for (final Aggregation existing : existingAggregations) {
|
||||
for (AggregatorFactory factory : existing.getAggregatorFactories()) {
|
||||
if (factory instanceof ApproximateHistogramAggregatorFactory) {
|
||||
final ApproximateHistogramAggregatorFactory theFactory = (ApproximateHistogramAggregatorFactory) factory;
|
||||
if (theFactory.getFieldName().equals(rex.getColumn())
|
||||
&& theFactory.getResolution() == resolution
|
||||
&& theFactory.getNumBuckets() == numBuckets
|
||||
&& theFactory.getLowerLimit() == lowerLimit
|
||||
&& theFactory.getUpperLimit() == upperLimit) {
|
||||
// Found existing one. Use this.
|
||||
return Aggregation.create(
|
||||
ImmutableList.<AggregatorFactory>of(),
|
||||
new QuantilePostAggregator(name, theFactory.getName(), probability)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (rowSignature.getColumnType(rex.getColumn()) == ValueType.COMPLEX) {
|
||||
aggregatorFactory = new ApproximateHistogramFoldingAggregatorFactory(
|
||||
histogramName,
|
||||
rex.getColumn(),
|
||||
resolution,
|
||||
numBuckets,
|
||||
lowerLimit,
|
||||
upperLimit
|
||||
);
|
||||
} else {
|
||||
aggregatorFactory = new ApproximateHistogramAggregatorFactory(
|
||||
histogramName,
|
||||
rex.getColumn(),
|
||||
resolution,
|
||||
numBuckets,
|
||||
lowerLimit,
|
||||
upperLimit
|
||||
);
|
||||
}
|
||||
|
||||
return Aggregation.create(
|
||||
ImmutableList.of(aggregatorFactory),
|
||||
new QuantilePostAggregator(name, histogramName, probability)
|
||||
);
|
||||
}
|
||||
|
||||
private static class QuantileSqlAggFunction extends SqlAggFunction
|
||||
{
|
||||
private static final String SIGNATURE = "'" + NAME + "(column, probability)'";
|
||||
|
||||
QuantileSqlAggFunction()
|
||||
{
|
||||
super(
|
||||
NAME,
|
||||
null,
|
||||
SqlKind.OTHER_FUNCTION,
|
||||
ReturnTypes.explicit(SqlTypeName.DOUBLE),
|
||||
null,
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(SIGNATURE, OperandTypes.ANY, OperandTypes.LITERAL),
|
||||
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC)
|
||||
),
|
||||
SqlFunctionCategory.NUMERIC,
|
||||
false,
|
||||
false
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,134 @@
|
|||
/*
|
||||
* 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.aggregation.histogram.sql;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import io.druid.granularity.QueryGranularities;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.aggregation.histogram.ApproximateHistogramAggregatorFactory;
|
||||
import io.druid.query.aggregation.histogram.QuantilePostAggregator;
|
||||
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import io.druid.sql.calcite.aggregation.SqlAggregator;
|
||||
import io.druid.sql.calcite.filtration.Filtration;
|
||||
import io.druid.sql.calcite.planner.Calcites;
|
||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||
import io.druid.sql.calcite.planner.PlannerFactory;
|
||||
import io.druid.sql.calcite.planner.PlannerResult;
|
||||
import io.druid.sql.calcite.util.CalciteTests;
|
||||
import io.druid.sql.calcite.util.QueryLogHook;
|
||||
import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.calcite.tools.Planner;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static io.druid.sql.calcite.CalciteQueryTest.TIMESERIES_CONTEXT;
|
||||
|
||||
public class QuantileSqlAggregatorTest
|
||||
{
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
@Rule
|
||||
public QueryLogHook queryLogHook = QueryLogHook.create();
|
||||
|
||||
private SpecificSegmentsQuerySegmentWalker walker;
|
||||
private PlannerFactory plannerFactory;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
Calcites.setSystemProperties();
|
||||
walker = CalciteTests.createMockWalker(temporaryFolder.newFolder());
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final SchemaPlus rootSchema = Calcites.createRootSchema(
|
||||
CalciteTests.createMockSchema(
|
||||
walker,
|
||||
plannerConfig
|
||||
)
|
||||
);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.<SqlAggregator>of(
|
||||
new QuantileSqlAggregator()
|
||||
)
|
||||
);
|
||||
plannerFactory = new PlannerFactory(rootSchema, operatorTable, plannerConfig);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception
|
||||
{
|
||||
walker.close();
|
||||
walker = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQuantileOnFloatAndLongs() throws Exception
|
||||
{
|
||||
try (final Planner planner = plannerFactory.createPlanner()) {
|
||||
final String sql = "SELECT QUANTILE(m1, 0.01), QUANTILE(m1, 0.5), QUANTILE(m1, 0.99), QUANTILE(cnt, 0.5) FROM foo";
|
||||
final PlannerResult plannerResult = Calcites.plan(planner, sql);
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = Sequences.toList(plannerResult.run(), new ArrayList<Object[]>());
|
||||
final List<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[]{1.0, 3.0, 5.940000057220459, 1.0}
|
||||
);
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(QueryGranularities.ALL)
|
||||
.aggregators(ImmutableList.<AggregatorFactory>of(
|
||||
new ApproximateHistogramAggregatorFactory("a0:agg", "m1", null, null, null, null),
|
||||
new ApproximateHistogramAggregatorFactory("a3:agg", "cnt", null, null, null, null)
|
||||
))
|
||||
.postAggregators(ImmutableList.<PostAggregator>of(
|
||||
new QuantilePostAggregator("a0", "a0:agg", 0.01f),
|
||||
new QuantilePostAggregator("a1", "a0:agg", 0.50f),
|
||||
new QuantilePostAggregator("a2", "a0:agg", 0.99f),
|
||||
new QuantilePostAggregator("a3", "a3:agg", 0.50f)
|
||||
))
|
||||
.context(TIMESERIES_CONTEXT)
|
||||
.build(),
|
||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -28,6 +28,22 @@ import java.io.IOException;
|
|||
*/
|
||||
public class Yielders
|
||||
{
|
||||
public static <T> Yielder<T> each(final Sequence<T> sequence)
|
||||
{
|
||||
return sequence.toYielder(
|
||||
null,
|
||||
new YieldingAccumulator<T, T>()
|
||||
{
|
||||
@Override
|
||||
public T accumulate(T accumulated, T in)
|
||||
{
|
||||
yield();
|
||||
return in;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
public static <T> Yielder<T> done(final T finalVal, final Closeable closeable)
|
||||
{
|
||||
return new Yielder<T>()
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.google.common.base.Function;
|
|||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
|
@ -31,7 +30,7 @@ import io.druid.granularity.QueryGranularity;
|
|||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.guava.Yielder;
|
||||
import io.druid.java.util.common.guava.YieldingAccumulator;
|
||||
import io.druid.java.util.common.guava.Yielders;
|
||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.select.EventHolder;
|
||||
|
@ -171,18 +170,7 @@ public class IngestSegmentFirehose implements Firehose
|
|||
}
|
||||
)
|
||||
);
|
||||
rowYielder = rows.toYielder(
|
||||
null,
|
||||
new YieldingAccumulator<InputRow, InputRow>()
|
||||
{
|
||||
@Override
|
||||
public InputRow accumulate(InputRow accumulated, InputRow in)
|
||||
{
|
||||
yield();
|
||||
return in;
|
||||
}
|
||||
}
|
||||
);
|
||||
rowYielder = Yielders.each(rows);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -36,7 +36,7 @@ import io.druid.java.util.common.ISE;
|
|||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.guava.Yielder;
|
||||
import io.druid.java.util.common.guava.YieldingAccumulator;
|
||||
import io.druid.java.util.common.guava.Yielders;
|
||||
import io.druid.query.DruidMetrics;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryContextKeys;
|
||||
|
@ -226,18 +226,7 @@ public class QueryResource implements QueryCountStatsProvider
|
|||
results = res;
|
||||
}
|
||||
|
||||
final Yielder yielder = results.toYielder(
|
||||
null,
|
||||
new YieldingAccumulator()
|
||||
{
|
||||
@Override
|
||||
public Object accumulate(Object accumulated, Object in)
|
||||
{
|
||||
yield();
|
||||
return in;
|
||||
}
|
||||
}
|
||||
);
|
||||
final Yielder yielder = Yielders.each(results);
|
||||
|
||||
try {
|
||||
final Query theQuery = query;
|
||||
|
|
|
@ -0,0 +1,50 @@
|
|||
/*
|
||||
* 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.sql.avatica;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.joda.time.Period;
|
||||
|
||||
public class AvaticaServerConfig
|
||||
{
|
||||
@JsonProperty
|
||||
public int maxConnections = 25;
|
||||
|
||||
@JsonProperty
|
||||
public int maxStatementsPerConnection = 4;
|
||||
|
||||
@JsonProperty
|
||||
public Period connectionIdleTimeout = new Period("PT30M");
|
||||
|
||||
public int getMaxConnections()
|
||||
{
|
||||
return maxConnections;
|
||||
}
|
||||
|
||||
public int getMaxStatementsPerConnection()
|
||||
{
|
||||
return maxStatementsPerConnection;
|
||||
}
|
||||
|
||||
public Period getConnectionIdleTimeout()
|
||||
{
|
||||
return connectionIdleTimeout;
|
||||
}
|
||||
}
|
|
@ -22,12 +22,9 @@ package io.druid.sql.avatica;
|
|||
import com.google.inject.Inject;
|
||||
import io.druid.guice.annotations.Self;
|
||||
import io.druid.server.DruidNode;
|
||||
import org.apache.calcite.avatica.Meta;
|
||||
import org.apache.calcite.avatica.remote.LocalService;
|
||||
import org.apache.calcite.avatica.remote.Service;
|
||||
import org.apache.calcite.avatica.server.AvaticaJsonHandler;
|
||||
import org.apache.calcite.jdbc.CalciteConnection;
|
||||
import org.apache.calcite.jdbc.CalciteMetaImpl;
|
||||
import org.eclipse.jetty.server.Request;
|
||||
|
||||
import javax.servlet.ServletException;
|
||||
|
@ -42,16 +39,12 @@ public class DruidAvaticaHandler extends AvaticaJsonHandler
|
|||
|
||||
@Inject
|
||||
public DruidAvaticaHandler(
|
||||
final CalciteConnection connection,
|
||||
final DruidMeta druidMeta,
|
||||
@Self final DruidNode druidNode,
|
||||
final AvaticaMonitor avaticaMonitor
|
||||
) throws InstantiationException, IllegalAccessException, InvocationTargetException
|
||||
{
|
||||
super(
|
||||
new LocalService((Meta) CalciteMetaImpl.class.getConstructors()[0].newInstance(connection), avaticaMonitor),
|
||||
avaticaMonitor
|
||||
);
|
||||
|
||||
super(new LocalService(druidMeta), avaticaMonitor);
|
||||
setServerRpcMetadata(new Service.RpcMetadataResponse(druidNode.getHostAndPort()));
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* 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.sql.avatica;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Future;
|
||||
|
||||
/**
|
||||
* Connection tracking for {@link DruidMeta}. Not thread-safe.
|
||||
*/
|
||||
public class DruidConnection
|
||||
{
|
||||
private final Map<Integer, DruidStatement> statements;
|
||||
private Future<?> timeoutFuture;
|
||||
|
||||
public DruidConnection()
|
||||
{
|
||||
this.statements = new HashMap<>();
|
||||
}
|
||||
|
||||
public Map<Integer, DruidStatement> statements()
|
||||
{
|
||||
return statements;
|
||||
}
|
||||
|
||||
public DruidConnection sync(final Future<?> newTimeoutFuture)
|
||||
{
|
||||
if (timeoutFuture != null) {
|
||||
timeoutFuture.cancel(false);
|
||||
}
|
||||
timeoutFuture = newTimeoutFuture;
|
||||
return this;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,592 @@
|
|||
/*
|
||||
* 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.sql.avatica;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.io.Closer;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import com.google.inject.Inject;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.sql.calcite.planner.Calcites;
|
||||
import io.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.calcite.avatica.MetaImpl;
|
||||
import org.apache.calcite.avatica.MissingResultsException;
|
||||
import org.apache.calcite.avatica.NoSuchStatementException;
|
||||
import org.apache.calcite.avatica.QueryState;
|
||||
import org.apache.calcite.avatica.remote.TypedValue;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
public class DruidMeta extends MetaImpl
|
||||
{
|
||||
private static final Logger log = new Logger(DruidMeta.class);
|
||||
|
||||
private final PlannerFactory plannerFactory;
|
||||
private final ScheduledExecutorService exec;
|
||||
private final AvaticaServerConfig config;
|
||||
|
||||
// Used to track statements for a connection. Connection id -> statement id -> statement.
|
||||
// Not concurrent; synchronize on it when reading or writing.
|
||||
private final Map<String, DruidConnection> connections = new HashMap<>();
|
||||
|
||||
// Used to generate statement ids.
|
||||
private final AtomicInteger statementCounter = new AtomicInteger();
|
||||
|
||||
@Inject
|
||||
public DruidMeta(final PlannerFactory plannerFactory, final AvaticaServerConfig config)
|
||||
{
|
||||
super(null);
|
||||
this.plannerFactory = Preconditions.checkNotNull(plannerFactory, "plannerFactory");
|
||||
this.config = config;
|
||||
this.exec = Executors.newSingleThreadScheduledExecutor(
|
||||
new ThreadFactoryBuilder()
|
||||
.setNameFormat(String.format("DruidMeta@%s-ScheduledExecutor", Integer.toHexString(hashCode())))
|
||||
.setDaemon(true)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void openConnection(final ConnectionHandle ch, final Map<String, String> info)
|
||||
{
|
||||
getDruidConnection(ch.id, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void closeConnection(final ConnectionHandle ch)
|
||||
{
|
||||
final List<DruidStatement> statements = new ArrayList<>();
|
||||
|
||||
synchronized (connections) {
|
||||
final DruidConnection connection = connections.remove(ch.id);
|
||||
if (connection != null) {
|
||||
connection.sync(null);
|
||||
statements.addAll(connection.statements().values());
|
||||
log.debug("Connection[%s] closed, closing %,d statements.", ch.id, statements.size());
|
||||
}
|
||||
}
|
||||
|
||||
final Closer closer = Closer.create();
|
||||
for (final DruidStatement statement : statements) {
|
||||
closer.register(statement);
|
||||
}
|
||||
try {
|
||||
closer.close();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ConnectionProperties connectionSync(final ConnectionHandle ch, final ConnectionProperties connProps)
|
||||
{
|
||||
// getDruidConnection re-syncs it.
|
||||
getDruidConnection(ch.id);
|
||||
return connProps;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StatementHandle createStatement(final ConnectionHandle ch)
|
||||
{
|
||||
synchronized (connections) {
|
||||
final DruidConnection connection = getDruidConnection(ch.id);
|
||||
final StatementHandle statement = new StatementHandle(ch.id, statementCounter.incrementAndGet(), null);
|
||||
|
||||
if (connection.statements().containsKey(statement.id)) {
|
||||
// Will only happen if statementCounter rolls over before old statements are cleaned up. If this
|
||||
// ever happens then something fishy is going on, because we shouldn't have billions of statements.
|
||||
throw new ISE("Uh oh, too many statements");
|
||||
}
|
||||
|
||||
if (connection.statements().size() >= config.getMaxStatementsPerConnection()) {
|
||||
throw new ISE("Too many open statements, limit is[%,d]", config.getMaxStatementsPerConnection());
|
||||
}
|
||||
|
||||
connection.statements().put(statement.id, new DruidStatement(ch.id, statement.id));
|
||||
log.debug("Connection[%s] opened statement[%s].", ch.id, statement.id);
|
||||
return statement;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public StatementHandle prepare(
|
||||
final ConnectionHandle ch,
|
||||
final String sql,
|
||||
final long maxRowCount
|
||||
)
|
||||
{
|
||||
final StatementHandle statement = createStatement(ch);
|
||||
final DruidStatement druidStatement = getDruidStatement(statement);
|
||||
statement.signature = druidStatement.prepare(plannerFactory, sql, maxRowCount).getSignature();
|
||||
return statement;
|
||||
}
|
||||
|
||||
@Deprecated
|
||||
@Override
|
||||
public ExecuteResult prepareAndExecute(
|
||||
final StatementHandle h,
|
||||
final String sql,
|
||||
final long maxRowCount,
|
||||
final PrepareCallback callback
|
||||
) throws NoSuchStatementException
|
||||
{
|
||||
// Avatica doesn't call this.
|
||||
throw new UnsupportedOperationException("Deprecated");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExecuteResult prepareAndExecute(
|
||||
final StatementHandle statement,
|
||||
final String sql,
|
||||
final long maxRowCount,
|
||||
final int maxRowsInFirstFrame,
|
||||
final PrepareCallback callback
|
||||
) throws NoSuchStatementException
|
||||
{
|
||||
// Ignore "callback", this class is designed for use with LocalService which doesn't use it.
|
||||
final DruidStatement druidStatement = getDruidStatement(statement);
|
||||
final Signature signature = druidStatement.prepare(plannerFactory, sql, maxRowCount).getSignature();
|
||||
final Frame firstFrame = druidStatement.execute().nextFrame(DruidStatement.START_OFFSET, maxRowsInFirstFrame);
|
||||
|
||||
return new ExecuteResult(
|
||||
ImmutableList.of(
|
||||
MetaResultSet.create(
|
||||
statement.connectionId,
|
||||
statement.id,
|
||||
false,
|
||||
signature,
|
||||
firstFrame
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExecuteBatchResult prepareAndExecuteBatch(
|
||||
final StatementHandle statement,
|
||||
final List<String> sqlCommands
|
||||
) throws NoSuchStatementException
|
||||
{
|
||||
// Batch statements are used for bulk updates, but we don't support updates.
|
||||
throw new UnsupportedOperationException("Batch statements not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExecuteBatchResult executeBatch(
|
||||
final StatementHandle statement,
|
||||
final List<List<TypedValue>> parameterValues
|
||||
) throws NoSuchStatementException
|
||||
{
|
||||
// Batch statements are used for bulk updates, but we don't support updates.
|
||||
throw new UnsupportedOperationException("Batch statements not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Frame fetch(
|
||||
final StatementHandle statement,
|
||||
final long offset,
|
||||
final int fetchMaxRowCount
|
||||
) throws NoSuchStatementException, MissingResultsException
|
||||
{
|
||||
return getDruidStatement(statement).nextFrame(offset, fetchMaxRowCount);
|
||||
}
|
||||
|
||||
@Deprecated
|
||||
@Override
|
||||
public ExecuteResult execute(
|
||||
final StatementHandle statement,
|
||||
final List<TypedValue> parameterValues,
|
||||
final long maxRowCount
|
||||
) throws NoSuchStatementException
|
||||
{
|
||||
// Avatica doesn't call this.
|
||||
throw new UnsupportedOperationException("Deprecated");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExecuteResult execute(
|
||||
final StatementHandle statement,
|
||||
final List<TypedValue> parameterValues,
|
||||
final int maxRowsInFirstFrame
|
||||
) throws NoSuchStatementException
|
||||
{
|
||||
Preconditions.checkArgument(parameterValues.isEmpty(), "Expected parameterValues to be empty");
|
||||
|
||||
final DruidStatement druidStatement = getDruidStatement(statement);
|
||||
final Signature signature = druidStatement.getSignature();
|
||||
final Frame firstFrame = druidStatement.execute().nextFrame(DruidStatement.START_OFFSET, maxRowsInFirstFrame);
|
||||
|
||||
return new ExecuteResult(
|
||||
ImmutableList.of(
|
||||
MetaResultSet.create(
|
||||
statement.connectionId,
|
||||
statement.id,
|
||||
false,
|
||||
signature,
|
||||
firstFrame
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Object> createIterable(
|
||||
final StatementHandle statement,
|
||||
final QueryState state,
|
||||
final Signature signature,
|
||||
final List<TypedValue> parameterValues,
|
||||
final Frame firstFrame
|
||||
)
|
||||
{
|
||||
// Avatica calls this but ignores the return value.
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void closeStatement(final StatementHandle h)
|
||||
{
|
||||
closeDruidStatement(getDruidStatement(h));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean syncResults(
|
||||
final StatementHandle sh,
|
||||
final QueryState state,
|
||||
final long offset
|
||||
) throws NoSuchStatementException
|
||||
{
|
||||
final DruidStatement druidStatement = getDruidStatement(sh);
|
||||
final boolean isDone = druidStatement.isDone();
|
||||
final long currentOffset = druidStatement.getCurrentOffset();
|
||||
if (currentOffset != offset) {
|
||||
throw new ISE("Requested offset[%,d] does not match currentOffset[%,d]", offset, currentOffset);
|
||||
}
|
||||
return !isDone;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void commit(final ConnectionHandle ch)
|
||||
{
|
||||
// We don't support writes, just ignore commits.
|
||||
}
|
||||
|
||||
@Override
|
||||
public void rollback(final ConnectionHandle ch)
|
||||
{
|
||||
// We don't support writes, just ignore rollbacks.
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<DatabaseProperty, Object> getDatabaseProperties(final ConnectionHandle ch)
|
||||
{
|
||||
return ImmutableMap.of();
|
||||
}
|
||||
|
||||
@Override
|
||||
public MetaResultSet getCatalogs(final ConnectionHandle ch)
|
||||
{
|
||||
final String sql = "SELECT\n"
|
||||
+ " DISTINCT CATALOG_NAME AS TABLE_CAT\n"
|
||||
+ "FROM\n"
|
||||
+ " INFORMATION_SCHEMA.SCHEMATA\n"
|
||||
+ "ORDER BY\n"
|
||||
+ " TABLE_CAT\n";
|
||||
|
||||
return sqlResultSet(ch, sql);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MetaResultSet getSchemas(
|
||||
final ConnectionHandle ch,
|
||||
final String catalog,
|
||||
final Pat schemaPattern
|
||||
)
|
||||
{
|
||||
final List<String> whereBuilder = new ArrayList<>();
|
||||
if (catalog != null) {
|
||||
whereBuilder.add("SCHEMATA.CATALOG_NAME = " + Calcites.escapeStringLiteral(catalog));
|
||||
}
|
||||
|
||||
if (schemaPattern.s != null) {
|
||||
whereBuilder.add("SCHEMATA.SCHEMA_NAME LIKE " + Calcites.escapeStringLiteral(schemaPattern.s));
|
||||
}
|
||||
|
||||
final String where = whereBuilder.isEmpty() ? "" : "WHERE " + Joiner.on(" AND ").join(whereBuilder);
|
||||
final String sql = "SELECT\n"
|
||||
+ " SCHEMA_NAME AS TABLE_SCHEM,\n"
|
||||
+ " CATALOG_NAME AS TABLE_CATALOG\n"
|
||||
+ "FROM\n"
|
||||
+ " INFORMATION_SCHEMA.SCHEMATA\n"
|
||||
+ where + "\n"
|
||||
+ "ORDER BY\n"
|
||||
+ " TABLE_CATALOG, TABLE_SCHEM\n";
|
||||
|
||||
return sqlResultSet(ch, sql);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MetaResultSet getTables(
|
||||
final ConnectionHandle ch,
|
||||
final String catalog,
|
||||
final Pat schemaPattern,
|
||||
final Pat tableNamePattern,
|
||||
final List<String> typeList
|
||||
)
|
||||
{
|
||||
final List<String> whereBuilder = new ArrayList<>();
|
||||
if (catalog != null) {
|
||||
whereBuilder.add("TABLES.TABLE_CATALOG = " + Calcites.escapeStringLiteral(catalog));
|
||||
}
|
||||
|
||||
if (schemaPattern.s != null) {
|
||||
whereBuilder.add("TABLES.TABLE_SCHEMA LIKE " + Calcites.escapeStringLiteral(schemaPattern.s));
|
||||
}
|
||||
|
||||
if (tableNamePattern.s != null) {
|
||||
whereBuilder.add("TABLES.TABLE_NAME LIKE " + Calcites.escapeStringLiteral(tableNamePattern.s));
|
||||
}
|
||||
|
||||
if (typeList != null) {
|
||||
final List<String> escapedTypes = new ArrayList<>();
|
||||
for (String type : typeList) {
|
||||
escapedTypes.add(Calcites.escapeStringLiteral(type));
|
||||
}
|
||||
whereBuilder.add("TABLES.TABLE_TYPE IN (" + Joiner.on(", ").join(escapedTypes) + ")");
|
||||
}
|
||||
|
||||
final String where = whereBuilder.isEmpty() ? "" : "WHERE " + Joiner.on(" AND ").join(whereBuilder);
|
||||
final String sql = "SELECT\n"
|
||||
+ " TABLE_CATALOG AS TABLE_CAT,\n"
|
||||
+ " TABLE_SCHEMA AS TABLE_SCHEM,\n"
|
||||
+ " TABLE_NAME AS TABLE_NAME,\n"
|
||||
+ " TABLE_TYPE AS TABLE_TYPE,\n"
|
||||
+ " CAST(NULL AS VARCHAR) AS REMARKS,\n"
|
||||
+ " CAST(NULL AS VARCHAR) AS TYPE_CAT,\n"
|
||||
+ " CAST(NULL AS VARCHAR) AS TYPE_SCHEM,\n"
|
||||
+ " CAST(NULL AS VARCHAR) AS TYPE_NAME,\n"
|
||||
+ " CAST(NULL AS VARCHAR) AS SELF_REFERENCING_COL_NAME,\n"
|
||||
+ " CAST(NULL AS VARCHAR) AS REF_GENERATION\n"
|
||||
+ "FROM\n"
|
||||
+ " INFORMATION_SCHEMA.TABLES\n"
|
||||
+ where + "\n"
|
||||
+ "ORDER BY\n"
|
||||
+ " TABLE_TYPE, TABLE_CAT, TABLE_SCHEM, TABLE_NAME\n";
|
||||
|
||||
return sqlResultSet(ch, sql);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MetaResultSet getColumns(
|
||||
final ConnectionHandle ch,
|
||||
final String catalog,
|
||||
final Pat schemaPattern,
|
||||
final Pat tableNamePattern,
|
||||
final Pat columnNamePattern
|
||||
)
|
||||
{
|
||||
final List<String> whereBuilder = new ArrayList<>();
|
||||
if (catalog != null) {
|
||||
whereBuilder.add("COLUMNS.TABLE_CATALOG = " + Calcites.escapeStringLiteral(catalog));
|
||||
}
|
||||
|
||||
if (schemaPattern.s != null) {
|
||||
whereBuilder.add("COLUMNS.TABLE_SCHEMA LIKE " + Calcites.escapeStringLiteral(schemaPattern.s));
|
||||
}
|
||||
|
||||
if (tableNamePattern.s != null) {
|
||||
whereBuilder.add("COLUMNS.TABLE_NAME LIKE " + Calcites.escapeStringLiteral(tableNamePattern.s));
|
||||
}
|
||||
|
||||
if (columnNamePattern.s != null) {
|
||||
whereBuilder.add("COLUMNS.COLUMN_NAME LIKE " + Calcites.escapeStringLiteral(columnNamePattern.s));
|
||||
}
|
||||
|
||||
final String where = whereBuilder.isEmpty() ? "" : "WHERE " + Joiner.on(" AND ").join(whereBuilder);
|
||||
final String sql = "SELECT\n"
|
||||
+ " TABLE_CATALOG AS TABLE_CAT,\n"
|
||||
+ " TABLE_SCHEMA AS TABLE_SCHEM,\n"
|
||||
+ " TABLE_NAME AS TABLE_NAME,\n"
|
||||
+ " COLUMN_NAME AS COLUMN_NAME,\n"
|
||||
+ " CAST(JDBC_TYPE AS INTEGER) AS DATA_TYPE,\n"
|
||||
+ " DATA_TYPE AS TYPE_NAME,\n"
|
||||
+ " -1 AS COLUMN_SIZE,\n"
|
||||
+ " -1 AS BUFFER_LENGTH,\n"
|
||||
+ " -1 AS DECIMAL_DIGITS,\n"
|
||||
+ " -1 AS NUM_PREC_RADIX,\n"
|
||||
+ " CASE IS_NULLABLE WHEN 'YES' THEN 1 ELSE 0 END AS NULLABLE,\n"
|
||||
+ " CAST(NULL AS VARCHAR) AS REMARKS,\n"
|
||||
+ " COLUMN_DEFAULT AS COLUMN_DEF,\n"
|
||||
+ " -1 AS SQL_DATA_TYPE,\n"
|
||||
+ " -1 AS SQL_DATETIME_SUB,\n"
|
||||
+ " -1 AS CHAR_OCTET_LENGTH,\n"
|
||||
+ " CAST(ORDINAL_POSITION AS INTEGER) AS ORDINAL_POSITION,\n"
|
||||
+ " IS_NULLABLE AS IS_NULLABLE,\n"
|
||||
+ " CAST(NULL AS VARCHAR) AS SCOPE_CATALOG,\n"
|
||||
+ " CAST(NULL AS VARCHAR) AS SCOPE_SCHEMA,\n"
|
||||
+ " CAST(NULL AS VARCHAR) AS SCOPE_TABLE,\n"
|
||||
+ " -1 AS SOURCE_DATA_TYPE,\n"
|
||||
+ " 'NO' AS IS_AUTOINCREMENT,\n"
|
||||
+ " 'NO' AS IS_GENERATEDCOLUMN\n"
|
||||
+ "FROM\n"
|
||||
+ " INFORMATION_SCHEMA.COLUMNS\n"
|
||||
+ where + "\n"
|
||||
+ "ORDER BY\n"
|
||||
+ " TABLE_CAT, TABLE_SCHEM, TABLE_NAME, ORDINAL_POSITION\n";
|
||||
|
||||
return sqlResultSet(ch, sql);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MetaResultSet getTableTypes(final ConnectionHandle ch)
|
||||
{
|
||||
final String sql = "SELECT\n"
|
||||
+ " DISTINCT TABLE_TYPE AS TABLE_TYPE\n"
|
||||
+ "FROM\n"
|
||||
+ " INFORMATION_SCHEMA.TABLES\n"
|
||||
+ "ORDER BY\n"
|
||||
+ " TABLE_TYPE\n";
|
||||
|
||||
return sqlResultSet(ch, sql);
|
||||
}
|
||||
|
||||
private DruidConnection getDruidConnection(final String connectionId)
|
||||
{
|
||||
return getDruidConnection(connectionId, false);
|
||||
}
|
||||
|
||||
private DruidConnection getDruidConnection(final String connectionId, final boolean createIfNotExists)
|
||||
{
|
||||
DruidConnection connection;
|
||||
|
||||
synchronized (connections) {
|
||||
connection = connections.get(connectionId);
|
||||
|
||||
if (connection == null && createIfNotExists) {
|
||||
if (connections.size() >= config.getMaxConnections()) {
|
||||
throw new ISE("Too many connections, limit is[%,d]", config.getMaxConnections());
|
||||
}
|
||||
connection = new DruidConnection();
|
||||
connections.put(connectionId, connection);
|
||||
log.debug("Connection[%s] opened.", connectionId);
|
||||
}
|
||||
|
||||
if (connection == null) {
|
||||
throw new ISE("Connection[%s] not open", connectionId);
|
||||
}
|
||||
}
|
||||
|
||||
final DruidConnection finalConnection = connection;
|
||||
|
||||
return finalConnection.sync(
|
||||
exec.schedule(
|
||||
new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
final List<DruidStatement> statements = new ArrayList<>();
|
||||
|
||||
synchronized (connections) {
|
||||
if (connections.remove(connectionId) == finalConnection) {
|
||||
statements.addAll(finalConnection.statements().values());
|
||||
log.debug("Connection[%s] timed out, closing %,d statements.", connectionId, statements.size());
|
||||
}
|
||||
}
|
||||
|
||||
final Closer closer = Closer.create();
|
||||
for (final DruidStatement statement : statements) {
|
||||
closer.register(statement);
|
||||
}
|
||||
try {
|
||||
closer.close();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
},
|
||||
new Interval(new DateTime(), config.getConnectionIdleTimeout()).toDurationMillis(),
|
||||
TimeUnit.MILLISECONDS
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private DruidStatement getDruidStatement(final StatementHandle statement)
|
||||
{
|
||||
synchronized (connections) {
|
||||
final DruidConnection connection = getDruidConnection(statement.connectionId);
|
||||
final DruidStatement druidStatement = connection.statements().get(statement.id);
|
||||
Preconditions.checkState(druidStatement != null, "Statement[%s] does not exist", statement.id);
|
||||
return druidStatement;
|
||||
}
|
||||
}
|
||||
|
||||
private void closeDruidStatement(final DruidStatement statement)
|
||||
{
|
||||
synchronized (connections) {
|
||||
final DruidConnection connection = getDruidConnection(statement.getConnectionId());
|
||||
if (connection.statements().get(statement.getStatementId()) == statement) {
|
||||
connection.statements().remove(statement.getStatementId());
|
||||
} else {
|
||||
// "statement" is not actually in the set of open statements for this connection
|
||||
throw new ISE("Statement[%s] not open", statement.getStatementId());
|
||||
}
|
||||
}
|
||||
|
||||
log.debug("Connection[%s] closed statement[%s].", statement.getConnectionId(), statement.getStatementId());
|
||||
statement.close();
|
||||
}
|
||||
|
||||
private MetaResultSet sqlResultSet(final ConnectionHandle ch, final String sql)
|
||||
{
|
||||
final StatementHandle statement = createStatement(ch);
|
||||
try {
|
||||
final ExecuteResult result = prepareAndExecute(statement, sql, -1, -1, null);
|
||||
final MetaResultSet metaResultSet = Iterables.getOnlyElement(result.resultSets);
|
||||
if (!metaResultSet.firstFrame.done) {
|
||||
throw new ISE("Expected all results to be in a single frame!");
|
||||
}
|
||||
return metaResultSet;
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
finally {
|
||||
closeStatement(statement);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,282 @@
|
|||
/*
|
||||
* 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.sql.avatica;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Throwables;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.guava.Yielder;
|
||||
import io.druid.java.util.common.guava.Yielders;
|
||||
import io.druid.sql.calcite.planner.Calcites;
|
||||
import io.druid.sql.calcite.planner.PlannerFactory;
|
||||
import io.druid.sql.calcite.planner.PlannerResult;
|
||||
import io.druid.sql.calcite.rel.QueryMaker;
|
||||
import org.apache.calcite.avatica.AvaticaParameter;
|
||||
import org.apache.calcite.avatica.ColumnMetaData;
|
||||
import org.apache.calcite.avatica.Meta;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.calcite.rel.type.RelDataTypeField;
|
||||
import org.apache.calcite.tools.Planner;
|
||||
|
||||
import javax.annotation.concurrent.GuardedBy;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.sql.DatabaseMetaData;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Statement handle for {@link DruidMeta}. Thread-safe.
|
||||
*/
|
||||
public class DruidStatement implements Closeable
|
||||
{
|
||||
public static final long START_OFFSET = 0;
|
||||
|
||||
enum State
|
||||
{
|
||||
NEW,
|
||||
PREPARED,
|
||||
RUNNING,
|
||||
DONE
|
||||
}
|
||||
|
||||
private final String connectionId;
|
||||
private final int statementId;
|
||||
private final Object lock = new Object();
|
||||
|
||||
private State state = State.NEW;
|
||||
private String query;
|
||||
private long maxRowCount;
|
||||
private PlannerResult plannerResult;
|
||||
private Meta.Signature signature;
|
||||
private Yielder<Object[]> yielder;
|
||||
private int offset = 0;
|
||||
|
||||
public DruidStatement(final String connectionId, final int statementId)
|
||||
{
|
||||
this.connectionId = connectionId;
|
||||
this.statementId = statementId;
|
||||
}
|
||||
|
||||
public static List<ColumnMetaData> createColumnMetaData(final RelDataType rowType)
|
||||
{
|
||||
final List<ColumnMetaData> columns = new ArrayList<>();
|
||||
List<RelDataTypeField> fieldList = rowType.getFieldList();
|
||||
|
||||
for (int i = 0; i < fieldList.size(); i++) {
|
||||
RelDataTypeField field = fieldList.get(i);
|
||||
final ColumnMetaData.Rep rep = QueryMaker.rep(field.getType().getSqlTypeName());
|
||||
final ColumnMetaData.ScalarType columnType = ColumnMetaData.scalar(
|
||||
field.getType().getSqlTypeName().getJdbcOrdinal(),
|
||||
field.getType().getSqlTypeName().getName(),
|
||||
rep
|
||||
);
|
||||
columns.add(
|
||||
new ColumnMetaData(
|
||||
i, // ordinal
|
||||
false, // auto increment
|
||||
true, // case sensitive
|
||||
false, // searchable
|
||||
false, // currency
|
||||
field.getType().isNullable()
|
||||
? DatabaseMetaData.columnNullable
|
||||
: DatabaseMetaData.columnNoNulls, // nullable
|
||||
true, // signed
|
||||
field.getType().getPrecision(), // display size
|
||||
field.getName(), // label
|
||||
null, // column name
|
||||
null, // schema name
|
||||
field.getType().getPrecision(), // precision
|
||||
field.getType().getScale(), // scale
|
||||
null, // table name
|
||||
null, // catalog name
|
||||
columnType, // avatica type
|
||||
true, // read only
|
||||
false, // writable
|
||||
false, // definitely writable
|
||||
columnType.columnClassName() // column class name
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
return columns;
|
||||
}
|
||||
|
||||
public DruidStatement prepare(final PlannerFactory plannerFactory, final String query, final long maxRowCount)
|
||||
{
|
||||
try (final Planner planner = plannerFactory.createPlanner()) {
|
||||
synchronized (lock) {
|
||||
ensure(State.NEW);
|
||||
this.plannerResult = Calcites.plan(planner, query);
|
||||
this.maxRowCount = maxRowCount;
|
||||
this.query = query;
|
||||
this.signature = Meta.Signature.create(
|
||||
createColumnMetaData(plannerResult.rowType()),
|
||||
query,
|
||||
new ArrayList<AvaticaParameter>(),
|
||||
Meta.CursorFactory.ARRAY,
|
||||
Meta.StatementType.SELECT // We only support SELECT
|
||||
);
|
||||
this.state = State.PREPARED;
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
return this;
|
||||
}
|
||||
|
||||
public DruidStatement execute()
|
||||
{
|
||||
synchronized (lock) {
|
||||
ensure(State.PREPARED);
|
||||
|
||||
final Sequence<Object[]> baseSequence = plannerResult.run();
|
||||
|
||||
// We can't apply limits greater than Integer.MAX_VALUE, ignore them.
|
||||
final Sequence<Object[]> retSequence =
|
||||
maxRowCount >= 0 && maxRowCount <= Integer.MAX_VALUE
|
||||
? Sequences.limit(baseSequence, (int) maxRowCount)
|
||||
: baseSequence;
|
||||
|
||||
yielder = Yielders.each(retSequence);
|
||||
state = State.RUNNING;
|
||||
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public String getConnectionId()
|
||||
{
|
||||
return connectionId;
|
||||
}
|
||||
|
||||
public int getStatementId()
|
||||
{
|
||||
return statementId;
|
||||
}
|
||||
|
||||
public String getQuery()
|
||||
{
|
||||
synchronized (lock) {
|
||||
ensure(State.PREPARED, State.RUNNING, State.DONE);
|
||||
return query;
|
||||
}
|
||||
}
|
||||
|
||||
public Meta.Signature getSignature()
|
||||
{
|
||||
synchronized (lock) {
|
||||
ensure(State.PREPARED, State.RUNNING, State.DONE);
|
||||
return signature;
|
||||
}
|
||||
}
|
||||
|
||||
public RelDataType getRowType()
|
||||
{
|
||||
synchronized (lock) {
|
||||
ensure(State.PREPARED, State.RUNNING, State.DONE);
|
||||
return plannerResult.rowType();
|
||||
}
|
||||
}
|
||||
|
||||
public long getCurrentOffset()
|
||||
{
|
||||
synchronized (lock) {
|
||||
ensure(State.RUNNING, State.DONE);
|
||||
return offset;
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isDone()
|
||||
{
|
||||
synchronized (lock) {
|
||||
return state == State.DONE;
|
||||
}
|
||||
}
|
||||
|
||||
public Meta.Frame nextFrame(final long fetchOffset, final int fetchMaxRowCount)
|
||||
{
|
||||
synchronized (lock) {
|
||||
ensure(State.RUNNING);
|
||||
Preconditions.checkState(fetchOffset == offset, "fetchOffset[%,d] != offset[%,d]", fetchOffset, offset);
|
||||
|
||||
try {
|
||||
final List<Object> rows = new ArrayList<>();
|
||||
while (!yielder.isDone() && (fetchMaxRowCount < 0 || offset < fetchOffset + fetchMaxRowCount)) {
|
||||
rows.add(yielder.get());
|
||||
yielder = yielder.next(null);
|
||||
offset++;
|
||||
}
|
||||
|
||||
final boolean done = yielder.isDone();
|
||||
if (done) {
|
||||
close();
|
||||
}
|
||||
|
||||
return new Meta.Frame(fetchOffset, done, rows);
|
||||
}
|
||||
catch (Throwable t) {
|
||||
try {
|
||||
close();
|
||||
}
|
||||
catch (Throwable t1) {
|
||||
t.addSuppressed(t1);
|
||||
}
|
||||
throw t;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
synchronized (lock) {
|
||||
state = State.DONE;
|
||||
|
||||
if (yielder != null) {
|
||||
Yielder<Object[]> theYielder = this.yielder;
|
||||
this.yielder = null;
|
||||
|
||||
// Put the close last, so any exceptions it throws are after we did the other cleanup above.
|
||||
try {
|
||||
theYielder.close();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@GuardedBy("lock")
|
||||
private void ensure(final State... desiredStates)
|
||||
{
|
||||
for (State desiredState : desiredStates) {
|
||||
if (state == desiredState) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
throw new ISE("Invalid action for state[%s]", state);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,122 @@
|
|||
/*
|
||||
* 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.sql.calcite.aggregation;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.sql.calcite.expression.Expressions;
|
||||
import io.druid.sql.calcite.expression.RowExtraction;
|
||||
import io.druid.sql.calcite.table.RowSignature;
|
||||
import org.apache.calcite.rel.core.AggregateCall;
|
||||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.sql.SqlAggFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.sql.type.InferTypes;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.ReturnTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class ApproxCountDistinctSqlAggregator implements SqlAggregator
|
||||
{
|
||||
private static final SqlAggFunction FUNCTION_INSTANCE = new ApproxCountDistinctSqlAggFunction();
|
||||
private static final String NAME = "APPROX_COUNT_DISTINCT";
|
||||
|
||||
@Override
|
||||
public SqlAggFunction calciteFunction()
|
||||
{
|
||||
return FUNCTION_INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Aggregation toDruidAggregation(
|
||||
final String name,
|
||||
final RowSignature rowSignature,
|
||||
final List<Aggregation> existingAggregations,
|
||||
final Project project,
|
||||
final AggregateCall aggregateCall
|
||||
)
|
||||
{
|
||||
final RowExtraction rex = Expressions.toRowExtraction(
|
||||
rowSignature.getRowOrder(),
|
||||
Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
Iterables.getOnlyElement(aggregateCall.getArgList())
|
||||
)
|
||||
);
|
||||
if (rex == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final AggregatorFactory aggregatorFactory;
|
||||
|
||||
if (rowSignature.getColumnType(rex.getColumn()) == ValueType.COMPLEX) {
|
||||
aggregatorFactory = new HyperUniquesAggregatorFactory(name, rex.getColumn());
|
||||
} else {
|
||||
final DimensionSpec dimensionSpec = rex.toDimensionSpec(rowSignature, null);
|
||||
if (dimensionSpec == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
aggregatorFactory = new CardinalityAggregatorFactory(name, ImmutableList.of(dimensionSpec), false);
|
||||
}
|
||||
|
||||
return Aggregation.createFinalizable(
|
||||
ImmutableList.<AggregatorFactory>of(aggregatorFactory),
|
||||
null,
|
||||
new PostAggregatorFactory()
|
||||
{
|
||||
@Override
|
||||
public PostAggregator factorize(String outputName)
|
||||
{
|
||||
return new HyperUniqueFinalizingPostAggregator(outputName, name);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private static class ApproxCountDistinctSqlAggFunction extends SqlAggFunction
|
||||
{
|
||||
ApproxCountDistinctSqlAggFunction()
|
||||
{
|
||||
super(
|
||||
NAME,
|
||||
null,
|
||||
SqlKind.OTHER_FUNCTION,
|
||||
ReturnTypes.explicit(SqlTypeName.BIGINT),
|
||||
InferTypes.VARCHAR_1024,
|
||||
OperandTypes.ANY,
|
||||
SqlFunctionCategory.STRING,
|
||||
false,
|
||||
false
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,61 @@
|
|||
/*
|
||||
* 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.sql.calcite.aggregation;
|
||||
|
||||
import io.druid.sql.calcite.table.RowSignature;
|
||||
import org.apache.calcite.rel.core.AggregateCall;
|
||||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.sql.SqlAggFunction;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Bridge between Druid and SQL aggregators.
|
||||
*/
|
||||
public interface SqlAggregator
|
||||
{
|
||||
/**
|
||||
* Returns the SQL operator corresponding to this aggregation function. Should be a singleton.
|
||||
*
|
||||
* @return operator
|
||||
*/
|
||||
SqlAggFunction calciteFunction();
|
||||
|
||||
/**
|
||||
* Returns Druid Aggregation corresponding to a SQL {@link AggregateCall}.
|
||||
*
|
||||
* @param name desired output name of the aggregation
|
||||
* @param rowSignature signature of the rows being aggregated
|
||||
* @param existingAggregations existing aggregations for this query; useful for re-using aggregators
|
||||
* @param project SQL projection to apply before the aggregate call
|
||||
* @param aggregateCall SQL aggregate call
|
||||
*
|
||||
* @return aggregation, or null if the call cannot be translated
|
||||
*/
|
||||
@Nullable
|
||||
Aggregation toDruidAggregation(
|
||||
final String name,
|
||||
final RowSignature rowSignature,
|
||||
final List<Aggregation> existingAggregations,
|
||||
final Project project,
|
||||
final AggregateCall aggregateCall
|
||||
);
|
||||
}
|
|
@ -31,6 +31,7 @@ import io.druid.query.filter.DimFilter;
|
|||
import io.druid.query.filter.NotDimFilter;
|
||||
import io.druid.query.filter.OrDimFilter;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -52,22 +53,57 @@ public class CombineAndSimplifyBounds extends BottomUpTransform
|
|||
public DimFilter process(DimFilter filter)
|
||||
{
|
||||
if (filter instanceof AndDimFilter) {
|
||||
final List<DimFilter> children = ((AndDimFilter) filter).getFields();
|
||||
final List<DimFilter> children = getAndFilterChildren((AndDimFilter) filter);
|
||||
final DimFilter one = doSimplifyAnd(children);
|
||||
final DimFilter two = negate(doSimplifyOr(negateAll(children)));
|
||||
return computeCost(one) <= computeCost(two) ? one : two;
|
||||
} else if (filter instanceof OrDimFilter) {
|
||||
final List<DimFilter> children = ((OrDimFilter) filter).getFields();
|
||||
final List<DimFilter> children = getOrFilterChildren((OrDimFilter) filter);
|
||||
final DimFilter one = doSimplifyOr(children);
|
||||
final DimFilter two = negate(doSimplifyAnd(negateAll(children)));
|
||||
return computeCost(one) <= computeCost(two) ? one : two;
|
||||
} else if (filter instanceof NotDimFilter) {
|
||||
return negate(((NotDimFilter) filter).getField());
|
||||
final DimFilter field = ((NotDimFilter) filter).getField();
|
||||
final DimFilter candidate;
|
||||
if (field instanceof OrDimFilter) {
|
||||
candidate = doSimplifyAnd(negateAll(getOrFilterChildren((OrDimFilter) field)));
|
||||
} else if (field instanceof AndDimFilter) {
|
||||
candidate = doSimplifyOr(negateAll(getAndFilterChildren((AndDimFilter) field)));
|
||||
} else {
|
||||
candidate = negate(field);
|
||||
}
|
||||
return computeCost(filter) <= computeCost(candidate) ? filter : candidate;
|
||||
} else {
|
||||
return filter;
|
||||
}
|
||||
}
|
||||
|
||||
private List<DimFilter> getAndFilterChildren(final AndDimFilter filter)
|
||||
{
|
||||
final List<DimFilter> children = new ArrayList<>();
|
||||
for (final DimFilter field : filter.getFields()) {
|
||||
if (field instanceof AndDimFilter) {
|
||||
children.addAll(getAndFilterChildren((AndDimFilter) field));
|
||||
} else {
|
||||
children.add(field);
|
||||
}
|
||||
}
|
||||
return children;
|
||||
}
|
||||
|
||||
private List<DimFilter> getOrFilterChildren(final OrDimFilter filter)
|
||||
{
|
||||
final List<DimFilter> children = new ArrayList<>();
|
||||
for (final DimFilter field : filter.getFields()) {
|
||||
if (field instanceof OrDimFilter) {
|
||||
children.addAll(getOrFilterChildren((OrDimFilter) field));
|
||||
} else {
|
||||
children.add(field);
|
||||
}
|
||||
}
|
||||
return children;
|
||||
}
|
||||
|
||||
private static DimFilter doSimplifyAnd(final List<DimFilter> children)
|
||||
{
|
||||
return doSimplify(children, false);
|
||||
|
|
|
@ -19,71 +19,275 @@
|
|||
|
||||
package io.druid.sql.calcite.planner;
|
||||
|
||||
import org.apache.calcite.jdbc.CalciteConnection;
|
||||
import org.apache.calcite.jdbc.CalciteJdbc41Factory;
|
||||
import org.apache.calcite.jdbc.CalcitePrepare;
|
||||
import org.apache.calcite.jdbc.Driver;
|
||||
import org.apache.calcite.linq4j.function.Function0;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.io.BaseEncoding;
|
||||
import com.google.common.primitives.Chars;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.sql.calcite.rel.DruidConvention;
|
||||
import io.druid.sql.calcite.rel.DruidRel;
|
||||
import io.druid.sql.calcite.schema.DruidSchema;
|
||||
import io.druid.sql.calcite.schema.InformationSchema;
|
||||
import org.apache.calcite.DataContext;
|
||||
import org.apache.calcite.adapter.java.JavaTypeFactory;
|
||||
import org.apache.calcite.interpreter.BindableConvention;
|
||||
import org.apache.calcite.interpreter.BindableRel;
|
||||
import org.apache.calcite.interpreter.Bindables;
|
||||
import org.apache.calcite.jdbc.CalciteSchema;
|
||||
import org.apache.calcite.linq4j.Enumerable;
|
||||
import org.apache.calcite.linq4j.QueryProvider;
|
||||
import org.apache.calcite.plan.RelOptPlanner;
|
||||
import org.apache.calcite.plan.RelOptUtil;
|
||||
import org.apache.calcite.rel.RelNode;
|
||||
import org.apache.calcite.rel.RelRoot;
|
||||
import org.apache.calcite.rel.type.RelDataTypeFactory;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.schema.Schema;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.calcite.sql.SqlExplain;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.sql.SqlNode;
|
||||
import org.apache.calcite.sql.parser.SqlParseException;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
import org.apache.calcite.tools.Planner;
|
||||
import org.apache.calcite.tools.RelConversionException;
|
||||
import org.apache.calcite.tools.ValidationException;
|
||||
import org.apache.calcite.util.ConversionUtil;
|
||||
import org.apache.calcite.util.Pair;
|
||||
|
||||
import java.sql.SQLException;
|
||||
import java.util.Properties;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Entry points for Calcite.
|
||||
*/
|
||||
public class Calcites
|
||||
{
|
||||
private static final String DRUID_SCHEMA_NAME = "druid";
|
||||
private static final Charset DEFAULT_CHARSET = Charset.forName(ConversionUtil.NATIVE_UTF16_CHARSET_NAME);
|
||||
|
||||
private Calcites()
|
||||
{
|
||||
// No instantiation.
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a Calcite JDBC driver.
|
||||
*
|
||||
* @param druidSchema "druid" schema
|
||||
*
|
||||
* @return JDBC driver
|
||||
*/
|
||||
public static CalciteConnection jdbc(
|
||||
final Schema druidSchema,
|
||||
final PlannerConfig plannerConfig
|
||||
) throws SQLException
|
||||
public static void setSystemProperties()
|
||||
{
|
||||
final Properties props = new Properties();
|
||||
props.setProperty("caseSensitive", "true");
|
||||
props.setProperty("unquotedCasing", "UNCHANGED");
|
||||
// These properties control the charsets used for SQL literals. I don't see a way to change this except through
|
||||
// system properties, so we'll have to set those...
|
||||
|
||||
final CalciteJdbc41Factory jdbcFactory = new CalciteJdbc41Factory();
|
||||
final Function0<CalcitePrepare> prepareFactory = new Function0<CalcitePrepare>()
|
||||
{
|
||||
@Override
|
||||
public CalcitePrepare apply()
|
||||
{
|
||||
return new DruidPlannerImpl(plannerConfig);
|
||||
final String charset = ConversionUtil.NATIVE_UTF16_CHARSET_NAME;
|
||||
System.setProperty("saffron.default.charset", Calcites.defaultCharset().name());
|
||||
System.setProperty("saffron.default.nationalcharset", Calcites.defaultCharset().name());
|
||||
System.setProperty("saffron.default.collation.name", String.format("%s$en_US", charset));
|
||||
}
|
||||
|
||||
public static Charset defaultCharset()
|
||||
{
|
||||
return DEFAULT_CHARSET;
|
||||
}
|
||||
|
||||
public static SchemaPlus createRootSchema(final Schema druidSchema)
|
||||
{
|
||||
final SchemaPlus rootSchema = CalciteSchema.createRootSchema(false, false).plus();
|
||||
rootSchema.add(DruidSchema.NAME, druidSchema);
|
||||
rootSchema.add(InformationSchema.NAME, new InformationSchema(rootSchema));
|
||||
return rootSchema;
|
||||
}
|
||||
|
||||
public static String escapeStringLiteral(final String s)
|
||||
{
|
||||
if (s == null) {
|
||||
return "''";
|
||||
} else {
|
||||
boolean isPlainAscii = true;
|
||||
final StringBuilder builder = new StringBuilder("'");
|
||||
for (int i = 0; i < s.length(); i++) {
|
||||
final char c = s.charAt(i);
|
||||
if (Character.isLetterOrDigit(c) || c == ' ') {
|
||||
builder.append(c);
|
||||
if (c > 127) {
|
||||
isPlainAscii = false;
|
||||
}
|
||||
} else {
|
||||
builder.append("\\").append(BaseEncoding.base16().encode(Chars.toByteArray(c)));
|
||||
isPlainAscii = false;
|
||||
}
|
||||
}
|
||||
};
|
||||
final Driver driver = new Driver()
|
||||
{
|
||||
@Override
|
||||
protected Function0<CalcitePrepare> createPrepareFactory()
|
||||
{
|
||||
return prepareFactory;
|
||||
builder.append("'");
|
||||
return isPlainAscii ? builder.toString() : "U&" + builder.toString();
|
||||
}
|
||||
}
|
||||
|
||||
public static PlannerResult plan(
|
||||
final Planner planner,
|
||||
final String sql
|
||||
) throws SqlParseException, ValidationException, RelConversionException
|
||||
{
|
||||
SqlExplain explain = null;
|
||||
SqlNode parsed = planner.parse(sql);
|
||||
if (parsed.getKind() == SqlKind.EXPLAIN) {
|
||||
explain = (SqlExplain) parsed;
|
||||
parsed = explain.getExplicandum();
|
||||
}
|
||||
final SqlNode validated = planner.validate(parsed);
|
||||
final RelRoot root = planner.rel(validated);
|
||||
|
||||
try {
|
||||
return planWithDruidConvention(planner, explain, root);
|
||||
}
|
||||
catch (RelOptPlanner.CannotPlanException e) {
|
||||
// Try again with BINDABLE convention. Used for querying Values, metadata tables, and fallback.
|
||||
try {
|
||||
return planWithBindableConvention(planner, explain, root);
|
||||
}
|
||||
};
|
||||
final CalciteConnection calciteConnection = (CalciteConnection) jdbcFactory.newConnection(
|
||||
driver,
|
||||
jdbcFactory,
|
||||
"jdbc:calcite:",
|
||||
props
|
||||
catch (Exception e2) {
|
||||
e.addSuppressed(e2);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static PlannerResult planWithDruidConvention(
|
||||
final Planner planner,
|
||||
final SqlExplain explain,
|
||||
final RelRoot root
|
||||
) throws RelConversionException
|
||||
{
|
||||
final DruidRel<?> druidRel = (DruidRel<?>) planner.transform(
|
||||
Rules.DRUID_CONVENTION_RULES,
|
||||
planner.getEmptyTraitSet()
|
||||
.replace(DruidConvention.instance())
|
||||
.plus(root.collation),
|
||||
root.rel
|
||||
);
|
||||
|
||||
final SchemaPlus druidSchemaPlus = calciteConnection.getRootSchema().add(DRUID_SCHEMA_NAME, druidSchema);
|
||||
druidSchemaPlus.setCacheEnabled(false);
|
||||
return calciteConnection;
|
||||
if (explain != null) {
|
||||
return planExplanation(druidRel, explain);
|
||||
} else {
|
||||
final Supplier<Sequence<Object[]>> resultsSupplier = new Supplier<Sequence<Object[]>>()
|
||||
{
|
||||
@Override
|
||||
public Sequence<Object[]> get()
|
||||
{
|
||||
if (root.isRefTrivial()) {
|
||||
return druidRel.runQuery();
|
||||
} else {
|
||||
// Add a mapping on top to accommodate root.fields.
|
||||
return Sequences.map(
|
||||
druidRel.runQuery(),
|
||||
new Function<Object[], Object[]>()
|
||||
{
|
||||
@Override
|
||||
public Object[] apply(final Object[] input)
|
||||
{
|
||||
final Object[] retVal = new Object[root.fields.size()];
|
||||
for (int i = 0; i < root.fields.size(); i++) {
|
||||
retVal[i] = input[root.fields.get(i).getKey()];
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
};
|
||||
return new PlannerResult(resultsSupplier, root.validatedRowType);
|
||||
}
|
||||
}
|
||||
|
||||
private static PlannerResult planWithBindableConvention(
|
||||
final Planner planner,
|
||||
final SqlExplain explain,
|
||||
final RelRoot root
|
||||
) throws RelConversionException
|
||||
{
|
||||
BindableRel bindableRel = (BindableRel) planner.transform(
|
||||
Rules.BINDABLE_CONVENTION_RULES,
|
||||
planner.getEmptyTraitSet()
|
||||
.replace(BindableConvention.INSTANCE)
|
||||
.plus(root.collation),
|
||||
root.rel
|
||||
);
|
||||
|
||||
if (!root.isRefTrivial()) {
|
||||
// Add a projection on top to accommodate root.fields.
|
||||
final List<RexNode> projects = new ArrayList<>();
|
||||
final RexBuilder rexBuilder = bindableRel.getCluster().getRexBuilder();
|
||||
for (int field : Pair.left(root.fields)) {
|
||||
projects.add(rexBuilder.makeInputRef(bindableRel, field));
|
||||
}
|
||||
bindableRel = new Bindables.BindableProject(
|
||||
bindableRel.getCluster(),
|
||||
bindableRel.getTraitSet(),
|
||||
bindableRel,
|
||||
projects,
|
||||
root.validatedRowType
|
||||
);
|
||||
}
|
||||
|
||||
if (explain != null) {
|
||||
return planExplanation(bindableRel, explain);
|
||||
} else {
|
||||
final BindableRel theRel = bindableRel;
|
||||
final DataContext dataContext = new DataContext()
|
||||
{
|
||||
@Override
|
||||
public SchemaPlus getRootSchema()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaTypeFactory getTypeFactory()
|
||||
{
|
||||
return (JavaTypeFactory) planner.getTypeFactory();
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryProvider getQueryProvider()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get(final String name)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
};
|
||||
final Supplier<Sequence<Object[]>> resultsSupplier = new Supplier<Sequence<Object[]>>()
|
||||
{
|
||||
@Override
|
||||
public Sequence<Object[]> get()
|
||||
{
|
||||
final Enumerable enumerable = theRel.bind(dataContext);
|
||||
return Sequences.simple(enumerable);
|
||||
}
|
||||
};
|
||||
return new PlannerResult(resultsSupplier, root.validatedRowType);
|
||||
}
|
||||
}
|
||||
|
||||
private static PlannerResult planExplanation(
|
||||
final RelNode rel,
|
||||
final SqlExplain explain
|
||||
)
|
||||
{
|
||||
final String explanation = RelOptUtil.dumpPlan("", rel, explain.getFormat(), explain.getDetailLevel());
|
||||
final Supplier<Sequence<Object[]>> resultsSupplier = Suppliers.ofInstance(
|
||||
Sequences.simple(ImmutableList.of(new Object[]{explanation})));
|
||||
final RelDataTypeFactory typeFactory = rel.getCluster().getTypeFactory();
|
||||
return new PlannerResult(
|
||||
resultsSupplier,
|
||||
typeFactory.createStructType(
|
||||
ImmutableList.of(typeFactory.createSqlType(SqlTypeName.VARCHAR)),
|
||||
ImmutableList.of("PLAN")
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,90 @@
|
|||
/*
|
||||
* 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.sql.calcite.planner;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Inject;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.sql.calcite.aggregation.SqlAggregator;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlIdentifier;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.SqlOperatorTable;
|
||||
import org.apache.calcite.sql.SqlSyntax;
|
||||
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class DruidOperatorTable implements SqlOperatorTable
|
||||
{
|
||||
private static final SqlStdOperatorTable STANDARD_TABLE = SqlStdOperatorTable.instance();
|
||||
|
||||
private final Map<String, SqlAggregator> aggregators;
|
||||
|
||||
@Inject
|
||||
public DruidOperatorTable(
|
||||
final Set<SqlAggregator> aggregators
|
||||
)
|
||||
{
|
||||
this.aggregators = Maps.newHashMap();
|
||||
for (SqlAggregator aggregator : aggregators) {
|
||||
final String lcname = aggregator.calciteFunction().getName().toLowerCase();
|
||||
if (this.aggregators.put(lcname, aggregator) != null) {
|
||||
throw new ISE("Cannot have two aggregators with name[%s]", lcname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public SqlAggregator lookupAggregator(final String opName)
|
||||
{
|
||||
return aggregators.get(opName.toLowerCase());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void lookupOperatorOverloads(
|
||||
final SqlIdentifier opName,
|
||||
final SqlFunctionCategory category,
|
||||
final SqlSyntax syntax,
|
||||
final List<SqlOperator> operatorList
|
||||
)
|
||||
{
|
||||
if (opName.names.size() == 1) {
|
||||
final SqlAggregator aggregator = aggregators.get(opName.getSimple().toLowerCase());
|
||||
if (aggregator != null && syntax == SqlSyntax.FUNCTION) {
|
||||
operatorList.add(aggregator.calciteFunction());
|
||||
}
|
||||
}
|
||||
STANDARD_TABLE.lookupOperatorOverloads(opName, category, syntax, operatorList);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<SqlOperator> getOperatorList()
|
||||
{
|
||||
final List<SqlOperator> retVal = new ArrayList<>();
|
||||
for (SqlAggregator aggregator : aggregators.values()) {
|
||||
retVal.add(aggregator.calciteFunction());
|
||||
}
|
||||
retVal.addAll(STANDARD_TABLE.getOperatorList());
|
||||
return retVal;
|
||||
}
|
||||
}
|
|
@ -1,75 +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.sql.calcite.planner;
|
||||
|
||||
import org.apache.calcite.plan.Contexts;
|
||||
import org.apache.calcite.plan.ConventionTraitDef;
|
||||
import org.apache.calcite.plan.RelOptCostFactory;
|
||||
import org.apache.calcite.plan.RelOptPlanner;
|
||||
import org.apache.calcite.plan.RelOptRule;
|
||||
import org.apache.calcite.plan.volcano.VolcanoPlanner;
|
||||
import org.apache.calcite.prepare.CalcitePrepareImpl;
|
||||
import org.apache.calcite.rel.RelCollationTraitDef;
|
||||
import org.apache.calcite.sql2rel.SqlRexConvertletTable;
|
||||
|
||||
/**
|
||||
* Our very own subclass of CalcitePrepareImpl, used to alter behaviors of the JDBC driver as necessary.
|
||||
*
|
||||
* When Calcite 1.11.0 is released, we should override "createConvertletTable" and provide the
|
||||
* DruidConvertletTable.
|
||||
*/
|
||||
public class DruidPlannerImpl extends CalcitePrepareImpl
|
||||
{
|
||||
private final PlannerConfig plannerConfig;
|
||||
|
||||
public DruidPlannerImpl(PlannerConfig plannerConfig)
|
||||
{
|
||||
this.plannerConfig = plannerConfig;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RelOptPlanner createPlanner(
|
||||
final Context prepareContext,
|
||||
final org.apache.calcite.plan.Context externalContext0,
|
||||
final RelOptCostFactory costFactory
|
||||
)
|
||||
{
|
||||
final org.apache.calcite.plan.Context externalContext = externalContext0 != null
|
||||
? externalContext0
|
||||
: Contexts.of(prepareContext.config());
|
||||
|
||||
final VolcanoPlanner planner = new VolcanoPlanner(costFactory, externalContext);
|
||||
planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
|
||||
planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
|
||||
|
||||
// Register planner rules.
|
||||
for (RelOptRule rule : Rules.ruleSet(plannerConfig)) {
|
||||
planner.addRule(rule);
|
||||
}
|
||||
|
||||
return planner;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SqlRexConvertletTable createConvertletTable()
|
||||
{
|
||||
return DruidConvertletTable.instance();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,81 @@
|
|||
/*
|
||||
* 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.sql.calcite.planner;
|
||||
|
||||
import com.google.inject.Inject;
|
||||
import io.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.calcite.avatica.util.Casing;
|
||||
import org.apache.calcite.avatica.util.Quoting;
|
||||
import org.apache.calcite.plan.Contexts;
|
||||
import org.apache.calcite.plan.ConventionTraitDef;
|
||||
import org.apache.calcite.rel.RelCollationTraitDef;
|
||||
import org.apache.calcite.rel.type.RelDataTypeSystem;
|
||||
import org.apache.calcite.rex.RexExecutorImpl;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.calcite.schema.Schemas;
|
||||
import org.apache.calcite.sql.parser.SqlParser;
|
||||
import org.apache.calcite.tools.FrameworkConfig;
|
||||
import org.apache.calcite.tools.Frameworks;
|
||||
import org.apache.calcite.tools.Planner;
|
||||
|
||||
public class PlannerFactory
|
||||
{
|
||||
private final SchemaPlus rootSchema;
|
||||
private final DruidOperatorTable operatorTable;
|
||||
private final PlannerConfig plannerConfig;
|
||||
|
||||
@Inject
|
||||
public PlannerFactory(
|
||||
final SchemaPlus rootSchema,
|
||||
final DruidOperatorTable operatorTable,
|
||||
final PlannerConfig plannerConfig
|
||||
)
|
||||
{
|
||||
this.rootSchema = rootSchema;
|
||||
this.operatorTable = operatorTable;
|
||||
this.plannerConfig = plannerConfig;
|
||||
}
|
||||
|
||||
public Planner createPlanner()
|
||||
{
|
||||
final FrameworkConfig frameworkConfig = Frameworks
|
||||
.newConfigBuilder()
|
||||
.parserConfig(
|
||||
SqlParser.configBuilder()
|
||||
.setCaseSensitive(true)
|
||||
.setUnquotedCasing(Casing.UNCHANGED)
|
||||
.setQuotedCasing(Casing.UNCHANGED)
|
||||
.setQuoting(Quoting.DOUBLE_QUOTE)
|
||||
.build()
|
||||
)
|
||||
.defaultSchema(rootSchema)
|
||||
.traitDefs(ConventionTraitDef.INSTANCE, RelCollationTraitDef.INSTANCE)
|
||||
.convertletTable(DruidConvertletTable.instance())
|
||||
.operatorTable(operatorTable)
|
||||
.programs(Rules.programs(operatorTable, plannerConfig))
|
||||
.executor(new RexExecutorImpl(Schemas.createDataContext(null)))
|
||||
.context(Contexts.EMPTY_CONTEXT)
|
||||
.typeSystem(RelDataTypeSystem.DEFAULT)
|
||||
.defaultSchema(rootSchema.getSubSchema(DruidSchema.NAME))
|
||||
.build();
|
||||
|
||||
return Frameworks.getPlanner(frameworkConfig);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* 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.sql.calcite.planner;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
public class PlannerResult
|
||||
{
|
||||
private final Supplier<Sequence<Object[]>> resultsSupplier;
|
||||
private final RelDataType rowType;
|
||||
private final AtomicBoolean didRun = new AtomicBoolean();
|
||||
|
||||
public PlannerResult(final Supplier<Sequence<Object[]>> resultsSupplier, final RelDataType rowType)
|
||||
{
|
||||
this.resultsSupplier = resultsSupplier;
|
||||
this.rowType = rowType;
|
||||
}
|
||||
|
||||
public Sequence<Object[]> run()
|
||||
{
|
||||
if (!didRun.compareAndSet(false, true)) {
|
||||
// Safety check.
|
||||
throw new IllegalStateException("Cannot run more than once");
|
||||
}
|
||||
return resultsSupplier.get();
|
||||
}
|
||||
|
||||
public RelDataType rowType()
|
||||
{
|
||||
return rowType;
|
||||
}
|
||||
}
|
|
@ -20,13 +20,12 @@
|
|||
package io.druid.sql.calcite.planner;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.sql.calcite.rule.DruidBindableConverterRule;
|
||||
import io.druid.sql.calcite.rule.DruidFilterRule;
|
||||
import io.druid.sql.calcite.rule.DruidRelToBindableRule;
|
||||
import io.druid.sql.calcite.rule.DruidRelToDruidRule;
|
||||
import io.druid.sql.calcite.rule.DruidSemiJoinRule;
|
||||
import io.druid.sql.calcite.rule.GroupByRules;
|
||||
import io.druid.sql.calcite.rule.SelectRules;
|
||||
import org.apache.calcite.adapter.enumerable.EnumerableInterpreterRule;
|
||||
import org.apache.calcite.adapter.enumerable.EnumerableRules;
|
||||
import org.apache.calcite.interpreter.Bindables;
|
||||
import org.apache.calcite.plan.RelOptRule;
|
||||
import org.apache.calcite.plan.volcano.AbstractConverter;
|
||||
|
@ -64,11 +63,16 @@ import org.apache.calcite.rel.rules.UnionMergeRule;
|
|||
import org.apache.calcite.rel.rules.UnionPullUpConstantsRule;
|
||||
import org.apache.calcite.rel.rules.UnionToDistinctRule;
|
||||
import org.apache.calcite.rel.rules.ValuesReduceRule;
|
||||
import org.apache.calcite.tools.Program;
|
||||
import org.apache.calcite.tools.Programs;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class Rules
|
||||
{
|
||||
public static final int DRUID_CONVENTION_RULES = 0;
|
||||
public static final int BINDABLE_CONVENTION_RULES = 1;
|
||||
|
||||
// Rules from CalcitePrepareImpl's DEFAULT_RULES, minus AggregateExpandDistinctAggregatesRule
|
||||
// and AggregateReduceFunctionsRule.
|
||||
private static final List<RelOptRule> DEFAULT_RULES =
|
||||
|
@ -97,9 +101,7 @@ public class Rules
|
|||
ImmutableList.of(
|
||||
Bindables.BINDABLE_TABLE_SCAN_RULE,
|
||||
ProjectTableScanRule.INSTANCE,
|
||||
ProjectTableScanRule.INTERPRETER,
|
||||
EnumerableInterpreterRule.INSTANCE,
|
||||
EnumerableRules.ENUMERABLE_VALUES_RULE
|
||||
ProjectTableScanRule.INTERPRETER
|
||||
);
|
||||
|
||||
// Rules from CalcitePrepareImpl's CONSTANT_REDUCTION_RULES.
|
||||
|
@ -115,30 +117,6 @@ public class Rules
|
|||
AggregateValuesRule.INSTANCE
|
||||
);
|
||||
|
||||
// Rules from CalcitePrepareImpl's ENUMERABLE_RULES.
|
||||
private static final List<RelOptRule> ENUMERABLE_RULES =
|
||||
ImmutableList.of(
|
||||
EnumerableRules.ENUMERABLE_JOIN_RULE,
|
||||
EnumerableRules.ENUMERABLE_MERGE_JOIN_RULE,
|
||||
EnumerableRules.ENUMERABLE_SEMI_JOIN_RULE,
|
||||
EnumerableRules.ENUMERABLE_CORRELATE_RULE,
|
||||
EnumerableRules.ENUMERABLE_PROJECT_RULE,
|
||||
EnumerableRules.ENUMERABLE_FILTER_RULE,
|
||||
EnumerableRules.ENUMERABLE_AGGREGATE_RULE,
|
||||
EnumerableRules.ENUMERABLE_SORT_RULE,
|
||||
EnumerableRules.ENUMERABLE_LIMIT_RULE,
|
||||
EnumerableRules.ENUMERABLE_COLLECT_RULE,
|
||||
EnumerableRules.ENUMERABLE_UNCOLLECT_RULE,
|
||||
EnumerableRules.ENUMERABLE_UNION_RULE,
|
||||
EnumerableRules.ENUMERABLE_INTERSECT_RULE,
|
||||
EnumerableRules.ENUMERABLE_MINUS_RULE,
|
||||
EnumerableRules.ENUMERABLE_TABLE_MODIFICATION_RULE,
|
||||
EnumerableRules.ENUMERABLE_VALUES_RULE,
|
||||
EnumerableRules.ENUMERABLE_WINDOW_RULE,
|
||||
EnumerableRules.ENUMERABLE_TABLE_SCAN_RULE,
|
||||
EnumerableRules.ENUMERABLE_TABLE_FUNCTION_SCAN_RULE
|
||||
);
|
||||
|
||||
// Rules from VolcanoPlanner's registerAbstractRelationalRules.
|
||||
private static final List<RelOptRule> VOLCANO_ABSTRACT_RULES =
|
||||
ImmutableList.of(
|
||||
|
@ -180,7 +158,37 @@ public class Rules
|
|||
// No instantiation.
|
||||
}
|
||||
|
||||
public static List<RelOptRule> ruleSet(final PlannerConfig plannerConfig)
|
||||
public static List<Program> programs(final DruidOperatorTable operatorTable, final PlannerConfig plannerConfig)
|
||||
{
|
||||
return ImmutableList.of(
|
||||
Programs.ofRules(druidConventionRuleSet(operatorTable, plannerConfig)),
|
||||
Programs.ofRules(bindableConventionRuleSet(operatorTable, plannerConfig))
|
||||
);
|
||||
}
|
||||
|
||||
private static List<RelOptRule> druidConventionRuleSet(
|
||||
final DruidOperatorTable operatorTable,
|
||||
final PlannerConfig plannerConfig
|
||||
)
|
||||
{
|
||||
return ImmutableList.<RelOptRule>builder()
|
||||
.addAll(baseRuleSet(operatorTable, plannerConfig))
|
||||
.add(DruidRelToDruidRule.instance())
|
||||
.build();
|
||||
}
|
||||
|
||||
private static List<RelOptRule> bindableConventionRuleSet(
|
||||
final DruidOperatorTable operatorTable,
|
||||
final PlannerConfig plannerConfig
|
||||
)
|
||||
{
|
||||
return ImmutableList.<RelOptRule>builder()
|
||||
.addAll(baseRuleSet(operatorTable, plannerConfig))
|
||||
.addAll(Bindables.RULES)
|
||||
.build();
|
||||
}
|
||||
|
||||
private static List<RelOptRule> baseRuleSet(final DruidOperatorTable operatorTable, final PlannerConfig plannerConfig)
|
||||
{
|
||||
final ImmutableList.Builder<RelOptRule> rules = ImmutableList.builder();
|
||||
|
||||
|
@ -192,7 +200,7 @@ public class Rules
|
|||
rules.addAll(RELOPTUTIL_ABSTRACT_RULES);
|
||||
|
||||
if (plannerConfig.isUseFallback()) {
|
||||
rules.addAll(ENUMERABLE_RULES);
|
||||
rules.add(DruidRelToBindableRule.instance());
|
||||
}
|
||||
|
||||
// Druid-specific rules.
|
||||
|
@ -203,10 +211,7 @@ public class Rules
|
|||
}
|
||||
|
||||
rules.addAll(SelectRules.rules());
|
||||
rules.addAll(GroupByRules.rules(plannerConfig));
|
||||
|
||||
// Allow conversion of Druid queries to Bindable convention.
|
||||
rules.add(DruidBindableConverterRule.instance());
|
||||
rules.addAll(GroupByRules.rules(operatorTable, plannerConfig));
|
||||
|
||||
return rules.build();
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package io.druid.sql.calcite.rel;
|
||||
|
||||
import org.apache.calcite.plan.Convention;
|
||||
import org.apache.calcite.plan.ConventionTraitDef;
|
||||
import org.apache.calcite.plan.RelOptPlanner;
|
||||
import org.apache.calcite.plan.RelTrait;
|
||||
import org.apache.calcite.plan.RelTraitDef;
|
||||
|
@ -28,6 +29,7 @@ import org.apache.calcite.plan.RelTraitSet;
|
|||
public class DruidConvention implements Convention
|
||||
{
|
||||
private static final DruidConvention INSTANCE = new DruidConvention();
|
||||
private static final String NAME = "DRUID";
|
||||
|
||||
private DruidConvention()
|
||||
{
|
||||
|
@ -41,13 +43,13 @@ public class DruidConvention implements Convention
|
|||
@Override
|
||||
public Class getInterface()
|
||||
{
|
||||
return null;
|
||||
return DruidRel.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName()
|
||||
{
|
||||
return null;
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -57,9 +59,7 @@ public class DruidConvention implements Convention
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean useAbstractConvertersForConversion(
|
||||
RelTraitSet fromTraits, RelTraitSet toTraits
|
||||
)
|
||||
public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
@ -67,18 +67,23 @@ public class DruidConvention implements Convention
|
|||
@Override
|
||||
public RelTraitDef getTraitDef()
|
||||
{
|
||||
return null;
|
||||
return ConventionTraitDef.INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean satisfies(RelTrait trait)
|
||||
{
|
||||
return false;
|
||||
return trait == this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void register(RelOptPlanner planner)
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return NAME;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,12 +19,12 @@
|
|||
|
||||
package io.druid.sql.calcite.rel;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.QueryDataSource;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.sql.calcite.table.RowSignature;
|
||||
import org.apache.calcite.interpreter.BindableConvention;
|
||||
import org.apache.calcite.interpreter.Row;
|
||||
import org.apache.calcite.plan.RelOptCluster;
|
||||
import org.apache.calcite.plan.RelOptCost;
|
||||
import org.apache.calcite.plan.RelOptPlanner;
|
||||
|
@ -92,16 +92,17 @@ public class DruidNestedGroupBy extends DruidRel<DruidNestedGroupBy>
|
|||
}
|
||||
|
||||
@Override
|
||||
public void accumulate(final Function<Row, Void> sink)
|
||||
public Sequence<Object[]> runQuery()
|
||||
{
|
||||
final QueryDataSource queryDataSource = sourceRel.asDataSource();
|
||||
if (queryDataSource != null) {
|
||||
getQueryMaker().accumulate(
|
||||
return getQueryMaker().runQuery(
|
||||
queryDataSource,
|
||||
sourceRel.getOutputRowSignature(),
|
||||
queryBuilder,
|
||||
sink
|
||||
queryBuilder
|
||||
);
|
||||
} else {
|
||||
return Sequences.empty();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -147,9 +148,14 @@ public class DruidNestedGroupBy extends DruidRel<DruidNestedGroupBy>
|
|||
}
|
||||
|
||||
@Override
|
||||
public Class<Object[]> getElementType()
|
||||
public DruidNestedGroupBy asDruidConvention()
|
||||
{
|
||||
return Object[].class;
|
||||
return new DruidNestedGroupBy(
|
||||
getCluster(),
|
||||
getTraitSet().plus(DruidConvention.instance()),
|
||||
sourceRel,
|
||||
queryBuilder
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -102,6 +102,8 @@ public class DruidQueryBuilder
|
|||
valueType = ValueType.LONG;
|
||||
} else if (SqlTypeName.CHAR_TYPES.contains(sqlTypeName)) {
|
||||
valueType = ValueType.STRING;
|
||||
} else if (SqlTypeName.OTHER == sqlTypeName) {
|
||||
valueType = ValueType.COMPLEX;
|
||||
} else {
|
||||
throw new ISE("Cannot translate sqlTypeName[%s] to Druid type for field[%s]", sqlTypeName, rowOrder.get(i));
|
||||
}
|
||||
|
@ -115,10 +117,7 @@ public class DruidQueryBuilder
|
|||
public static DruidQueryBuilder fullScan(final RowSignature rowSignature, final RelDataTypeFactory relDataTypeFactory)
|
||||
{
|
||||
final RelDataType rowType = rowSignature.getRelDataType(relDataTypeFactory);
|
||||
final List<String> rowOrder = Lists.newArrayListWithCapacity(rowType.getFieldCount());
|
||||
for (RelDataTypeField field : rowType.getFieldList()) {
|
||||
rowOrder.add(field.getName());
|
||||
}
|
||||
final List<String> rowOrder = rowSignature.getRowOrder();
|
||||
return new DruidQueryBuilder(null, null, null, null, null, rowType, rowOrder);
|
||||
}
|
||||
|
||||
|
|
|
@ -19,16 +19,16 @@
|
|||
|
||||
package io.druid.sql.calcite.rel;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.query.QueryDataSource;
|
||||
import io.druid.query.groupby.GroupByQuery;
|
||||
import io.druid.sql.calcite.filtration.Filtration;
|
||||
import io.druid.sql.calcite.table.DruidTable;
|
||||
import io.druid.sql.calcite.table.RowSignature;
|
||||
import org.apache.calcite.interpreter.BindableConvention;
|
||||
import org.apache.calcite.interpreter.Row;
|
||||
import org.apache.calcite.plan.Convention;
|
||||
import org.apache.calcite.plan.RelOptCluster;
|
||||
import org.apache.calcite.plan.RelOptCost;
|
||||
import org.apache.calcite.plan.RelOptPlanner;
|
||||
|
@ -70,14 +70,13 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
|
|||
*/
|
||||
public static DruidQueryRel fullScan(
|
||||
final RelOptCluster cluster,
|
||||
final RelTraitSet traitSet,
|
||||
final RelOptTable table,
|
||||
final DruidTable druidTable
|
||||
)
|
||||
{
|
||||
return new DruidQueryRel(
|
||||
cluster,
|
||||
traitSet,
|
||||
cluster.traitSetOf(Convention.NONE),
|
||||
table,
|
||||
druidTable,
|
||||
DruidQueryBuilder.fullScan(druidTable.getRowSignature(), cluster.getTypeFactory())
|
||||
|
@ -114,6 +113,18 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
|
|||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidQueryRel asDruidConvention()
|
||||
{
|
||||
return new DruidQueryRel(
|
||||
getCluster(),
|
||||
getTraitSet().replace(DruidConvention.instance()),
|
||||
table,
|
||||
druidTable,
|
||||
queryBuilder
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowSignature getSourceRowSignature()
|
||||
{
|
||||
|
@ -145,9 +156,9 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
|
|||
}
|
||||
|
||||
@Override
|
||||
public void accumulate(final Function<Row, Void> sink)
|
||||
public Sequence<Object[]> runQuery()
|
||||
{
|
||||
getQueryMaker().accumulate(druidTable.getDataSource(), druidTable.getRowSignature(), queryBuilder, sink);
|
||||
return getQueryMaker().runQuery(druidTable.getDataSource(), druidTable.getRowSignature(), queryBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -156,12 +167,6 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
|
|||
return table;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<Object[]> getElementType()
|
||||
{
|
||||
return Object[].class;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RelDataType deriveRowType()
|
||||
{
|
||||
|
|
|
@ -19,7 +19,9 @@
|
|||
|
||||
package io.druid.sql.calcite.rel;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Throwables;
|
||||
import io.druid.java.util.common.guava.Accumulator;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.query.QueryDataSource;
|
||||
import io.druid.sql.calcite.table.RowSignature;
|
||||
import org.apache.calcite.DataContext;
|
||||
|
@ -58,7 +60,7 @@ public abstract class DruidRel<T extends DruidRel> extends AbstractRelNode imple
|
|||
*/
|
||||
public abstract int getQueryCount();
|
||||
|
||||
public abstract void accumulate(Function<Row, Void> sink);
|
||||
public abstract Sequence<Object[]> runQuery();
|
||||
|
||||
public abstract T withQueryBuilder(DruidQueryBuilder newQueryBuilder);
|
||||
|
||||
|
@ -80,6 +82,14 @@ public abstract class DruidRel<T extends DruidRel> extends AbstractRelNode imple
|
|||
return queryMaker;
|
||||
}
|
||||
|
||||
public abstract T asDruidConvention();
|
||||
|
||||
@Override
|
||||
public Class<Object[]> getElementType()
|
||||
{
|
||||
return Object[].class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Node implement(InterpreterImplementor implementor)
|
||||
{
|
||||
|
@ -89,7 +99,23 @@ public abstract class DruidRel<T extends DruidRel> extends AbstractRelNode imple
|
|||
@Override
|
||||
public void run() throws InterruptedException
|
||||
{
|
||||
accumulate(QueryMaker.sinkFunction(sink));
|
||||
runQuery().accumulate(
|
||||
sink,
|
||||
new Accumulator<Sink, Object[]>()
|
||||
{
|
||||
@Override
|
||||
public Sink accumulate(final Sink theSink, final Object[] in)
|
||||
{
|
||||
try {
|
||||
theSink.send(Row.of(in));
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
return theSink;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -19,12 +19,14 @@
|
|||
|
||||
package io.druid.sql.calcite.rel;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.Pair;
|
||||
import io.druid.java.util.common.guava.Accumulator;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.QueryDataSource;
|
||||
import io.druid.query.ResourceLimitExceededException;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
|
@ -35,10 +37,7 @@ import io.druid.query.filter.OrDimFilter;
|
|||
import io.druid.sql.calcite.aggregation.Aggregation;
|
||||
import io.druid.sql.calcite.expression.RowExtraction;
|
||||
import io.druid.sql.calcite.table.RowSignature;
|
||||
import org.apache.calcite.DataContext;
|
||||
import org.apache.calcite.interpreter.BindableConvention;
|
||||
import org.apache.calcite.interpreter.Row;
|
||||
import org.apache.calcite.linq4j.Enumerable;
|
||||
import org.apache.calcite.plan.RelOptCluster;
|
||||
import org.apache.calcite.plan.RelOptCost;
|
||||
import org.apache.calcite.plan.RelOptPlanner;
|
||||
|
@ -118,12 +117,6 @@ public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
|
|||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<Object[]> getElementType()
|
||||
{
|
||||
return Object[].class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowSignature getSourceRowSignature()
|
||||
{
|
||||
|
@ -164,7 +157,23 @@ public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
|
|||
{
|
||||
return new DruidSemiJoin(
|
||||
getCluster(),
|
||||
getTraitSet().plus(BindableConvention.INSTANCE),
|
||||
getTraitSet().replace(BindableConvention.INSTANCE),
|
||||
semiJoin,
|
||||
left,
|
||||
right,
|
||||
condition,
|
||||
leftRowExtractions,
|
||||
rightKeys,
|
||||
maxSemiJoinRowsInMemory
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidSemiJoin asDruidConvention()
|
||||
{
|
||||
return new DruidSemiJoin(
|
||||
getCluster(),
|
||||
getTraitSet().replace(DruidConvention.instance()),
|
||||
semiJoin,
|
||||
left,
|
||||
right,
|
||||
|
@ -182,20 +191,16 @@ public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
|
|||
}
|
||||
|
||||
@Override
|
||||
public void accumulate(final Function<Row, Void> sink)
|
||||
public Sequence<Object[]> runQuery()
|
||||
{
|
||||
final DruidRel rel = getLeftRelWithFilter();
|
||||
final DruidRel<?> rel = getLeftRelWithFilter();
|
||||
if (rel != null) {
|
||||
rel.accumulate(sink);
|
||||
return rel.runQuery();
|
||||
} else {
|
||||
return Sequences.empty();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Enumerable<Object[]> bind(final DataContext dataContext)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RelDataType deriveRowType()
|
||||
{
|
||||
|
@ -265,25 +270,26 @@ public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
|
|||
* Returns a copy of the left rel with the filter applied from the right-hand side. This is an expensive operation
|
||||
* since it actually executes the right-hand side query.
|
||||
*/
|
||||
private DruidRel getLeftRelWithFilter()
|
||||
private DruidRel<?> getLeftRelWithFilter()
|
||||
{
|
||||
final Pair<DruidQueryBuilder, List<Integer>> pair = getRightQueryBuilderWithGrouping();
|
||||
final DruidRel rightRelAdjusted = right.withQueryBuilder(pair.lhs);
|
||||
final DruidRel<?> rightRelAdjusted = right.withQueryBuilder(pair.lhs);
|
||||
final List<Integer> rightKeysAdjusted = pair.rhs;
|
||||
|
||||
// Build list of acceptable values from right side.
|
||||
final Set<List<String>> valuess = Sets.newHashSet();
|
||||
final List<DimFilter> filters = Lists.newArrayList();
|
||||
rightRelAdjusted.accumulate(
|
||||
new Function<Row, Void>()
|
||||
rightRelAdjusted.runQuery().accumulate(
|
||||
null,
|
||||
new Accumulator<Object, Object[]>()
|
||||
{
|
||||
@Override
|
||||
public Void apply(final Row row)
|
||||
public Object accumulate(final Object dummyValue, final Object[] row)
|
||||
{
|
||||
final List<String> values = Lists.newArrayListWithCapacity(rightKeysAdjusted.size());
|
||||
|
||||
for (int i : rightKeysAdjusted) {
|
||||
final Object value = row.getObject(i);
|
||||
final Object value = row[i];
|
||||
final String stringValue = value != null ? String.valueOf(value) : "";
|
||||
values.add(stringValue);
|
||||
if (values.size() > maxSemiJoinRowsInMemory) {
|
||||
|
|
|
@ -22,14 +22,14 @@ package io.druid.sql.calcite.rel;
|
|||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Doubles;
|
||||
import com.google.common.primitives.Ints;
|
||||
import io.druid.common.guava.GuavaUtils;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.guava.Accumulator;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.QueryDataSource;
|
||||
import io.druid.query.QuerySegmentWalker;
|
||||
|
@ -48,15 +48,16 @@ import io.druid.query.topn.TopNResultValue;
|
|||
import io.druid.segment.column.Column;
|
||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||
import io.druid.sql.calcite.table.RowSignature;
|
||||
import org.apache.calcite.interpreter.Row;
|
||||
import org.apache.calcite.interpreter.Sink;
|
||||
import org.apache.calcite.avatica.ColumnMetaData;
|
||||
import org.apache.calcite.rel.type.RelDataTypeField;
|
||||
import org.apache.calcite.runtime.Hook;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
import org.apache.calcite.util.NlsString;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Calendar;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
@ -77,29 +78,10 @@ public class QueryMaker
|
|||
this.plannerConfig = plannerConfig;
|
||||
}
|
||||
|
||||
public static Function<Row, Void> sinkFunction(final Sink sink)
|
||||
{
|
||||
return new Function<Row, Void>()
|
||||
{
|
||||
@Override
|
||||
public Void apply(final Row row)
|
||||
{
|
||||
try {
|
||||
sink.send(row);
|
||||
return null;
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public void accumulate(
|
||||
public Sequence<Object[]> runQuery(
|
||||
final DataSource dataSource,
|
||||
final RowSignature sourceRowSignature,
|
||||
final DruidQueryBuilder queryBuilder,
|
||||
final Function<Row, Void> sink
|
||||
final DruidQueryBuilder queryBuilder
|
||||
)
|
||||
{
|
||||
if (dataSource instanceof QueryDataSource) {
|
||||
|
@ -109,14 +91,12 @@ public class QueryMaker
|
|||
throw new IllegalStateException("Can't use QueryDataSource without an outer groupBy query!");
|
||||
}
|
||||
|
||||
executeGroupBy(queryBuilder, outerQuery, sink);
|
||||
return;
|
||||
return executeGroupBy(queryBuilder, outerQuery);
|
||||
}
|
||||
|
||||
final TimeseriesQuery timeseriesQuery = queryBuilder.toTimeseriesQuery(dataSource, sourceRowSignature);
|
||||
if (timeseriesQuery != null) {
|
||||
executeTimeseries(queryBuilder, timeseriesQuery, sink);
|
||||
return;
|
||||
return executeTimeseries(queryBuilder, timeseriesQuery);
|
||||
}
|
||||
|
||||
final TopNQuery topNQuery = queryBuilder.toTopNQuery(
|
||||
|
@ -126,214 +106,250 @@ public class QueryMaker
|
|||
plannerConfig.isUseApproximateTopN()
|
||||
);
|
||||
if (topNQuery != null) {
|
||||
executeTopN(queryBuilder, topNQuery, sink);
|
||||
return;
|
||||
return executeTopN(queryBuilder, topNQuery);
|
||||
}
|
||||
|
||||
final GroupByQuery groupByQuery = queryBuilder.toGroupByQuery(dataSource, sourceRowSignature);
|
||||
if (groupByQuery != null) {
|
||||
executeGroupBy(queryBuilder, groupByQuery, sink);
|
||||
return;
|
||||
return executeGroupBy(queryBuilder, groupByQuery);
|
||||
}
|
||||
|
||||
final SelectQuery selectQuery = queryBuilder.toSelectQuery(dataSource, sourceRowSignature);
|
||||
if (selectQuery != null) {
|
||||
executeSelect(queryBuilder, selectQuery, sink);
|
||||
return;
|
||||
return executeSelect(queryBuilder, selectQuery);
|
||||
}
|
||||
|
||||
throw new IllegalStateException("WTF?! Cannot execute query even though we planned it?");
|
||||
}
|
||||
|
||||
private void executeSelect(
|
||||
private Sequence<Object[]> executeSelect(
|
||||
final DruidQueryBuilder queryBuilder,
|
||||
final SelectQuery baseQuery,
|
||||
final Function<Row, Void> sink
|
||||
final SelectQuery baseQuery
|
||||
)
|
||||
{
|
||||
Preconditions.checkState(queryBuilder.getGrouping() == null, "grouping must be null");
|
||||
|
||||
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
|
||||
final Row.RowBuilder rowBuilder = Row.newBuilder(fieldList.size());
|
||||
final Integer limit = queryBuilder.getLimitSpec() != null ? queryBuilder.getLimitSpec().getLimit() : null;
|
||||
|
||||
// Loop through pages.
|
||||
final AtomicBoolean morePages = new AtomicBoolean(true);
|
||||
final AtomicReference<Map<String, Integer>> pagingIdentifiers = new AtomicReference<>();
|
||||
final AtomicLong rowsRead = new AtomicLong();
|
||||
|
||||
while (morePages.get()) {
|
||||
final SelectQuery query = baseQuery.withPagingSpec(
|
||||
new PagingSpec(
|
||||
pagingIdentifiers.get(),
|
||||
plannerConfig.getSelectThreshold(),
|
||||
true
|
||||
)
|
||||
);
|
||||
|
||||
Hook.QUERY_PLAN.run(query);
|
||||
|
||||
morePages.set(false);
|
||||
final AtomicBoolean gotResult = new AtomicBoolean();
|
||||
|
||||
query.run(walker, Maps.<String, Object>newHashMap()).accumulate(
|
||||
null,
|
||||
new Accumulator<Object, Result<SelectResultValue>>()
|
||||
// Select is paginated, we need to make multiple queries.
|
||||
final Sequence<Sequence<Object[]>> sequenceOfSequences = Sequences.simple(
|
||||
new Iterable<Sequence<Object[]>>()
|
||||
{
|
||||
@Override
|
||||
public Iterator<Sequence<Object[]>> iterator()
|
||||
{
|
||||
@Override
|
||||
public Object accumulate(final Object accumulated, final Result<SelectResultValue> result)
|
||||
final AtomicBoolean morePages = new AtomicBoolean(true);
|
||||
final AtomicReference<Map<String, Integer>> pagingIdentifiers = new AtomicReference<>();
|
||||
final AtomicLong rowsRead = new AtomicLong();
|
||||
|
||||
// Each Sequence<Object[]> is one page.
|
||||
return new Iterator<Sequence<Object[]>>()
|
||||
{
|
||||
if (!gotResult.compareAndSet(false, true)) {
|
||||
throw new ISE("WTF?! Expected single result from Select query but got multiple!");
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
return morePages.get();
|
||||
}
|
||||
|
||||
pagingIdentifiers.set(result.getValue().getPagingIdentifiers());
|
||||
@Override
|
||||
public Sequence<Object[]> next()
|
||||
{
|
||||
final SelectQuery query = baseQuery.withPagingSpec(
|
||||
new PagingSpec(
|
||||
pagingIdentifiers.get(),
|
||||
plannerConfig.getSelectThreshold(),
|
||||
true
|
||||
)
|
||||
);
|
||||
|
||||
for (EventHolder holder : result.getValue().getEvents()) {
|
||||
morePages.set(true);
|
||||
final Map<String, Object> map = holder.getEvent();
|
||||
for (RelDataTypeField field : fieldList) {
|
||||
final String outputName = queryBuilder.getRowOrder().get(field.getIndex());
|
||||
if (outputName.equals(Column.TIME_COLUMN_NAME)) {
|
||||
rowBuilder.set(
|
||||
field.getIndex(),
|
||||
coerce(holder.getTimestamp().getMillis(), field.getType().getSqlTypeName())
|
||||
);
|
||||
} else {
|
||||
rowBuilder.set(
|
||||
field.getIndex(),
|
||||
coerce(map.get(outputName), field.getType().getSqlTypeName())
|
||||
);
|
||||
}
|
||||
}
|
||||
if (limit == null || rowsRead.incrementAndGet() <= limit) {
|
||||
sink.apply(rowBuilder.build());
|
||||
} else {
|
||||
morePages.set(false);
|
||||
break;
|
||||
}
|
||||
rowBuilder.reset();
|
||||
Hook.QUERY_PLAN.run(query);
|
||||
|
||||
morePages.set(false);
|
||||
final AtomicBoolean gotResult = new AtomicBoolean();
|
||||
|
||||
return Sequences.concat(
|
||||
Sequences.map(
|
||||
query.run(walker, Maps.<String, Object>newHashMap()),
|
||||
new Function<Result<SelectResultValue>, Sequence<Object[]>>()
|
||||
{
|
||||
@Override
|
||||
public Sequence<Object[]> apply(final Result<SelectResultValue> result)
|
||||
{
|
||||
if (!gotResult.compareAndSet(false, true)) {
|
||||
throw new ISE("WTF?! Expected single result from Select query but got multiple!");
|
||||
}
|
||||
|
||||
pagingIdentifiers.set(result.getValue().getPagingIdentifiers());
|
||||
final List<Object[]> retVals = new ArrayList<>();
|
||||
|
||||
for (EventHolder holder : result.getValue().getEvents()) {
|
||||
morePages.set(true);
|
||||
final Map<String, Object> map = holder.getEvent();
|
||||
final Object[] retVal = new Object[fieldList.size()];
|
||||
for (RelDataTypeField field : fieldList) {
|
||||
final String outputName = queryBuilder.getRowOrder().get(field.getIndex());
|
||||
if (outputName.equals(Column.TIME_COLUMN_NAME)) {
|
||||
retVal[field.getIndex()] = coerce(
|
||||
holder.getTimestamp().getMillis(),
|
||||
field.getType().getSqlTypeName()
|
||||
);
|
||||
} else {
|
||||
retVal[field.getIndex()] = coerce(
|
||||
map.get(outputName),
|
||||
field.getType().getSqlTypeName()
|
||||
);
|
||||
}
|
||||
}
|
||||
if (limit == null || rowsRead.incrementAndGet() <= limit) {
|
||||
retVals.add(retVal);
|
||||
} else {
|
||||
morePages.set(false);
|
||||
return Sequences.simple(retVals);
|
||||
}
|
||||
}
|
||||
|
||||
return Sequences.simple(retVals);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
@Override
|
||||
public void remove()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return Sequences.concat(sequenceOfSequences);
|
||||
}
|
||||
|
||||
private void executeTimeseries(
|
||||
private Sequence<Object[]> executeTimeseries(
|
||||
final DruidQueryBuilder queryBuilder,
|
||||
final TimeseriesQuery query,
|
||||
final Function<Row, Void> sink
|
||||
final TimeseriesQuery query
|
||||
)
|
||||
{
|
||||
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
|
||||
final List<DimensionSpec> dimensions = queryBuilder.getGrouping().getDimensions();
|
||||
final String timeOutputName = dimensions.isEmpty() ? null : Iterables.getOnlyElement(dimensions).getOutputName();
|
||||
final Row.RowBuilder rowBuilder = Row.newBuilder(fieldList.size());
|
||||
|
||||
Hook.QUERY_PLAN.run(query);
|
||||
|
||||
query.run(walker, Maps.<String, Object>newHashMap()).accumulate(
|
||||
null,
|
||||
new Accumulator<Object, Result<TimeseriesResultValue>>()
|
||||
return Sequences.map(
|
||||
query.run(walker, Maps.<String, Object>newHashMap()),
|
||||
new Function<Result<TimeseriesResultValue>, Object[]>()
|
||||
{
|
||||
@Override
|
||||
public Object accumulate(final Object accumulated, final Result<TimeseriesResultValue> result)
|
||||
public Object[] apply(final Result<TimeseriesResultValue> result)
|
||||
{
|
||||
final Map<String, Object> row = result.getValue().getBaseObject();
|
||||
final Object[] retVal = new Object[fieldList.size()];
|
||||
|
||||
for (final RelDataTypeField field : fieldList) {
|
||||
final String outputName = queryBuilder.getRowOrder().get(field.getIndex());
|
||||
if (outputName.equals(timeOutputName)) {
|
||||
rowBuilder.set(field.getIndex(), coerce(result.getTimestamp(), field.getType().getSqlTypeName()));
|
||||
retVal[field.getIndex()] = coerce(result.getTimestamp(), field.getType().getSqlTypeName());
|
||||
} else {
|
||||
rowBuilder.set(field.getIndex(), coerce(row.get(outputName), field.getType().getSqlTypeName()));
|
||||
retVal[field.getIndex()] = coerce(row.get(outputName), field.getType().getSqlTypeName());
|
||||
}
|
||||
}
|
||||
|
||||
sink.apply(rowBuilder.build());
|
||||
rowBuilder.reset();
|
||||
|
||||
return null;
|
||||
return retVal;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private void executeTopN(
|
||||
private Sequence<Object[]> executeTopN(
|
||||
final DruidQueryBuilder queryBuilder,
|
||||
final TopNQuery query,
|
||||
final Function<Row, Void> sink
|
||||
final TopNQuery query
|
||||
)
|
||||
{
|
||||
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
|
||||
final Row.RowBuilder rowBuilder = Row.newBuilder(fieldList.size());
|
||||
|
||||
Hook.QUERY_PLAN.run(query);
|
||||
|
||||
query.run(walker, Maps.<String, Object>newHashMap()).accumulate(
|
||||
null,
|
||||
new Accumulator<Object, Result<TopNResultValue>>()
|
||||
{
|
||||
@Override
|
||||
public Object accumulate(final Object accumulated, final Result<TopNResultValue> result)
|
||||
{
|
||||
final List<DimensionAndMetricValueExtractor> values = result.getValue().getValue();
|
||||
return Sequences.concat(
|
||||
Sequences.map(
|
||||
query.run(walker, Maps.<String, Object>newHashMap()),
|
||||
new Function<Result<TopNResultValue>, Sequence<Object[]>>()
|
||||
{
|
||||
@Override
|
||||
public Sequence<Object[]> apply(final Result<TopNResultValue> result)
|
||||
{
|
||||
final List<DimensionAndMetricValueExtractor> rows = result.getValue().getValue();
|
||||
final List<Object[]> retVals = new ArrayList<>(rows.size());
|
||||
|
||||
for (DimensionAndMetricValueExtractor value : values) {
|
||||
for (final RelDataTypeField field : fieldList) {
|
||||
final String outputName = queryBuilder.getRowOrder().get(field.getIndex());
|
||||
rowBuilder.set(field.getIndex(), coerce(value.getMetric(outputName), field.getType().getSqlTypeName()));
|
||||
for (DimensionAndMetricValueExtractor row : rows) {
|
||||
final Object[] retVal = new Object[fieldList.size()];
|
||||
for (final RelDataTypeField field : fieldList) {
|
||||
final String outputName = queryBuilder.getRowOrder().get(field.getIndex());
|
||||
retVal[field.getIndex()] = coerce(row.getMetric(outputName), field.getType().getSqlTypeName());
|
||||
}
|
||||
|
||||
retVals.add(retVal);
|
||||
}
|
||||
|
||||
return Sequences.simple(retVals);
|
||||
}
|
||||
|
||||
sink.apply(rowBuilder.build());
|
||||
rowBuilder.reset();
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private void executeGroupBy(
|
||||
private Sequence<Object[]> executeGroupBy(
|
||||
final DruidQueryBuilder queryBuilder,
|
||||
final GroupByQuery query,
|
||||
final Function<Row, Void> sink
|
||||
final GroupByQuery query
|
||||
)
|
||||
{
|
||||
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
|
||||
final Row.RowBuilder rowBuilder = Row.newBuilder(fieldList.size());
|
||||
|
||||
Hook.QUERY_PLAN.run(query);
|
||||
|
||||
query.run(walker, Maps.<String, Object>newHashMap()).accumulate(
|
||||
null,
|
||||
new Accumulator<Object, io.druid.data.input.Row>()
|
||||
return Sequences.map(
|
||||
query.run(walker, Maps.<String, Object>newHashMap()),
|
||||
new Function<io.druid.data.input.Row, Object[]>()
|
||||
{
|
||||
@Override
|
||||
public Object accumulate(final Object accumulated, final io.druid.data.input.Row row)
|
||||
public Object[] apply(final io.druid.data.input.Row row)
|
||||
{
|
||||
final Object[] retVal = new Object[fieldList.size()];
|
||||
for (RelDataTypeField field : fieldList) {
|
||||
rowBuilder.set(
|
||||
field.getIndex(),
|
||||
coerce(
|
||||
row.getRaw(queryBuilder.getRowOrder().get(field.getIndex())),
|
||||
field.getType().getSqlTypeName()
|
||||
)
|
||||
retVal[field.getIndex()] = coerce(
|
||||
row.getRaw(queryBuilder.getRowOrder().get(field.getIndex())),
|
||||
field.getType().getSqlTypeName()
|
||||
);
|
||||
}
|
||||
sink.apply(rowBuilder.build());
|
||||
rowBuilder.reset();
|
||||
|
||||
return null;
|
||||
return retVal;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
public static ColumnMetaData.Rep rep(final SqlTypeName sqlType)
|
||||
{
|
||||
if (SqlTypeName.CHAR_TYPES.contains(sqlType)) {
|
||||
return ColumnMetaData.Rep.of(String.class);
|
||||
} else if (SqlTypeName.DATETIME_TYPES.contains(sqlType)) {
|
||||
return ColumnMetaData.Rep.of(Long.class);
|
||||
} else if (sqlType == SqlTypeName.INTEGER) {
|
||||
return ColumnMetaData.Rep.of(Integer.class);
|
||||
} else if (sqlType == SqlTypeName.BIGINT) {
|
||||
return ColumnMetaData.Rep.of(Long.class);
|
||||
} else if (sqlType == SqlTypeName.FLOAT || sqlType == SqlTypeName.DOUBLE) {
|
||||
return ColumnMetaData.Rep.of(Double.class);
|
||||
} else if (sqlType == SqlTypeName.OTHER) {
|
||||
return ColumnMetaData.Rep.of(Object.class);
|
||||
} else {
|
||||
throw new ISE("No rep for SQL type[%s]", sqlType);
|
||||
}
|
||||
}
|
||||
|
||||
private static Object coerce(final Object value, final SqlTypeName sqlType)
|
||||
{
|
||||
final Object coercedValue;
|
||||
|
@ -391,6 +407,9 @@ public class QueryMaker
|
|||
} else {
|
||||
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
|
||||
}
|
||||
} else if (sqlType == SqlTypeName.OTHER) {
|
||||
// Complex type got out somehow.
|
||||
coercedValue = value.getClass().getName();
|
||||
} else {
|
||||
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
|
||||
}
|
||||
|
|
|
@ -25,21 +25,21 @@ import org.apache.calcite.plan.Convention;
|
|||
import org.apache.calcite.rel.RelNode;
|
||||
import org.apache.calcite.rel.convert.ConverterRule;
|
||||
|
||||
public class DruidBindableConverterRule extends ConverterRule
|
||||
public class DruidRelToBindableRule extends ConverterRule
|
||||
{
|
||||
private static DruidBindableConverterRule INSTANCE = new DruidBindableConverterRule();
|
||||
private static DruidRelToBindableRule INSTANCE = new DruidRelToBindableRule();
|
||||
|
||||
private DruidBindableConverterRule()
|
||||
private DruidRelToBindableRule()
|
||||
{
|
||||
super(
|
||||
DruidRel.class,
|
||||
Convention.NONE,
|
||||
BindableConvention.INSTANCE,
|
||||
DruidBindableConverterRule.class.getSimpleName()
|
||||
DruidRelToBindableRule.class.getSimpleName()
|
||||
);
|
||||
}
|
||||
|
||||
public static DruidBindableConverterRule instance()
|
||||
public static DruidRelToBindableRule instance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
|
@ -47,7 +47,6 @@ public class DruidBindableConverterRule extends ConverterRule
|
|||
@Override
|
||||
public RelNode convert(RelNode rel)
|
||||
{
|
||||
final DruidRel druidRel = (DruidRel) rel;
|
||||
return druidRel.asBindable();
|
||||
return ((DruidRel) rel).asBindable();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* 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.sql.calcite.rule;
|
||||
|
||||
import io.druid.sql.calcite.rel.DruidConvention;
|
||||
import io.druid.sql.calcite.rel.DruidRel;
|
||||
import org.apache.calcite.plan.Convention;
|
||||
import org.apache.calcite.rel.RelNode;
|
||||
import org.apache.calcite.rel.convert.ConverterRule;
|
||||
|
||||
public class DruidRelToDruidRule extends ConverterRule
|
||||
{
|
||||
private static DruidRelToDruidRule INSTANCE = new DruidRelToDruidRule();
|
||||
|
||||
private DruidRelToDruidRule()
|
||||
{
|
||||
super(
|
||||
DruidRel.class,
|
||||
Convention.NONE,
|
||||
DruidConvention.instance(),
|
||||
DruidRelToDruidRule.class.getSimpleName()
|
||||
);
|
||||
}
|
||||
|
||||
public static DruidRelToDruidRule instance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RelNode convert(RelNode rel)
|
||||
{
|
||||
return ((DruidRel) rel).asDruidConvention();
|
||||
}
|
||||
}
|
|
@ -34,8 +34,6 @@ import io.druid.query.aggregation.LongMaxAggregatorFactory;
|
|||
import io.druid.query.aggregation.LongMinAggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator;
|
||||
import io.druid.query.aggregation.post.ArithmeticPostAggregator;
|
||||
import io.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
|
@ -47,10 +45,13 @@ import io.druid.query.groupby.orderby.OrderByColumnSpec;
|
|||
import io.druid.query.ordering.StringComparator;
|
||||
import io.druid.query.ordering.StringComparators;
|
||||
import io.druid.sql.calcite.aggregation.Aggregation;
|
||||
import io.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator;
|
||||
import io.druid.sql.calcite.aggregation.PostAggregatorFactory;
|
||||
import io.druid.sql.calcite.aggregation.SqlAggregator;
|
||||
import io.druid.sql.calcite.expression.Expressions;
|
||||
import io.druid.sql.calcite.expression.RowExtraction;
|
||||
import io.druid.sql.calcite.filtration.Filtration;
|
||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||
import io.druid.sql.calcite.rel.DruidNestedGroupBy;
|
||||
import io.druid.sql.calcite.rel.DruidRel;
|
||||
|
@ -77,17 +78,19 @@ import java.util.Map;
|
|||
|
||||
public class GroupByRules
|
||||
{
|
||||
private static final ApproxCountDistinctSqlAggregator APPROX_COUNT_DISTINCT = new ApproxCountDistinctSqlAggregator();
|
||||
|
||||
private GroupByRules()
|
||||
{
|
||||
// No instantiation.
|
||||
}
|
||||
|
||||
public static List<RelOptRule> rules(final PlannerConfig plannerConfig)
|
||||
public static List<RelOptRule> rules(final DruidOperatorTable operatorTable, final PlannerConfig plannerConfig)
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new DruidAggregateRule(plannerConfig),
|
||||
new DruidAggregateProjectRule(plannerConfig),
|
||||
new DruidAggregateProjectFilterRule(plannerConfig),
|
||||
new DruidAggregateRule(operatorTable, plannerConfig),
|
||||
new DruidAggregateProjectRule(operatorTable, plannerConfig),
|
||||
new DruidAggregateProjectFilterRule(operatorTable, plannerConfig),
|
||||
new DruidGroupByPostAggregationRule(),
|
||||
new DruidGroupByHavingRule(),
|
||||
new DruidGroupByLimitRule()
|
||||
|
@ -150,11 +153,13 @@ public class GroupByRules
|
|||
|
||||
public static class DruidAggregateRule extends RelOptRule
|
||||
{
|
||||
private final DruidOperatorTable operatorTable;
|
||||
private final PlannerConfig plannerConfig;
|
||||
|
||||
private DruidAggregateRule(final PlannerConfig plannerConfig)
|
||||
private DruidAggregateRule(final DruidOperatorTable operatorTable, final PlannerConfig plannerConfig)
|
||||
{
|
||||
super(operand(Aggregate.class, operand(DruidRel.class, none())));
|
||||
this.operatorTable = operatorTable;
|
||||
this.plannerConfig = plannerConfig;
|
||||
}
|
||||
|
||||
|
@ -176,6 +181,7 @@ public class GroupByRules
|
|||
null,
|
||||
null,
|
||||
aggregate,
|
||||
operatorTable,
|
||||
plannerConfig.isUseApproximateCountDistinct(),
|
||||
plannerConfig.getMaxQueryCount()
|
||||
);
|
||||
|
@ -187,11 +193,13 @@ public class GroupByRules
|
|||
|
||||
public static class DruidAggregateProjectRule extends RelOptRule
|
||||
{
|
||||
private final DruidOperatorTable operatorTable;
|
||||
private final PlannerConfig plannerConfig;
|
||||
|
||||
private DruidAggregateProjectRule(final PlannerConfig plannerConfig)
|
||||
private DruidAggregateProjectRule(final DruidOperatorTable operatorTable, final PlannerConfig plannerConfig)
|
||||
{
|
||||
super(operand(Aggregate.class, operand(Project.class, operand(DruidRel.class, none()))));
|
||||
this.operatorTable = operatorTable;
|
||||
this.plannerConfig = plannerConfig;
|
||||
}
|
||||
|
||||
|
@ -215,6 +223,7 @@ public class GroupByRules
|
|||
null,
|
||||
project,
|
||||
aggregate,
|
||||
operatorTable,
|
||||
plannerConfig.isUseApproximateCountDistinct(),
|
||||
plannerConfig.getMaxQueryCount()
|
||||
);
|
||||
|
@ -226,11 +235,13 @@ public class GroupByRules
|
|||
|
||||
public static class DruidAggregateProjectFilterRule extends RelOptRule
|
||||
{
|
||||
private final DruidOperatorTable operatorTable;
|
||||
private final PlannerConfig plannerConfig;
|
||||
|
||||
private DruidAggregateProjectFilterRule(final PlannerConfig plannerConfig)
|
||||
private DruidAggregateProjectFilterRule(final DruidOperatorTable operatorTable, final PlannerConfig plannerConfig)
|
||||
{
|
||||
super(operand(Aggregate.class, operand(Project.class, operand(Filter.class, operand(DruidRel.class, none())))));
|
||||
this.operatorTable = operatorTable;
|
||||
this.plannerConfig = plannerConfig;
|
||||
}
|
||||
|
||||
|
@ -256,6 +267,7 @@ public class GroupByRules
|
|||
filter,
|
||||
project,
|
||||
aggregate,
|
||||
operatorTable,
|
||||
plannerConfig.isUseApproximateCountDistinct(),
|
||||
plannerConfig.getMaxQueryCount()
|
||||
);
|
||||
|
@ -367,6 +379,7 @@ public class GroupByRules
|
|||
final Filter filter0,
|
||||
final Project project0,
|
||||
final Aggregate aggregate,
|
||||
final DruidOperatorTable operatorTable,
|
||||
final boolean approximateCountDistinct,
|
||||
final int maxQueryCount
|
||||
)
|
||||
|
@ -451,6 +464,8 @@ public class GroupByRules
|
|||
sourceRowSignature,
|
||||
project,
|
||||
aggCall,
|
||||
operatorTable,
|
||||
aggregations,
|
||||
i,
|
||||
approximateCountDistinct
|
||||
);
|
||||
|
@ -713,6 +728,8 @@ public class GroupByRules
|
|||
final RowSignature sourceRowSignature,
|
||||
final Project project,
|
||||
final AggregateCall call,
|
||||
final DruidOperatorTable operatorTable,
|
||||
final List<Aggregation> existingAggregations,
|
||||
final int aggNumber,
|
||||
final boolean approximateCountDistinct
|
||||
)
|
||||
|
@ -740,47 +757,25 @@ public class GroupByRules
|
|||
filters.add(filter);
|
||||
}
|
||||
|
||||
if (call.getAggregation().getKind() == SqlKind.COUNT && call.getArgList().isEmpty()) {
|
||||
if (kind == SqlKind.COUNT && call.getArgList().isEmpty()) {
|
||||
// COUNT(*)
|
||||
retVal = Aggregation.create(new CountAggregatorFactory(name));
|
||||
} else if (call.getAggregation().getKind() == SqlKind.COUNT && call.isDistinct() && approximateCountDistinct) {
|
||||
} else if (kind == SqlKind.COUNT && call.isDistinct()) {
|
||||
// COUNT(DISTINCT x)
|
||||
final RowExtraction rex = Expressions.toRowExtraction(
|
||||
rowOrder,
|
||||
Expressions.fromFieldAccess(
|
||||
sourceRowSignature,
|
||||
project,
|
||||
Iterables.getOnlyElement(call.getArgList())
|
||||
)
|
||||
);
|
||||
if (rex == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final DimensionSpec dimensionSpec = rex.toDimensionSpec(
|
||||
retVal = approximateCountDistinct ? APPROX_COUNT_DISTINCT.toDruidAggregation(
|
||||
name,
|
||||
sourceRowSignature,
|
||||
aggInternalName(aggNumber, "dimSpec")
|
||||
);
|
||||
if (dimensionSpec == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
retVal = Aggregation.createFinalizable(
|
||||
ImmutableList.<AggregatorFactory>of(
|
||||
new CardinalityAggregatorFactory(name, ImmutableList.of(dimensionSpec), false)
|
||||
),
|
||||
null,
|
||||
new PostAggregatorFactory()
|
||||
{
|
||||
@Override
|
||||
public PostAggregator factorize(String outputName)
|
||||
{
|
||||
return new HyperUniqueFinalizingPostAggregator(outputName, name);
|
||||
}
|
||||
}
|
||||
);
|
||||
} else if (!call.isDistinct() && call.getArgList().size() == 1) {
|
||||
// AGG(xxx), not distinct, not COUNT(*)
|
||||
existingAggregations,
|
||||
project,
|
||||
call
|
||||
) : null;
|
||||
} else if (kind == SqlKind.COUNT
|
||||
|| kind == SqlKind.SUM
|
||||
|| kind == SqlKind.SUM0
|
||||
|| kind == SqlKind.MIN
|
||||
|| kind == SqlKind.MAX
|
||||
|| kind == SqlKind.AVG) {
|
||||
// Built-in agg, not distinct, not COUNT(*)
|
||||
boolean forceCount = false;
|
||||
final FieldOrExpression input;
|
||||
|
||||
|
@ -840,7 +835,7 @@ public class GroupByRules
|
|||
return null;
|
||||
}
|
||||
} else {
|
||||
// Can't translate aggregator expression.
|
||||
// Can't translate operand.
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -852,7 +847,7 @@ public class GroupByRules
|
|||
// COUNT(x)
|
||||
retVal = Aggregation.create(new CountAggregatorFactory(name));
|
||||
} else {
|
||||
// All aggregators other than COUNT expect a single argument with no extractionFn.
|
||||
// Built-in aggregator that is not COUNT.
|
||||
final String fieldName = input.getFieldName();
|
||||
final String expression = input.getExpression();
|
||||
|
||||
|
@ -890,11 +885,20 @@ public class GroupByRules
|
|||
)
|
||||
);
|
||||
} else {
|
||||
retVal = null;
|
||||
// Not reached.
|
||||
throw new ISE("WTF?! Kind[%s] got into the built-in aggregator path somehow?!", kind);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
retVal = null;
|
||||
// Not a built-in aggregator, check operator table.
|
||||
final SqlAggregator sqlAggregator = operatorTable.lookupAggregator(call.getAggregation().getName());
|
||||
retVal = sqlAggregator != null ? sqlAggregator.toDruidAggregation(
|
||||
name,
|
||||
sourceRowSignature,
|
||||
existingAggregations,
|
||||
project,
|
||||
call
|
||||
) : null;
|
||||
}
|
||||
|
||||
final DimFilter filter = filters.isEmpty()
|
||||
|
|
|
@ -17,17 +17,15 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.sql.calcite;
|
||||
package io.druid.sql.calcite.schema;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableMultimap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Multimap;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import com.google.inject.Inject;
|
||||
|
@ -53,10 +51,6 @@ import io.druid.sql.calcite.planner.PlannerConfig;
|
|||
import io.druid.sql.calcite.rel.QueryMaker;
|
||||
import io.druid.sql.calcite.table.DruidTable;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.apache.calcite.linq4j.tree.Expression;
|
||||
import org.apache.calcite.schema.Function;
|
||||
import org.apache.calcite.schema.Schema;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.calcite.schema.Table;
|
||||
import org.apache.calcite.schema.impl.AbstractSchema;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -72,6 +66,8 @@ import java.util.concurrent.ExecutorService;
|
|||
@ManageLifecycle
|
||||
public class DruidSchema extends AbstractSchema
|
||||
{
|
||||
public static final String NAME = "druid";
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(DruidSchema.class);
|
||||
|
||||
private final QuerySegmentWalker walker;
|
||||
|
@ -274,42 +270,12 @@ public class DruidSchema extends AbstractSchema
|
|||
initializationLatch.await();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isMutable()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean contentsHaveChangedSince(final long lastCheck, final long now)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Expression getExpression(final SchemaPlus parentSchema, final String name)
|
||||
{
|
||||
return super.getExpression(parentSchema, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, Table> getTableMap()
|
||||
{
|
||||
return ImmutableMap.copyOf(tables);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Multimap<String, Function> getFunctionMultimap()
|
||||
{
|
||||
return ImmutableMultimap.of();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, Schema> getSubSchemaMap()
|
||||
{
|
||||
return ImmutableMap.of();
|
||||
}
|
||||
|
||||
private DruidTable computeTable(final String dataSource)
|
||||
{
|
||||
final SegmentMetadataQuery segmentMetadataQuery = new SegmentMetadataQuery(
|
||||
|
@ -338,14 +304,14 @@ public class DruidSchema extends AbstractSchema
|
|||
continue;
|
||||
}
|
||||
|
||||
final ValueType valueType;
|
||||
ValueType valueType;
|
||||
try {
|
||||
valueType = ValueType.valueOf(entry.getValue().getType().toUpperCase());
|
||||
}
|
||||
catch (IllegalArgumentException e) {
|
||||
// Ignore unrecognized types. This includes complex types like hyperUnique, etc.
|
||||
// So, that means currently they are not supported.
|
||||
continue;
|
||||
// Assume unrecognized types are some flavor of COMPLEX. This throws away information about exactly
|
||||
// what kind of complex column it is, which we may want to preserve some day.
|
||||
valueType = ValueType.COMPLEX;
|
||||
}
|
||||
|
||||
columnValueTypes.put(entry.getKey(), valueType);
|
|
@ -0,0 +1,303 @@
|
|||
/*
|
||||
* 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.sql.calcite.schema;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.FluentIterable;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.inject.Inject;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.sql.calcite.table.RowSignature;
|
||||
import org.apache.calcite.DataContext;
|
||||
import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
|
||||
import org.apache.calcite.linq4j.Enumerable;
|
||||
import org.apache.calcite.linq4j.Linq4j;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.calcite.rel.type.RelDataTypeFactory;
|
||||
import org.apache.calcite.rel.type.RelDataTypeField;
|
||||
import org.apache.calcite.rel.type.RelDataTypeSystem;
|
||||
import org.apache.calcite.schema.ScannableTable;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.calcite.schema.Statistic;
|
||||
import org.apache.calcite.schema.Statistics;
|
||||
import org.apache.calcite.schema.Table;
|
||||
import org.apache.calcite.schema.impl.AbstractSchema;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class InformationSchema extends AbstractSchema
|
||||
{
|
||||
public static final String NAME = "INFORMATION_SCHEMA";
|
||||
|
||||
private static final String SCHEMATA_TABLE = "SCHEMATA";
|
||||
private static final String TABLES_TABLE = "TABLES";
|
||||
private static final String COLUMNS_TABLE = "COLUMNS";
|
||||
private static final RowSignature SCHEMATA_SIGNATURE = RowSignature
|
||||
.builder()
|
||||
.add("CATALOG_NAME", ValueType.STRING)
|
||||
.add("SCHEMA_NAME", ValueType.STRING)
|
||||
.add("SCHEMA_OWNER", ValueType.STRING)
|
||||
.add("DEFAULT_CHARACTER_SET_CATALOG", ValueType.STRING)
|
||||
.add("DEFAULT_CHARACTER_SET_SCHEMA", ValueType.STRING)
|
||||
.add("DEFAULT_CHARACTER_SET_NAME", ValueType.STRING)
|
||||
.add("SQL_PATH", ValueType.STRING)
|
||||
.build();
|
||||
private static final RowSignature TABLES_SIGNATURE = RowSignature
|
||||
.builder()
|
||||
.add("TABLE_CATALOG", ValueType.STRING)
|
||||
.add("TABLE_SCHEMA", ValueType.STRING)
|
||||
.add("TABLE_NAME", ValueType.STRING)
|
||||
.add("TABLE_TYPE", ValueType.STRING)
|
||||
.build();
|
||||
private static final RowSignature COLUMNS_SIGNATURE = RowSignature
|
||||
.builder()
|
||||
.add("TABLE_CATALOG", ValueType.STRING)
|
||||
.add("TABLE_SCHEMA", ValueType.STRING)
|
||||
.add("TABLE_NAME", ValueType.STRING)
|
||||
.add("COLUMN_NAME", ValueType.STRING)
|
||||
.add("ORDINAL_POSITION", ValueType.STRING)
|
||||
.add("COLUMN_DEFAULT", ValueType.STRING)
|
||||
.add("IS_NULLABLE", ValueType.STRING)
|
||||
.add("DATA_TYPE", ValueType.STRING)
|
||||
.add("CHARACTER_MAXIMUM_LENGTH", ValueType.STRING)
|
||||
.add("CHARACTER_OCTET_LENGTH", ValueType.STRING)
|
||||
.add("NUMERIC_PRECISION", ValueType.STRING)
|
||||
.add("NUMERIC_PRECISION_RADIX", ValueType.STRING)
|
||||
.add("NUMERIC_SCALE", ValueType.STRING)
|
||||
.add("DATETIME_PRECISION", ValueType.STRING)
|
||||
.add("CHARACTER_SET_NAME", ValueType.STRING)
|
||||
.add("COLLATION_NAME", ValueType.STRING)
|
||||
.add("JDBC_TYPE", ValueType.LONG)
|
||||
.build();
|
||||
private static final RelDataTypeSystem TYPE_SYSTEM = RelDataTypeSystem.DEFAULT;
|
||||
|
||||
private final SchemaPlus rootSchema;
|
||||
private final Map<String, Table> tableMap;
|
||||
|
||||
@Inject
|
||||
public InformationSchema(final SchemaPlus rootSchema)
|
||||
{
|
||||
this.rootSchema = Preconditions.checkNotNull(rootSchema, "rootSchema");
|
||||
this.tableMap = ImmutableMap.<String, Table>of(
|
||||
SCHEMATA_TABLE, new SchemataTable(),
|
||||
TABLES_TABLE, new TablesTable(),
|
||||
COLUMNS_TABLE, new ColumnsTable()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, Table> getTableMap()
|
||||
{
|
||||
return tableMap;
|
||||
}
|
||||
|
||||
class SchemataTable implements ScannableTable
|
||||
{
|
||||
@Override
|
||||
public Enumerable<Object[]> scan(final DataContext root)
|
||||
{
|
||||
final FluentIterable<Object[]> results = FluentIterable
|
||||
.from(rootSchema.getSubSchemaNames())
|
||||
.transform(
|
||||
new Function<String, Object[]>()
|
||||
{
|
||||
@Override
|
||||
public Object[] apply(final String schemaName)
|
||||
{
|
||||
final SchemaPlus subSchema = rootSchema.getSubSchema(schemaName);
|
||||
return new Object[]{
|
||||
"", // CATALOG_NAME
|
||||
subSchema.getName(), // SCHEMA_NAME
|
||||
null, // SCHEMA_OWNER
|
||||
null, // DEFAULT_CHARACTER_SET_CATALOG
|
||||
null, // DEFAULT_CHARACTER_SET_SCHEMA
|
||||
null, // DEFAULT_CHARACTER_SET_NAME
|
||||
null // SQL_PATH
|
||||
};
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return Linq4j.asEnumerable(results);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RelDataType getRowType(final RelDataTypeFactory typeFactory)
|
||||
{
|
||||
return SCHEMATA_SIGNATURE.getRelDataType(typeFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Statistic getStatistic()
|
||||
{
|
||||
return Statistics.UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableType getJdbcTableType()
|
||||
{
|
||||
return TableType.SYSTEM_TABLE;
|
||||
}
|
||||
}
|
||||
|
||||
class TablesTable implements ScannableTable
|
||||
{
|
||||
@Override
|
||||
public Enumerable<Object[]> scan(final DataContext root)
|
||||
{
|
||||
final FluentIterable<Object[]> results = FluentIterable
|
||||
.from(rootSchema.getSubSchemaNames())
|
||||
.transformAndConcat(
|
||||
new Function<String, Iterable<Object[]>>()
|
||||
{
|
||||
@Override
|
||||
public Iterable<Object[]> apply(final String schemaName)
|
||||
{
|
||||
final SchemaPlus subSchema = rootSchema.getSubSchema(schemaName);
|
||||
final Set<String> tableNames = subSchema.getTableNames();
|
||||
return FluentIterable.from(tableNames).transform(
|
||||
new Function<String, Object[]>()
|
||||
{
|
||||
@Override
|
||||
public Object[] apply(final String tableName)
|
||||
{
|
||||
return new Object[]{
|
||||
null, // TABLE_CATALOG
|
||||
schemaName, // TABLE_SCHEMA
|
||||
tableName, // TABLE_NAME
|
||||
subSchema.getTable(tableName).getJdbcTableType().toString() // TABLE_TYPE
|
||||
};
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return Linq4j.asEnumerable(results);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RelDataType getRowType(final RelDataTypeFactory typeFactory)
|
||||
{
|
||||
return TABLES_SIGNATURE.getRelDataType(typeFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Statistic getStatistic()
|
||||
{
|
||||
return Statistics.UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableType getJdbcTableType()
|
||||
{
|
||||
return TableType.SYSTEM_TABLE;
|
||||
}
|
||||
}
|
||||
|
||||
class ColumnsTable implements ScannableTable
|
||||
{
|
||||
@Override
|
||||
public Enumerable<Object[]> scan(final DataContext root)
|
||||
{
|
||||
final FluentIterable<Object[]> results = FluentIterable
|
||||
.from(rootSchema.getSubSchemaNames())
|
||||
.transformAndConcat(
|
||||
new Function<String, Iterable<Object[]>>()
|
||||
{
|
||||
@Override
|
||||
public Iterable<Object[]> apply(final String schemaName)
|
||||
{
|
||||
final SchemaPlus subSchema = rootSchema.getSubSchema(schemaName);
|
||||
final Set<String> tableNames = subSchema.getTableNames();
|
||||
final JavaTypeFactoryImpl typeFactory = new JavaTypeFactoryImpl(TYPE_SYSTEM);
|
||||
return FluentIterable.from(tableNames).transformAndConcat(
|
||||
new Function<String, Iterable<Object[]>>()
|
||||
{
|
||||
@Override
|
||||
public Iterable<Object[]> apply(final String tableName)
|
||||
{
|
||||
return FluentIterable
|
||||
.from(subSchema.getTable(tableName).getRowType(typeFactory).getFieldList())
|
||||
.transform(
|
||||
new Function<RelDataTypeField, Object[]>()
|
||||
{
|
||||
@Override
|
||||
public Object[] apply(final RelDataTypeField field)
|
||||
{
|
||||
final RelDataType type = field.getType();
|
||||
boolean isNumeric = SqlTypeName.NUMERIC_TYPES.contains(type.getSqlTypeName());
|
||||
boolean isCharacter = SqlTypeName.CHAR_TYPES.contains(type.getSqlTypeName());
|
||||
boolean isDateTime = SqlTypeName.DATETIME_TYPES.contains(type.getSqlTypeName());
|
||||
return new Object[]{
|
||||
"", // TABLE_CATALOG
|
||||
schemaName, // TABLE_SCHEMA
|
||||
tableName, // TABLE_NAME
|
||||
field.getName(), // COLUMN_NAME
|
||||
String.valueOf(field.getIndex()), // ORDINAL_POSITION
|
||||
"", // COLUMN_DEFAULT
|
||||
type.isNullable() ? "YES" : "NO", // IS_NULLABLE
|
||||
type.getSqlTypeName().toString(), // DATA_TYPE
|
||||
null, // CHARACTER_MAXIMUM_LENGTH
|
||||
null, // CHARACTER_OCTET_LENGTH
|
||||
isNumeric ? String.valueOf(type.getPrecision()) : null, // NUMERIC_PRECISION
|
||||
isNumeric ? "10" : null, // NUMERIC_PRECISION_RADIX
|
||||
isNumeric ? String.valueOf(type.getScale()) : null, // NUMERIC_SCALE
|
||||
isDateTime ? String.valueOf(type.getPrecision()) : null, // DATETIME_PRECISION
|
||||
isCharacter ? type.getCharset().name() : null, // CHARACTER_SET_NAME
|
||||
isCharacter ? type.getCollation().getCollationName() : null, // COLLATION_NAME
|
||||
type.getSqlTypeName().getJdbcOrdinal() // JDBC_TYPE (Druid extension)
|
||||
};
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return Linq4j.asEnumerable(results);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RelDataType getRowType(final RelDataTypeFactory typeFactory)
|
||||
{
|
||||
return COLUMNS_SIGNATURE.getRelDataType(typeFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Statistic getStatistic()
|
||||
{
|
||||
return Statistics.UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableType getJdbcTableType()
|
||||
{
|
||||
return TableType.SYSTEM_TABLE;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -96,12 +96,7 @@ public class DruidTable implements TranslatableTable
|
|||
public RelNode toRel(final RelOptTable.ToRelContext context, final RelOptTable table)
|
||||
{
|
||||
final RelOptCluster cluster = context.getCluster();
|
||||
return DruidQueryRel.fullScan(
|
||||
cluster,
|
||||
cluster.traitSet(),
|
||||
table,
|
||||
this
|
||||
);
|
||||
return DruidQueryRel.fullScan(cluster, table, this);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -32,8 +32,10 @@ import io.druid.query.ordering.StringComparators;
|
|||
import io.druid.segment.column.Column;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.sql.calcite.expression.RowExtraction;
|
||||
import io.druid.sql.calcite.planner.Calcites;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.calcite.rel.type.RelDataTypeFactory;
|
||||
import org.apache.calcite.sql.SqlCollation;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
|
||||
import java.util.List;
|
||||
|
@ -118,31 +120,38 @@ public class RowSignature
|
|||
public RelDataType getRelDataType(final RelDataTypeFactory typeFactory)
|
||||
{
|
||||
final RelDataTypeFactory.FieldInfoBuilder builder = typeFactory.builder();
|
||||
for (int columnNumber = 0; columnNumber < columnNames.size(); columnNumber++) {
|
||||
final String columnName = columnNames.get(columnNumber);
|
||||
for (final String columnName : columnNames) {
|
||||
final ValueType columnType = getColumnType(columnName);
|
||||
final RelDataType sqlTypeName;
|
||||
final RelDataType type;
|
||||
|
||||
if (Column.TIME_COLUMN_NAME.equals(columnName)) {
|
||||
sqlTypeName = typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
|
||||
type = typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
|
||||
} else {
|
||||
switch (columnType) {
|
||||
case STRING:
|
||||
// Note that there is no attempt here to handle multi-value in any special way. Maybe one day...
|
||||
sqlTypeName = typeFactory.createSqlType(SqlTypeName.VARCHAR, RelDataType.PRECISION_NOT_SPECIFIED);
|
||||
type = typeFactory.createTypeWithCharsetAndCollation(
|
||||
typeFactory.createSqlType(SqlTypeName.VARCHAR),
|
||||
Calcites.defaultCharset(),
|
||||
SqlCollation.IMPLICIT
|
||||
);
|
||||
break;
|
||||
case LONG:
|
||||
sqlTypeName = typeFactory.createSqlType(SqlTypeName.BIGINT);
|
||||
type = typeFactory.createSqlType(SqlTypeName.BIGINT);
|
||||
break;
|
||||
case FLOAT:
|
||||
sqlTypeName = typeFactory.createSqlType(SqlTypeName.FLOAT);
|
||||
type = typeFactory.createSqlType(SqlTypeName.FLOAT);
|
||||
break;
|
||||
case COMPLEX:
|
||||
// Loses information about exactly what kind of complex column this is.
|
||||
type = typeFactory.createSqlType(SqlTypeName.OTHER);
|
||||
break;
|
||||
default:
|
||||
throw new ISE("WTF?! valueType[%s] not translatable?", columnType);
|
||||
}
|
||||
}
|
||||
|
||||
builder.add(columnName, sqlTypeName);
|
||||
builder.add(columnName, type);
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
|
|
|
@ -0,0 +1,36 @@
|
|||
/*
|
||||
* 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.sql.guice;
|
||||
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.multibindings.Multibinder;
|
||||
import io.druid.sql.calcite.aggregation.SqlAggregator;
|
||||
|
||||
public class SqlBindings
|
||||
{
|
||||
public static void addAggregator(
|
||||
final Binder binder,
|
||||
final Class<? extends SqlAggregator> aggregatorClass
|
||||
)
|
||||
{
|
||||
final Multibinder<SqlAggregator> setBinder = Multibinder.newSetBinder(binder, SqlAggregator.class);
|
||||
setBinder.addBinding().to(aggregatorClass);
|
||||
}
|
||||
}
|
|
@ -31,21 +31,22 @@ import io.druid.guice.LifecycleModule;
|
|||
import io.druid.server.initialization.jetty.JettyBindings;
|
||||
import io.druid.server.metrics.MetricsModule;
|
||||
import io.druid.sql.avatica.AvaticaMonitor;
|
||||
import io.druid.sql.avatica.AvaticaServerConfig;
|
||||
import io.druid.sql.avatica.DruidAvaticaHandler;
|
||||
import io.druid.sql.calcite.DruidSchema;
|
||||
import io.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator;
|
||||
import io.druid.sql.calcite.planner.Calcites;
|
||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||
import io.druid.sql.calcite.schema.DruidSchema;
|
||||
import io.druid.sql.http.SqlResource;
|
||||
import org.apache.calcite.jdbc.CalciteConnection;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
|
||||
import java.sql.SQLException;
|
||||
import java.util.Properties;
|
||||
|
||||
public class SqlModule implements Module
|
||||
{
|
||||
private static final String PROPERTY_SQL_ENABLE = "druid.sql.enable";
|
||||
private static final String PROPERTY_SQL_ENABLE_JSON_OVER_HTTP = "druid.sql.server.enableJsonOverHttp";
|
||||
private static final String PROPERTY_SQL_ENABLE_AVATICA = "druid.sql.server.enableAvatica";
|
||||
private static final String PROPERTY_SQL_ENABLE_JSON_OVER_HTTP = "druid.sql.http.enable";
|
||||
private static final String PROPERTY_SQL_ENABLE_AVATICA = "druid.sql.avatica.enable";
|
||||
|
||||
@Inject
|
||||
private Properties props;
|
||||
|
@ -58,8 +59,12 @@ public class SqlModule implements Module
|
|||
public void configure(Binder binder)
|
||||
{
|
||||
if (isEnabled()) {
|
||||
Calcites.setSystemProperties();
|
||||
|
||||
JsonConfigProvider.bind(binder, "druid.sql.planner", PlannerConfig.class);
|
||||
JsonConfigProvider.bind(binder, "druid.sql.avatica", AvaticaServerConfig.class);
|
||||
LifecycleModule.register(binder, DruidSchema.class);
|
||||
SqlBindings.addAggregator(binder, ApproxCountDistinctSqlAggregator.class);
|
||||
|
||||
if (isJsonOverHttpEnabled()) {
|
||||
Jerseys.addResource(binder, SqlResource.class);
|
||||
|
@ -74,15 +79,12 @@ public class SqlModule implements Module
|
|||
}
|
||||
|
||||
@Provides
|
||||
public CalciteConnection createCalciteConnection(
|
||||
final DruidSchema druidSchema,
|
||||
final PlannerConfig plannerConfig
|
||||
) throws SQLException
|
||||
public SchemaPlus createRootSchema(final DruidSchema druidSchema)
|
||||
{
|
||||
if (isEnabled()) {
|
||||
return Calcites.jdbc(druidSchema, plannerConfig);
|
||||
return Calcites.createRootSchema(druidSchema);
|
||||
} else {
|
||||
throw new IllegalStateException("Cannot provide CalciteConnection when SQL is disabled.");
|
||||
throw new IllegalStateException("Cannot provide SchemaPlus when SQL is disabled.");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,12 +22,20 @@ package io.druid.sql.http;
|
|||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.inject.Inject;
|
||||
import io.druid.guice.annotations.Json;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.guava.Yielder;
|
||||
import io.druid.java.util.common.guava.Yielders;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.query.QueryInterruptedException;
|
||||
import org.apache.calcite.jdbc.CalciteConnection;
|
||||
import io.druid.sql.calcite.planner.Calcites;
|
||||
import io.druid.sql.calcite.planner.PlannerFactory;
|
||||
import io.druid.sql.calcite.planner.PlannerResult;
|
||||
import org.apache.calcite.plan.RelOptPlanner;
|
||||
import org.apache.calcite.rel.type.RelDataTypeField;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
import org.apache.calcite.tools.Planner;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.ws.rs.Consumes;
|
||||
|
@ -40,11 +48,8 @@ import javax.ws.rs.core.Response;
|
|||
import javax.ws.rs.core.StreamingOutput;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.sql.Connection;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.ResultSetMetaData;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Types;
|
||||
import java.util.List;
|
||||
|
||||
@Path("/druid/v2/sql/")
|
||||
public class SqlResource
|
||||
|
@ -52,16 +57,16 @@ public class SqlResource
|
|||
private static final Logger log = new Logger(SqlResource.class);
|
||||
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final Connection connection;
|
||||
private final PlannerFactory plannerFactory;
|
||||
|
||||
@Inject
|
||||
public SqlResource(
|
||||
@Json ObjectMapper jsonMapper,
|
||||
CalciteConnection connection
|
||||
PlannerFactory plannerFactory
|
||||
)
|
||||
{
|
||||
this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper");
|
||||
this.connection = Preconditions.checkNotNull(connection, "connection");
|
||||
this.plannerFactory = Preconditions.checkNotNull(plannerFactory, "connection");
|
||||
}
|
||||
|
||||
@POST
|
||||
|
@ -72,81 +77,79 @@ public class SqlResource
|
|||
// This is not integrated with the experimental authorization framework.
|
||||
// (Non-trivial since we don't know the dataSources up-front)
|
||||
|
||||
try {
|
||||
final ResultSet resultSet = connection.createStatement().executeQuery(sqlQuery.getQuery());
|
||||
final ResultSetMetaData metaData = resultSet.getMetaData();
|
||||
final PlannerResult plannerResult;
|
||||
|
||||
// Remember which columns are time-typed, so we can emit ISO8601 instead of millis values.
|
||||
final boolean[] timeColumns = new boolean[metaData.getColumnCount()];
|
||||
for (int i = 0; i < metaData.getColumnCount(); i++) {
|
||||
final int columnType = metaData.getColumnType(i + 1);
|
||||
if (columnType == Types.TIMESTAMP || columnType == Types.TIME || columnType == Types.DATE) {
|
||||
timeColumns[i] = true;
|
||||
} else {
|
||||
timeColumns[i] = false;
|
||||
}
|
||||
}
|
||||
|
||||
return Response.ok(
|
||||
new StreamingOutput()
|
||||
{
|
||||
@Override
|
||||
public void write(final OutputStream outputStream) throws IOException, WebApplicationException
|
||||
{
|
||||
try (final JsonGenerator jsonGenerator = jsonMapper.getFactory().createGenerator(outputStream)) {
|
||||
jsonGenerator.writeStartArray();
|
||||
while (resultSet.next()) {
|
||||
jsonGenerator.writeStartObject();
|
||||
for (int i = 0; i < metaData.getColumnCount(); i++) {
|
||||
final Object value;
|
||||
|
||||
if (timeColumns[i]) {
|
||||
value = new DateTime(resultSet.getLong(i + 1));
|
||||
} else {
|
||||
value = resultSet.getObject(i + 1);
|
||||
}
|
||||
|
||||
jsonGenerator.writeObjectField(metaData.getColumnLabel(i + 1), value);
|
||||
}
|
||||
jsonGenerator.writeEndObject();
|
||||
}
|
||||
jsonGenerator.writeEndArray();
|
||||
jsonGenerator.flush();
|
||||
|
||||
// End with CRLF
|
||||
outputStream.write('\r');
|
||||
outputStream.write('\n');
|
||||
}
|
||||
catch (SQLException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
finally {
|
||||
try {
|
||||
resultSet.close();
|
||||
}
|
||||
catch (SQLException e) {
|
||||
log.warn(e, "Failed to close ResultSet, ignoring.");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
).build();
|
||||
try (final Planner planner = plannerFactory.createPlanner()) {
|
||||
plannerResult = Calcites.plan(planner, sqlQuery.getQuery());
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.warn(e, "Failed to handle query: %s", sqlQuery);
|
||||
|
||||
// Unwrap preparing exceptions into potentially more useful exceptions.
|
||||
final Throwable maybeUnwrapped;
|
||||
if (e instanceof SQLException && e.getMessage().contains("Error while preparing statement")) {
|
||||
maybeUnwrapped = e.getCause();
|
||||
final Exception exceptionToReport;
|
||||
|
||||
if (e instanceof RelOptPlanner.CannotPlanException) {
|
||||
exceptionToReport = new ISE("Cannot build plan for query: %s", sqlQuery.getQuery());
|
||||
} else {
|
||||
maybeUnwrapped = e;
|
||||
exceptionToReport = e;
|
||||
}
|
||||
|
||||
return Response.serverError()
|
||||
.type(MediaType.APPLICATION_JSON_TYPE)
|
||||
.entity(jsonMapper.writeValueAsBytes(QueryInterruptedException.wrapIfNeeded(maybeUnwrapped)))
|
||||
.entity(jsonMapper.writeValueAsBytes(QueryInterruptedException.wrapIfNeeded(exceptionToReport)))
|
||||
.build();
|
||||
}
|
||||
|
||||
// Remember which columns are time-typed, so we can emit ISO8601 instead of millis values.
|
||||
final List<RelDataTypeField> fieldList = plannerResult.rowType().getFieldList();
|
||||
final boolean[] timeColumns = new boolean[fieldList.size()];
|
||||
for (int i = 0; i < fieldList.size(); i++) {
|
||||
final SqlTypeName sqlTypeName = fieldList.get(i).getType().getSqlTypeName();
|
||||
timeColumns[i] = SqlTypeName.DATETIME_TYPES.contains(sqlTypeName);
|
||||
}
|
||||
|
||||
final Yielder<Object[]> yielder0 = Yielders.each(plannerResult.run());
|
||||
|
||||
return Response.ok(
|
||||
new StreamingOutput()
|
||||
{
|
||||
@Override
|
||||
public void write(final OutputStream outputStream) throws IOException, WebApplicationException
|
||||
{
|
||||
Yielder<Object[]> yielder = yielder0;
|
||||
|
||||
try (final JsonGenerator jsonGenerator = jsonMapper.getFactory().createGenerator(outputStream)) {
|
||||
jsonGenerator.writeStartArray();
|
||||
|
||||
while (!yielder.isDone()) {
|
||||
final Object[] row = yielder.get();
|
||||
jsonGenerator.writeStartObject();
|
||||
for (int i = 0; i < fieldList.size(); i++) {
|
||||
final Object value;
|
||||
|
||||
if (timeColumns[i]) {
|
||||
value = new DateTime((long) row[i]);
|
||||
} else {
|
||||
value = row[i];
|
||||
}
|
||||
|
||||
jsonGenerator.writeObjectField(fieldList.get(i).getName(), value);
|
||||
}
|
||||
jsonGenerator.writeEndObject();
|
||||
yielder = yielder.next(null);
|
||||
}
|
||||
|
||||
jsonGenerator.writeEndArray();
|
||||
jsonGenerator.flush();
|
||||
|
||||
// End with CRLF
|
||||
outputStream.write('\r');
|
||||
outputStream.write('\n');
|
||||
}
|
||||
finally {
|
||||
yielder.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
).build();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,10 +27,16 @@ import com.google.common.collect.Maps;
|
|||
import io.druid.java.util.common.Pair;
|
||||
import io.druid.server.DruidNode;
|
||||
import io.druid.sql.calcite.planner.Calcites;
|
||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||
import io.druid.sql.calcite.planner.PlannerFactory;
|
||||
import io.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.calcite.jdbc.CalciteConnection;
|
||||
import io.druid.sql.calcite.util.QueryLogHook;
|
||||
import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
import org.apache.calcite.avatica.AvaticaClientRuntimeException;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.eclipse.jetty.server.Server;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
@ -46,6 +52,8 @@ import java.sql.DriverManager;
|
|||
import java.sql.ResultSet;
|
||||
import java.sql.ResultSetMetaData;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.sql.Types;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
|
@ -53,29 +61,49 @@ import java.util.Set;
|
|||
|
||||
public class DruidAvaticaHandlerTest
|
||||
{
|
||||
private static final AvaticaServerConfig AVATICA_CONFIG = new AvaticaServerConfig()
|
||||
{
|
||||
@Override
|
||||
public int getMaxConnections()
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxStatementsPerConnection()
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
};
|
||||
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
private CalciteConnection serverConnection;
|
||||
@Rule
|
||||
public QueryLogHook queryLogHook = QueryLogHook.create();
|
||||
|
||||
private SpecificSegmentsQuerySegmentWalker walker;
|
||||
private Server server;
|
||||
private Connection client;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
Calcites.setSystemProperties();
|
||||
walker = CalciteTests.createMockWalker(temporaryFolder.newFolder());
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
serverConnection = Calcites.jdbc(
|
||||
final SchemaPlus rootSchema = Calcites.createRootSchema(
|
||||
CalciteTests.createMockSchema(
|
||||
CalciteTests.createWalker(temporaryFolder.newFolder()),
|
||||
walker,
|
||||
plannerConfig
|
||||
),
|
||||
plannerConfig
|
||||
)
|
||||
);
|
||||
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
|
||||
final DruidAvaticaHandler handler = new DruidAvaticaHandler(
|
||||
serverConnection,
|
||||
new DruidMeta(new PlannerFactory(rootSchema, operatorTable, plannerConfig), AVATICA_CONFIG),
|
||||
new DruidNode("dummy", "dummy", 1),
|
||||
new AvaticaMonitor()
|
||||
);
|
||||
|
@ -96,10 +124,10 @@ public class DruidAvaticaHandlerTest
|
|||
{
|
||||
client.close();
|
||||
server.stop();
|
||||
serverConnection.close();
|
||||
walker.close();
|
||||
walker = null;
|
||||
client = null;
|
||||
server = null;
|
||||
serverConnection = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -115,18 +143,46 @@ public class DruidAvaticaHandlerTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSelectCountAlternateStyle() throws Exception
|
||||
{
|
||||
final ResultSet resultSet = client.prepareStatement("SELECT COUNT(*) AS cnt FROM druid.foo").executeQuery();
|
||||
final List<Map<String, Object>> rows = getRows(resultSet);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ImmutableMap.of("cnt", 6L)
|
||||
),
|
||||
rows
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimestampsInResponse() throws Exception
|
||||
{
|
||||
final ResultSet resultSet = client.createStatement().executeQuery(
|
||||
"SELECT __time FROM druid.foo LIMIT 1"
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ImmutableMap.of("__time", new DateTime("2000-01-01T00:00:00.000Z").toDate())
|
||||
),
|
||||
getRows(resultSet)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFieldAliasingSelect() throws Exception
|
||||
{
|
||||
final ResultSet resultSet = client.createStatement().executeQuery(
|
||||
"SELECT dim2 AS \"x\", dim2 AS \"y\" FROM druid.foo LIMIT 1"
|
||||
);
|
||||
final List<Map<String, Object>> rows = getRows(resultSet);
|
||||
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ImmutableMap.of("x", "a", "y", "a")
|
||||
),
|
||||
rows
|
||||
getRows(resultSet)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -136,16 +192,27 @@ public class DruidAvaticaHandlerTest
|
|||
final ResultSet resultSet = client.createStatement().executeQuery(
|
||||
"EXPLAIN PLAN FOR SELECT COUNT(*) AS cnt FROM druid.foo"
|
||||
);
|
||||
final List<Map<String, Object>> rows = getRows(resultSet);
|
||||
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ImmutableMap.of(
|
||||
"PLAN",
|
||||
"EnumerableInterpreter\n"
|
||||
+ " DruidQueryRel(dataSource=[foo], dimensions=[[]], aggregations=[[Aggregation{aggregatorFactories=[CountAggregatorFactory{name='a0'}], postAggregator=null, finalizingPostAggregatorFactory=null}]])\n"
|
||||
"DruidQueryRel(dataSource=[foo], dimensions=[[]], aggregations=[[Aggregation{aggregatorFactories=[CountAggregatorFactory{name='a0'}], postAggregator=null, finalizingPostAggregatorFactory=null}]])\n"
|
||||
)
|
||||
),
|
||||
rows
|
||||
getRows(resultSet)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDatabaseMetaDataCatalogs() throws Exception
|
||||
{
|
||||
final DatabaseMetaData metaData = client.getMetaData();
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ROW(Pair.of("TABLE_CAT", ""))
|
||||
),
|
||||
getRows(metaData.getCatalogs())
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -155,7 +222,7 @@ public class DruidAvaticaHandlerTest
|
|||
final DatabaseMetaData metaData = client.getMetaData();
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ROW(Pair.of("TABLE_CATALOG", null), Pair.of("TABLE_SCHEM", "druid"))
|
||||
ROW(Pair.of("TABLE_CATALOG", ""), Pair.of("TABLE_SCHEM", "druid"))
|
||||
),
|
||||
getRows(metaData.getSchemas(null, "druid"))
|
||||
);
|
||||
|
@ -172,6 +239,12 @@ public class DruidAvaticaHandlerTest
|
|||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_TYPE", "TABLE")
|
||||
),
|
||||
ROW(
|
||||
Pair.of("TABLE_CAT", null),
|
||||
Pair.of("TABLE_NAME", "foo2"),
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_TYPE", "TABLE")
|
||||
)
|
||||
),
|
||||
getRows(
|
||||
|
@ -185,57 +258,87 @@ public class DruidAvaticaHandlerTest
|
|||
public void testDatabaseMetaDataColumns() throws Exception
|
||||
{
|
||||
final DatabaseMetaData metaData = client.getMetaData();
|
||||
final String varcharDescription = "VARCHAR(1) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\" NOT NULL";
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ROW(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "__time"),
|
||||
Pair.of("DATA_TYPE", 93),
|
||||
Pair.of("TYPE_NAME", "TIMESTAMP(0) NOT NULL"),
|
||||
Pair.of("DATA_TYPE", Types.TIMESTAMP),
|
||||
Pair.of("TYPE_NAME", "TIMESTAMP"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "cnt"),
|
||||
Pair.of("DATA_TYPE", -5),
|
||||
Pair.of("TYPE_NAME", "BIGINT NOT NULL"),
|
||||
Pair.of("DATA_TYPE", Types.BIGINT),
|
||||
Pair.of("TYPE_NAME", "BIGINT"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "dim1"),
|
||||
Pair.of("DATA_TYPE", 12),
|
||||
Pair.of("TYPE_NAME", varcharDescription),
|
||||
Pair.of("DATA_TYPE", Types.VARCHAR),
|
||||
Pair.of("TYPE_NAME", "VARCHAR"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "dim2"),
|
||||
Pair.of("DATA_TYPE", 12),
|
||||
Pair.of("TYPE_NAME", varcharDescription),
|
||||
Pair.of("DATA_TYPE", Types.VARCHAR),
|
||||
Pair.of("TYPE_NAME", "VARCHAR"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "m1"),
|
||||
Pair.of("DATA_TYPE", 6),
|
||||
Pair.of("TYPE_NAME", "FLOAT NOT NULL"),
|
||||
Pair.of("DATA_TYPE", Types.FLOAT),
|
||||
Pair.of("TYPE_NAME", "FLOAT"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "unique_dim1"),
|
||||
Pair.of("DATA_TYPE", Types.OTHER),
|
||||
Pair.of("TYPE_NAME", "OTHER"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
)
|
||||
),
|
||||
getRows(
|
||||
metaData.getColumns(null, "druid", "foo", "%"),
|
||||
metaData.getColumns(null, "dr_id", "foo", null),
|
||||
ImmutableSet.of("IS_NULLABLE", "TABLE_NAME", "TABLE_SCHEM", "COLUMN_NAME", "DATA_TYPE", "TYPE_NAME")
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTooManyStatements() throws Exception
|
||||
{
|
||||
final Statement statement1 = client.createStatement();
|
||||
final Statement statement2 = client.createStatement();
|
||||
|
||||
expectedException.expect(AvaticaClientRuntimeException.class);
|
||||
expectedException.expectMessage("Too many open statements, limit is[2]");
|
||||
final Statement statement3 = client.createStatement();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNotTooManyStatementsWhenYouCloseThem() throws Exception
|
||||
{
|
||||
client.createStatement().close();
|
||||
client.createStatement().close();
|
||||
client.createStatement().close();
|
||||
client.createStatement().close();
|
||||
client.createStatement().close();
|
||||
|
||||
Assert.assertTrue(true);
|
||||
}
|
||||
|
||||
private static List<Map<String, Object>> getRows(final ResultSet resultSet) throws SQLException
|
||||
{
|
||||
return getRows(resultSet, null);
|
||||
|
|
|
@ -0,0 +1,180 @@
|
|||
/*
|
||||
* 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.sql.avatica;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.sql.calcite.planner.Calcites;
|
||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||
import io.druid.sql.calcite.planner.PlannerFactory;
|
||||
import io.druid.sql.calcite.util.CalciteTests;
|
||||
import io.druid.sql.calcite.util.QueryLogHook;
|
||||
import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
import org.apache.calcite.avatica.ColumnMetaData;
|
||||
import org.apache.calcite.avatica.Meta;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class DruidStatementTest
|
||||
{
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
@Rule
|
||||
public QueryLogHook queryLogHook = QueryLogHook.create();
|
||||
|
||||
private SpecificSegmentsQuerySegmentWalker walker;
|
||||
private PlannerFactory plannerFactory;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
Calcites.setSystemProperties();
|
||||
walker = CalciteTests.createMockWalker(temporaryFolder.newFolder());
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final SchemaPlus rootSchema = Calcites.createRootSchema(
|
||||
CalciteTests.createMockSchema(
|
||||
walker,
|
||||
plannerConfig
|
||||
)
|
||||
);
|
||||
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
|
||||
plannerFactory = new PlannerFactory(rootSchema, operatorTable, plannerConfig);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception
|
||||
{
|
||||
walker.close();
|
||||
walker = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSignature() throws Exception
|
||||
{
|
||||
final String sql = "SELECT * FROM druid.foo";
|
||||
final DruidStatement statement = new DruidStatement("", 0).prepare(plannerFactory, sql, -1);
|
||||
|
||||
// Check signature.
|
||||
final Meta.Signature signature = statement.getSignature();
|
||||
Assert.assertEquals(Meta.CursorFactory.ARRAY, signature.cursorFactory);
|
||||
Assert.assertEquals(Meta.StatementType.SELECT, signature.statementType);
|
||||
Assert.assertEquals(sql, signature.sql);
|
||||
Assert.assertEquals(
|
||||
Lists.newArrayList(
|
||||
Lists.newArrayList("__time", "TIMESTAMP", "java.lang.Long"),
|
||||
Lists.newArrayList("cnt", "BIGINT", "java.lang.Long"),
|
||||
Lists.newArrayList("dim1", "VARCHAR", "java.lang.String"),
|
||||
Lists.newArrayList("dim2", "VARCHAR", "java.lang.String"),
|
||||
Lists.newArrayList("m1", "FLOAT", "java.lang.Double"),
|
||||
Lists.newArrayList("unique_dim1", "OTHER", "java.lang.Object")
|
||||
),
|
||||
Lists.transform(
|
||||
signature.columns,
|
||||
new Function<ColumnMetaData, List<String>>()
|
||||
{
|
||||
@Override
|
||||
public List<String> apply(final ColumnMetaData columnMetaData)
|
||||
{
|
||||
return Lists.newArrayList(
|
||||
columnMetaData.label,
|
||||
columnMetaData.type.name,
|
||||
columnMetaData.type.rep.clazz.getName()
|
||||
);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSelectAllInFirstFrame() throws Exception
|
||||
{
|
||||
final String sql = "SELECT __time, cnt, dim1, dim2, m1 FROM druid.foo";
|
||||
final DruidStatement statement = new DruidStatement("", 0).prepare(plannerFactory, sql, -1);
|
||||
|
||||
// First frame, ask for all rows.
|
||||
Meta.Frame frame = statement.execute().nextFrame(DruidStatement.START_OFFSET, 6);
|
||||
Assert.assertEquals(
|
||||
Meta.Frame.create(
|
||||
0,
|
||||
true,
|
||||
Lists.<Object>newArrayList(
|
||||
new Object[]{new DateTime("2000-01-01").getMillis(), 1L, "", "a", 1.0},
|
||||
new Object[]{new DateTime("2000-01-02").getMillis(), 1L, "10.1", "", 2.0},
|
||||
new Object[]{new DateTime("2000-01-03").getMillis(), 1L, "2", "", 3.0},
|
||||
new Object[]{new DateTime("2001-01-01").getMillis(), 1L, "1", "a", 4.0},
|
||||
new Object[]{new DateTime("2001-01-02").getMillis(), 1L, "def", "abc", 5.0},
|
||||
new Object[]{new DateTime("2001-01-03").getMillis(), 1L, "abc", "", 6.0}
|
||||
)
|
||||
),
|
||||
frame
|
||||
);
|
||||
Assert.assertTrue(statement.isDone());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSelectSplitOverTwoFrames() throws Exception
|
||||
{
|
||||
final String sql = "SELECT __time, cnt, dim1, dim2, m1 FROM druid.foo";
|
||||
final DruidStatement statement = new DruidStatement("", 0).prepare(plannerFactory, sql, -1);
|
||||
|
||||
// First frame, ask for 2 rows.
|
||||
Meta.Frame frame = statement.execute().nextFrame(DruidStatement.START_OFFSET, 2);
|
||||
Assert.assertEquals(
|
||||
Meta.Frame.create(
|
||||
0,
|
||||
false,
|
||||
Lists.<Object>newArrayList(
|
||||
new Object[]{new DateTime("2000-01-01").getMillis(), 1L, "", "a", 1.0},
|
||||
new Object[]{new DateTime("2000-01-02").getMillis(), 1L, "10.1", "", 2.0}
|
||||
)
|
||||
),
|
||||
frame
|
||||
);
|
||||
Assert.assertFalse(statement.isDone());
|
||||
|
||||
// Last frame, ask for all remaining rows.
|
||||
frame = statement.nextFrame(2, 10);
|
||||
Assert.assertEquals(
|
||||
Meta.Frame.create(
|
||||
2,
|
||||
true,
|
||||
Lists.<Object>newArrayList(
|
||||
new Object[]{new DateTime("2000-01-03").getMillis(), 1L, "2", "", 3.0},
|
||||
new Object[]{new DateTime("2001-01-01").getMillis(), 1L, "1", "a", 4.0},
|
||||
new Object[]{new DateTime("2001-01-02").getMillis(), 1L, "def", "abc", 5.0},
|
||||
new Object[]{new DateTime("2001-01-03").getMillis(), 1L, "abc", "", 6.0}
|
||||
)
|
||||
),
|
||||
frame
|
||||
);
|
||||
Assert.assertTrue(statement.isDone());
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -26,12 +26,13 @@ import com.google.common.collect.ImmutableMap;
|
|||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.query.QueryInterruptedException;
|
||||
import io.druid.sql.calcite.planner.Calcites;
|
||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||
import io.druid.sql.calcite.planner.PlannerFactory;
|
||||
import io.druid.sql.calcite.util.CalciteTests;
|
||||
import io.druid.sql.http.SqlQuery;
|
||||
import io.druid.sql.http.SqlResource;
|
||||
import org.apache.calcite.jdbc.CalciteConnection;
|
||||
import org.junit.After;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
|
@ -55,28 +56,21 @@ public class SqlResourceTest
|
|||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
private CalciteConnection connection;
|
||||
private SqlResource resource;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
Calcites.setSystemProperties();
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
connection = Calcites.jdbc(
|
||||
final SchemaPlus rootSchema = Calcites.createRootSchema(
|
||||
CalciteTests.createMockSchema(
|
||||
CalciteTests.createWalker(temporaryFolder.newFolder()),
|
||||
CalciteTests.createMockWalker(temporaryFolder.newFolder()),
|
||||
plannerConfig
|
||||
),
|
||||
plannerConfig
|
||||
)
|
||||
);
|
||||
resource = new SqlResource(JSON_MAPPER, connection);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception
|
||||
{
|
||||
connection.close();
|
||||
connection = null;
|
||||
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
|
||||
resource = new SqlResource(JSON_MAPPER, new PlannerFactory(rootSchema, operatorTable, plannerConfig));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -152,8 +146,7 @@ public class SqlResourceTest
|
|||
ImmutableList.of(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"PLAN",
|
||||
"EnumerableInterpreter\n"
|
||||
+ " DruidQueryRel(dataSource=[foo], dimensions=[[]], aggregations=[[Aggregation{aggregatorFactories=[CountAggregatorFactory{name='a0'}], postAggregator=null, finalizingPostAggregatorFactory=null}]])\n"
|
||||
"DruidQueryRel(dataSource=[foo], dimensions=[[]], aggregations=[[Aggregation{aggregatorFactories=[CountAggregatorFactory{name='a0'}], postAggregator=null, finalizingPostAggregatorFactory=null}]])\n"
|
||||
)
|
||||
),
|
||||
rows
|
||||
|
@ -161,7 +154,7 @@ public class SqlResourceTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testCannotPlan() throws Exception
|
||||
public void testCannotValidate() throws Exception
|
||||
{
|
||||
expectedException.expect(QueryInterruptedException.class);
|
||||
expectedException.expectMessage("Column 'dim3' not found in any table");
|
||||
|
@ -173,6 +166,18 @@ public class SqlResourceTest
|
|||
Assert.fail();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCannotConvert() throws Exception
|
||||
{
|
||||
expectedException.expect(QueryInterruptedException.class);
|
||||
expectedException.expectMessage("Cannot build plan for query: SELECT TRIM(dim1) FROM druid.foo");
|
||||
|
||||
// TRIM unsupported
|
||||
doPost(new SqlQuery("SELECT TRIM(dim1) FROM druid.foo"));
|
||||
|
||||
Assert.fail();
|
||||
}
|
||||
|
||||
private List<Map<String, Object>> doPost(final SqlQuery query) throws Exception
|
||||
{
|
||||
final Response response = resource.doPost(query);
|
||||
|
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* 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.sql.calcite.planner;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class CalcitesTest
|
||||
{
|
||||
@Test
|
||||
public void testEscapeStringLiteral()
|
||||
{
|
||||
Assert.assertEquals("''", Calcites.escapeStringLiteral(null));
|
||||
Assert.assertEquals("''", Calcites.escapeStringLiteral(""));
|
||||
Assert.assertEquals("'foo'", Calcites.escapeStringLiteral("foo"));
|
||||
Assert.assertEquals("'foo bar'", Calcites.escapeStringLiteral("foo bar"));
|
||||
Assert.assertEquals("U&'foö bar'", Calcites.escapeStringLiteral("foö bar"));
|
||||
Assert.assertEquals("U&'foo \\0026\\0026 bar'", Calcites.escapeStringLiteral("foo && bar"));
|
||||
Assert.assertEquals("U&'foo \\005C bar'", Calcites.escapeStringLiteral("foo \\ bar"));
|
||||
Assert.assertEquals("U&'foo\\0027s bar'", Calcites.escapeStringLiteral("foo's bar"));
|
||||
Assert.assertEquals("U&'друид'", Calcites.escapeStringLiteral("друид"));
|
||||
}
|
||||
}
|
|
@ -17,10 +17,10 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.sql.calcite;
|
||||
package io.druid.sql.calcite.schema;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import io.druid.sql.calcite.planner.Calcites;
|
||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||
import io.druid.sql.calcite.table.DruidTable;
|
||||
import io.druid.sql.calcite.util.CalciteTests;
|
||||
|
@ -48,20 +48,6 @@ import java.util.Properties;
|
|||
public class DruidSchemaTest
|
||||
{
|
||||
private static final PlannerConfig PLANNER_CONFIG_DEFAULT = new PlannerConfig();
|
||||
private static final PlannerConfig PLANNER_CONFIG_NO_TOPN = new PlannerConfig()
|
||||
{
|
||||
@Override
|
||||
public int getMaxTopNLimit()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isUseApproximateTopN()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
@ -73,7 +59,8 @@ public class DruidSchemaTest
|
|||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
walker = CalciteTests.createWalker(temporaryFolder.newFolder());
|
||||
Calcites.setSystemProperties();
|
||||
walker = CalciteTests.createMockWalker(temporaryFolder.newFolder());
|
||||
|
||||
Properties props = new Properties();
|
||||
props.setProperty("caseSensitive", "true");
|
||||
|
@ -103,17 +90,16 @@ public class DruidSchemaTest
|
|||
@Test
|
||||
public void testGetTableMap()
|
||||
{
|
||||
Assert.assertEquals(ImmutableSet.of("foo"), schema.getTableNames());
|
||||
Assert.assertEquals(ImmutableSet.of("foo", "foo2"), schema.getTableNames());
|
||||
|
||||
final Map<String, Table> tableMap = schema.getTableMap();
|
||||
Assert.assertEquals(1, tableMap.size());
|
||||
Assert.assertEquals("foo", Iterables.getOnlyElement(tableMap.keySet()));
|
||||
Assert.assertEquals(ImmutableSet.of("foo", "foo2"), tableMap.keySet());
|
||||
|
||||
final DruidTable druidTable = (DruidTable) Iterables.getOnlyElement(tableMap.values());
|
||||
final RelDataType rowType = druidTable.getRowType(new JavaTypeFactoryImpl());
|
||||
final DruidTable fooTable = (DruidTable) tableMap.get("foo");
|
||||
final RelDataType rowType = fooTable.getRowType(new JavaTypeFactoryImpl());
|
||||
final List<RelDataTypeField> fields = rowType.getFieldList();
|
||||
|
||||
Assert.assertEquals(5, fields.size());
|
||||
Assert.assertEquals(6, fields.size());
|
||||
|
||||
Assert.assertEquals("__time", fields.get(0).getName());
|
||||
Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName());
|
||||
|
@ -129,5 +115,8 @@ public class DruidSchemaTest
|
|||
|
||||
Assert.assertEquals("m1", fields.get(4).getName());
|
||||
Assert.assertEquals(SqlTypeName.FLOAT, fields.get(4).getType().getSqlTypeName());
|
||||
|
||||
Assert.assertEquals("unique_dim1", fields.get(5).getName());
|
||||
Assert.assertEquals(SqlTypeName.OTHER, fields.get(5).getType().getSqlTypeName());
|
||||
}
|
||||
}
|
|
@ -22,6 +22,7 @@ package io.druid.sql.calcite.util;
|
|||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
|
@ -66,6 +67,9 @@ import io.druid.segment.QueryableIndex;
|
|||
import io.druid.segment.TestHelper;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import io.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator;
|
||||
import io.druid.sql.calcite.aggregation.SqlAggregator;
|
||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||
import io.druid.sql.calcite.rel.QueryMaker;
|
||||
import io.druid.sql.calcite.table.DruidTable;
|
||||
|
@ -74,6 +78,7 @@ import io.druid.timeline.partition.LinearShardSpec;
|
|||
import org.apache.calcite.schema.Schema;
|
||||
import org.apache.calcite.schema.Table;
|
||||
import org.apache.calcite.schema.impl.AbstractSchema;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.io.File;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -85,9 +90,102 @@ import java.util.Map;
|
|||
*/
|
||||
public class CalciteTests
|
||||
{
|
||||
public static final String DATASOURCE = "foo";
|
||||
public static final String DATASOURCE1 = "foo";
|
||||
public static final String DATASOURCE2 = "foo2";
|
||||
|
||||
private static final String TIMESTAMP_COLUMN = "t";
|
||||
|
||||
private static final QueryRunnerFactoryConglomerate CONGLOMERATE = new DefaultQueryRunnerFactoryConglomerate(
|
||||
ImmutableMap.<Class<? extends Query>, QueryRunnerFactory>builder()
|
||||
.put(
|
||||
SegmentMetadataQuery.class,
|
||||
new SegmentMetadataQueryRunnerFactory(
|
||||
new SegmentMetadataQueryQueryToolChest(
|
||||
new SegmentMetadataQueryConfig("P1W")
|
||||
),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
)
|
||||
.put(
|
||||
SelectQuery.class,
|
||||
new SelectQueryRunnerFactory(
|
||||
new SelectQueryQueryToolChest(
|
||||
TestHelper.getObjectMapper(),
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
new SelectQueryEngine(),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
)
|
||||
.put(
|
||||
TimeseriesQuery.class,
|
||||
new TimeseriesQueryRunnerFactory(
|
||||
new TimeseriesQueryQueryToolChest(
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
new TimeseriesQueryEngine(),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
)
|
||||
.put(
|
||||
TopNQuery.class,
|
||||
new TopNQueryRunnerFactory(
|
||||
new StupidPool<>(
|
||||
"TopNQueryRunnerFactory-bufferPool",
|
||||
new Supplier<ByteBuffer>()
|
||||
{
|
||||
@Override
|
||||
public ByteBuffer get()
|
||||
{
|
||||
return ByteBuffer.allocate(10 * 1024 * 1024);
|
||||
}
|
||||
}
|
||||
),
|
||||
new TopNQueryQueryToolChest(
|
||||
new TopNQueryConfig(),
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
)
|
||||
.put(
|
||||
GroupByQuery.class,
|
||||
GroupByQueryRunnerTest.makeQueryRunnerFactory(
|
||||
GroupByQueryRunnerTest.DEFAULT_MAPPER,
|
||||
new GroupByQueryConfig()
|
||||
{
|
||||
@Override
|
||||
public String getDefaultStrategy()
|
||||
{
|
||||
return GroupByStrategySelector.STRATEGY_V2;
|
||||
}
|
||||
},
|
||||
new DruidProcessingConfig()
|
||||
{
|
||||
@Override
|
||||
public String getFormatString()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int intermediateComputeSizeBytes()
|
||||
{
|
||||
return 10 * 1024 * 1024;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumMergeBuffers()
|
||||
{
|
||||
// Need 3 buffers for CalciteQueryTest.testDoubleNestedGroupby.
|
||||
return 3;
|
||||
}
|
||||
}
|
||||
)
|
||||
)
|
||||
.build()
|
||||
);
|
||||
|
||||
private static final InputRowParser<Map<String, Object>> PARSER = new MapInputRowParser(
|
||||
new TimeAndDimsParseSpec(
|
||||
new TimestampSpec(TIMESTAMP_COLUMN, "iso", null),
|
||||
|
@ -98,163 +196,101 @@ public class CalciteTests
|
|||
)
|
||||
)
|
||||
);
|
||||
private static final List<InputRow> ROWS = ImmutableList.of(
|
||||
ROW(ImmutableMap.of("t", "2000-01-01", "m1", "1.0", "dim1", "", "dim2", ImmutableList.of("a"))),
|
||||
ROW(ImmutableMap.of("t", "2000-01-02", "m1", "2.0", "dim1", "10.1", "dim2", ImmutableList.of())),
|
||||
ROW(ImmutableMap.of("t", "2000-01-03", "m1", "3.0", "dim1", "2", "dim2", ImmutableList.of(""))),
|
||||
ROW(ImmutableMap.of("t", "2001-01-01", "m1", "4.0", "dim1", "1", "dim2", ImmutableList.of("a"))),
|
||||
ROW(ImmutableMap.of("t", "2001-01-02", "m1", "5.0", "dim1", "def", "dim2", ImmutableList.of("abc"))),
|
||||
ROW(ImmutableMap.of("t", "2001-01-03", "m1", "6.0", "dim1", "abc"))
|
||||
|
||||
private static final IncrementalIndexSchema INDEX_SCHEMA = new IncrementalIndexSchema.Builder()
|
||||
.withMetrics(
|
||||
new AggregatorFactory[]{
|
||||
new CountAggregatorFactory("cnt"),
|
||||
new DoubleSumAggregatorFactory("m1", "m1"),
|
||||
new HyperUniquesAggregatorFactory("unique_dim1", "dim1")
|
||||
}
|
||||
)
|
||||
.withRollup(false)
|
||||
.build();
|
||||
|
||||
private static final List<InputRow> ROWS1 = ImmutableList.of(
|
||||
createRow(ImmutableMap.of("t", "2000-01-01", "m1", "1.0", "dim1", "", "dim2", ImmutableList.of("a"))),
|
||||
createRow(ImmutableMap.of("t", "2000-01-02", "m1", "2.0", "dim1", "10.1", "dim2", ImmutableList.of())),
|
||||
createRow(ImmutableMap.of("t", "2000-01-03", "m1", "3.0", "dim1", "2", "dim2", ImmutableList.of(""))),
|
||||
createRow(ImmutableMap.of("t", "2001-01-01", "m1", "4.0", "dim1", "1", "dim2", ImmutableList.of("a"))),
|
||||
createRow(ImmutableMap.of("t", "2001-01-02", "m1", "5.0", "dim1", "def", "dim2", ImmutableList.of("abc"))),
|
||||
createRow(ImmutableMap.of("t", "2001-01-03", "m1", "6.0", "dim1", "abc"))
|
||||
);
|
||||
private static final Map<String, ValueType> COLUMN_TYPES = ImmutableMap.of(
|
||||
"__time", ValueType.LONG,
|
||||
"cnt", ValueType.LONG,
|
||||
"dim1", ValueType.STRING,
|
||||
"dim2", ValueType.STRING,
|
||||
"m1", ValueType.FLOAT
|
||||
|
||||
private static final List<InputRow> ROWS2 = ImmutableList.of(
|
||||
createRow("2000-01-01", "דרואיד", "he", 1.0),
|
||||
createRow("2000-01-01", "druid", "en", 1.0),
|
||||
createRow("2000-01-01", "друид", "ru", 1.0)
|
||||
);
|
||||
|
||||
private static final Map<String, ValueType> COLUMN_TYPES = ImmutableMap.<String, ValueType>builder()
|
||||
.put("__time", ValueType.LONG)
|
||||
.put("cnt", ValueType.LONG)
|
||||
.put("dim1", ValueType.STRING)
|
||||
.put("dim2", ValueType.STRING)
|
||||
.put("m1", ValueType.FLOAT)
|
||||
.put("unique_dim1", ValueType.COMPLEX)
|
||||
.build();
|
||||
|
||||
private CalciteTests()
|
||||
{
|
||||
// No instantiation.
|
||||
}
|
||||
|
||||
public static SpecificSegmentsQuerySegmentWalker createWalker(final File tmpDir)
|
||||
public static QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate()
|
||||
{
|
||||
return createWalker(tmpDir, ROWS);
|
||||
return CONGLOMERATE;
|
||||
}
|
||||
|
||||
public static SpecificSegmentsQuerySegmentWalker createWalker(final File tmpDir, final List<InputRow> rows)
|
||||
public static SpecificSegmentsQuerySegmentWalker createMockWalker(final File tmpDir)
|
||||
{
|
||||
final QueryableIndex index = IndexBuilder.create()
|
||||
.tmpDir(tmpDir)
|
||||
.indexMerger(TestHelper.getTestIndexMergerV9())
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMetrics(
|
||||
new AggregatorFactory[]{
|
||||
new CountAggregatorFactory("cnt"),
|
||||
new DoubleSumAggregatorFactory("m1", "m1"),
|
||||
new HyperUniquesAggregatorFactory("unique_dim1", "dim1")
|
||||
}
|
||||
)
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.rows(rows)
|
||||
.buildMMappedIndex();
|
||||
final QueryableIndex index1 = IndexBuilder.create()
|
||||
.tmpDir(new File(tmpDir, "1"))
|
||||
.indexMerger(TestHelper.getTestIndexMergerV9())
|
||||
.schema(INDEX_SCHEMA)
|
||||
.rows(ROWS1)
|
||||
.buildMMappedIndex();
|
||||
|
||||
final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate(
|
||||
ImmutableMap.<Class<? extends Query>, QueryRunnerFactory>builder()
|
||||
.put(
|
||||
SegmentMetadataQuery.class,
|
||||
new SegmentMetadataQueryRunnerFactory(
|
||||
new SegmentMetadataQueryQueryToolChest(
|
||||
new SegmentMetadataQueryConfig("P1W")
|
||||
),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
)
|
||||
.put(
|
||||
SelectQuery.class,
|
||||
new SelectQueryRunnerFactory(
|
||||
new SelectQueryQueryToolChest(
|
||||
TestHelper.getObjectMapper(),
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
new SelectQueryEngine(),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
)
|
||||
.put(
|
||||
TimeseriesQuery.class,
|
||||
new TimeseriesQueryRunnerFactory(
|
||||
new TimeseriesQueryQueryToolChest(
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
new TimeseriesQueryEngine(),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
)
|
||||
.put(
|
||||
TopNQuery.class,
|
||||
new TopNQueryRunnerFactory(
|
||||
new StupidPool<>(
|
||||
"TopNQueryRunnerFactory-bufferPool",
|
||||
new Supplier<ByteBuffer>()
|
||||
{
|
||||
@Override
|
||||
public ByteBuffer get()
|
||||
{
|
||||
return ByteBuffer.allocate(10 * 1024 * 1024);
|
||||
}
|
||||
}
|
||||
),
|
||||
new TopNQueryQueryToolChest(
|
||||
new TopNQueryConfig(),
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
)
|
||||
.put(
|
||||
GroupByQuery.class,
|
||||
GroupByQueryRunnerTest.makeQueryRunnerFactory(
|
||||
GroupByQueryRunnerTest.DEFAULT_MAPPER,
|
||||
new GroupByQueryConfig()
|
||||
{
|
||||
@Override
|
||||
public String getDefaultStrategy()
|
||||
{
|
||||
return GroupByStrategySelector.STRATEGY_V2;
|
||||
}
|
||||
},
|
||||
new DruidProcessingConfig()
|
||||
{
|
||||
@Override
|
||||
public String getFormatString()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
final QueryableIndex index2 = IndexBuilder.create()
|
||||
.tmpDir(new File(tmpDir, "2"))
|
||||
.indexMerger(TestHelper.getTestIndexMergerV9())
|
||||
.schema(INDEX_SCHEMA)
|
||||
.rows(ROWS2)
|
||||
.buildMMappedIndex();
|
||||
|
||||
@Override
|
||||
public int intermediateComputeSizeBytes()
|
||||
{
|
||||
return 10 * 1024 * 1024;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumMergeBuffers()
|
||||
{
|
||||
// Need 3 buffers for CalciteQueryTest.testDoubleNestedGroupby.
|
||||
return 3;
|
||||
}
|
||||
}
|
||||
)
|
||||
)
|
||||
.build()
|
||||
);
|
||||
|
||||
return new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
|
||||
return new SpecificSegmentsQuerySegmentWalker(queryRunnerFactoryConglomerate()).add(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATASOURCE)
|
||||
.interval(index.getDataInterval())
|
||||
.dataSource(DATASOURCE1)
|
||||
.interval(index1.getDataInterval())
|
||||
.version("1")
|
||||
.shardSpec(new LinearShardSpec(0))
|
||||
.build(),
|
||||
index
|
||||
index1
|
||||
).add(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATASOURCE2)
|
||||
.interval(index2.getDataInterval())
|
||||
.version("1")
|
||||
.shardSpec(new LinearShardSpec(0))
|
||||
.build(),
|
||||
index2
|
||||
);
|
||||
}
|
||||
|
||||
public static DruidTable createDruidTable(final QuerySegmentWalker walker, final PlannerConfig plannerConfig)
|
||||
public static DruidOperatorTable createOperatorTable()
|
||||
{
|
||||
return new DruidTable(new QueryMaker(walker, plannerConfig), new TableDataSource(DATASOURCE), COLUMN_TYPES);
|
||||
return new DruidOperatorTable(ImmutableSet.<SqlAggregator>of(new ApproxCountDistinctSqlAggregator()));
|
||||
}
|
||||
|
||||
public static Schema createMockSchema(final QuerySegmentWalker walker, final PlannerConfig plannerConfig)
|
||||
{
|
||||
final DruidTable druidTable = createDruidTable(walker, plannerConfig);
|
||||
final Map<String, Table> tableMap = ImmutableMap.<String, Table>of(DATASOURCE, druidTable);
|
||||
final QueryMaker queryMaker = new QueryMaker(walker, plannerConfig);
|
||||
final DruidTable druidTable1 = new DruidTable(queryMaker, new TableDataSource(DATASOURCE1), COLUMN_TYPES);
|
||||
final DruidTable druidTable2 = new DruidTable(queryMaker, new TableDataSource(DATASOURCE2), COLUMN_TYPES);
|
||||
final Map<String, Table> tableMap = ImmutableMap.<String, Table>of(
|
||||
DATASOURCE1, druidTable1,
|
||||
DATASOURCE2, druidTable2
|
||||
);
|
||||
return new AbstractSchema()
|
||||
{
|
||||
@Override
|
||||
|
@ -265,8 +301,20 @@ public class CalciteTests
|
|||
};
|
||||
}
|
||||
|
||||
private static InputRow ROW(final ImmutableMap<String, ?> map)
|
||||
public static InputRow createRow(final ImmutableMap<String, ?> map)
|
||||
{
|
||||
return PARSER.parse((Map<String, Object>) map);
|
||||
}
|
||||
|
||||
public static InputRow createRow(final Object t, final String dim1, final String dim2, final double m1)
|
||||
{
|
||||
return PARSER.parse(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"t", new DateTime(t).getMillis(),
|
||||
"dim1", dim1,
|
||||
"dim2", dim2,
|
||||
"m1", m1
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,106 @@
|
|||
/*
|
||||
* 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.sql.calcite.util;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.query.Query;
|
||||
import org.apache.calcite.runtime.Hook;
|
||||
import org.junit.rules.TestRule;
|
||||
import org.junit.runner.Description;
|
||||
import org.junit.runners.model.Statement;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* JUnit Rule that adds a Calcite hook to log and remember Druid queries.
|
||||
*/
|
||||
public class QueryLogHook implements TestRule
|
||||
{
|
||||
private static final Logger log = new Logger(QueryLogHook.class);
|
||||
|
||||
private final ObjectMapper objectMapper;
|
||||
private final List<Query> recordedQueries = Lists.newCopyOnWriteArrayList();
|
||||
|
||||
public QueryLogHook(final ObjectMapper objectMapper)
|
||||
{
|
||||
this.objectMapper = objectMapper;
|
||||
}
|
||||
|
||||
public static QueryLogHook create()
|
||||
{
|
||||
return new QueryLogHook(new DefaultObjectMapper());
|
||||
}
|
||||
|
||||
public static QueryLogHook create(final ObjectMapper objectMapper)
|
||||
{
|
||||
return new QueryLogHook(objectMapper);
|
||||
}
|
||||
|
||||
public void clearRecordedQueries()
|
||||
{
|
||||
recordedQueries.clear();
|
||||
}
|
||||
|
||||
public List<Query> getRecordedQueries()
|
||||
{
|
||||
return ImmutableList.copyOf(recordedQueries);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Statement apply(final Statement base, final Description description)
|
||||
{
|
||||
return new Statement()
|
||||
{
|
||||
@Override
|
||||
public void evaluate() throws Throwable
|
||||
{
|
||||
clearRecordedQueries();
|
||||
|
||||
final Function<Object, Object> function = new Function<Object, Object>()
|
||||
{
|
||||
@Override
|
||||
public Object apply(final Object query)
|
||||
{
|
||||
try {
|
||||
recordedQueries.add((Query) query);
|
||||
log.info(
|
||||
"Issued query: %s",
|
||||
objectMapper.writerWithDefaultPrettyPrinter().writeValueAsString(query)
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.warn(e, "Failed to serialize query: %s", query);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
||||
try (final Hook.Closeable unhook = Hook.QUERY_PLAN.add(function)) {
|
||||
base.evaluate();
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -81,6 +81,7 @@ public class SpecificSegmentsQuerySegmentWalker implements QuerySegmentWalker, C
|
|||
final VersionedIntervalTimeline<String, Segment> timeline = timelines.get(descriptor.getDataSource());
|
||||
timeline.add(descriptor.getInterval(), descriptor.getVersion(), descriptor.getShardSpec().createChunk(segment));
|
||||
segments.add(descriptor);
|
||||
closeables.add(index);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue