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:
Gian Merlino 2017-01-19 16:32:20 -08:00 committed by Fangjin Yang
parent 33ae9dd485
commit d51f5e058d
51 changed files with 4105 additions and 1063 deletions

View File

@ -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);
}
}
}

View File

@ -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

View File

@ -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)|

View File

@ -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>

View File

@ -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);
}
}
}

View File

@ -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()
{

View File

@ -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
);
}
}
}

View File

@ -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())
);
}
}
}

View File

@ -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>()

View File

@ -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

View File

@ -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;

View File

@ -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;
}
}

View File

@ -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()));
}

View File

@ -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;
}
}

View File

@ -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);
}
}
}

View File

@ -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);
}
}

View File

@ -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
);
}
}
}

View File

@ -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
);
}

View File

@ -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);

View File

@ -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")
)
);
}
}

View File

@ -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;
}
}

View File

@ -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();
}
}

View File

@ -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);
}
}

View File

@ -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;
}
}

View File

@ -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();
}

View File

@ -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;
}
}

View File

@ -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

View File

@ -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);
}

View File

@ -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()
{

View File

@ -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;
}
}
);
}
};
}

View File

@ -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) {

View File

@ -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);
}

View File

@ -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();
}
}

View File

@ -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();
}
}

View File

@ -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()

View File

@ -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);

View File

@ -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;
}
}
}

View File

@ -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

View File

@ -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();

View File

@ -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);
}
}

View File

@ -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.");
}
}

View File

@ -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();
}
}

View File

@ -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);

View File

@ -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

View File

@ -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);

View File

@ -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("друид"));
}
}

View File

@ -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());
}
}

View File

@ -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
)
);
}
}

View File

@ -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();
}
}
};
}
}

View File

@ -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;
}