Built-in SQL. (#3682)

This commit is contained in:
Gian Merlino 2016-12-16 17:15:59 -08:00 committed by Fangjin Yang
parent 2bfcc8a592
commit dd63f54325
75 changed files with 12132 additions and 11 deletions

6
NOTICE
View File

@ -44,3 +44,9 @@ This product contains a modified version of Metamarkets bytebuffer-collections l
* https://github.com/metamx/bytebuffer-collections
* COMMIT TAG:
* https://github.com/metamx/bytebuffer-collections/commit/3d1e7c8
This product contains SQL query planning code adapted from Apache Calcite
* LICENSE:
* https://github.com/apache/calcite/blob/master/LICENSE (Apache License, Version 2.0)
* HOMEPAGE:
* https://calcite.apache.org/

View File

@ -56,6 +56,23 @@
<artifactId>druid-server</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-sql</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-processing</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-sql</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>com.github.wnameless</groupId>
<artifactId>json-flattener</artifactId>

View File

@ -0,0 +1,237 @@
/*
* 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.benchmark.query;
/*
* 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.
*/
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.hash.Hashing;
import com.google.common.io.Files;
import io.druid.benchmark.datagen.BenchmarkDataGenerator;
import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
import io.druid.benchmark.datagen.BenchmarkSchemas;
import io.druid.common.utils.JodaUtils;
import io.druid.data.input.InputRow;
import io.druid.data.input.Row;
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.TableDataSource;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
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.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.table.DruidTable;
import io.druid.sql.calcite.util.CalciteTests;
import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.calcite.jdbc.CalciteConnection;
import org.apache.calcite.schema.Schema;
import org.apache.calcite.schema.Table;
import org.apache.calcite.schema.impl.AbstractSchema;
import org.apache.commons.io.FileUtils;
import org.joda.time.Interval;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Level;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
import java.io.File;
import java.sql.ResultSet;
import java.sql.ResultSetMetaData;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
/**
* Benchmark that compares the same groupBy query through the native query layer and through the SQL layer.
*/
@State(Scope.Benchmark)
@Fork(jvmArgsPrepend = "-server", value = 1)
@Warmup(iterations = 15)
@Measurement(iterations = 30)
public class SqlBenchmark
{
@Param({"10000", "100000", "200000"})
private int rowsPerSegment;
private static final Logger log = new Logger(SqlBenchmark.class);
private static final int RNG_SEED = 9999;
private File tmpDir;
private SpecificSegmentsQuerySegmentWalker walker;
private CalciteConnection calciteConnection;
private GroupByQuery groupByQuery;
private String sqlQuery;
@Setup(Level.Trial)
public void setup() throws Exception
{
tmpDir = Files.createTempDir();
log.info("Starting benchmark setup using tmpDir[%s], rows[%,d].", tmpDir, rowsPerSegment);
if (ComplexMetrics.getSerdeForType("hyperUnique") == null) {
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128()));
}
final BenchmarkSchemaInfo schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get("basic");
final BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(
schemaInfo.getColumnSchemas(),
RNG_SEED + 1,
schemaInfo.getDataInterval(),
rowsPerSegment
);
final List<InputRow> rows = Lists.newArrayList();
for (int i = 0; i < rowsPerSegment; i++) {
final InputRow row = dataGenerator.nextRow();
if (i % 20000 == 0) {
log.info("%,d/%,d rows generated.", i, rowsPerSegment);
}
rows.add(row);
}
log.info("%,d/%,d rows generated.", rows.size(), rowsPerSegment);
final PlannerConfig plannerConfig = new PlannerConfig();
walker = CalciteTests.createWalker(tmpDir, rows);
final Map<String, Table> tableMap = ImmutableMap.<String, Table>of(
"foo",
new DruidTable(
walker,
new TableDataSource("foo"),
plannerConfig,
ImmutableMap.of(
"__time", ValueType.LONG,
"dimSequential", ValueType.STRING,
"dimZipf", ValueType.STRING,
"dimUniform", ValueType.STRING
)
)
);
final Schema druidSchema = new AbstractSchema()
{
@Override
protected Map<String, Table> getTableMap()
{
return tableMap;
}
};
calciteConnection = Calcites.jdbc(druidSchema, plannerConfig);
groupByQuery = GroupByQuery
.builder()
.setDataSource("foo")
.setInterval(new Interval(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT))
.setDimensions(
Arrays.<DimensionSpec>asList(
new DefaultDimensionSpec("dimZipf", "d0"),
new DefaultDimensionSpec("dimSequential", "d1")
)
)
.setAggregatorSpecs(Arrays.<AggregatorFactory>asList(new CountAggregatorFactory("c")))
.setGranularity(QueryGranularities.ALL)
.build();
sqlQuery = "SELECT\n"
+ " dimZipf AS d0,"
+ " dimSequential AS d1,\n"
+ " COUNT(*) AS c\n"
+ "FROM druid.foo\n"
+ "GROUP BY dimZipf, dimSequential";
}
@TearDown(Level.Trial)
public void tearDown() throws Exception
{
if (walker != null) {
walker.close();
walker = null;
}
if (tmpDir != null) {
FileUtils.deleteDirectory(tmpDir);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryNative(Blackhole blackhole) throws Exception
{
final Sequence<Row> resultSequence = groupByQuery.run(walker, Maps.<String, Object>newHashMap());
final ArrayList<Row> resultList = Sequences.toList(resultSequence, Lists.<Row>newArrayList());
for (Row row : resultList) {
blackhole.consume(row);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void querySql(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));
}
}
}
}

View File

@ -85,6 +85,30 @@ See [groupBy server configuration](../querying/groupbyquery.html#server-configur
|--------|-----------|-------|
|`druid.query.segmentMetadata.defaultHistory`|When no interval is specified in the query, use a default interval of defaultHistory before the end time of the most recent segment, specified in ISO8601 format. This property also controls the duration of the default interval used by GET /druid/v2/datasources/{dataSourceName} interactions for retrieving datasource dimensions/metrics.|P1W|
#### SQL Server Configuration
The broker's [built-in SQL server](../querying/sql.html) can be configured through the following properties.
|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/`.|false|
|`druid.sql.server.enableJsonOverHttp`|Whether to enable a simple JSON over HTTP route at `/druid/v2/sql/`.|true|
#### SQL Planner Configuration
The broker's [SQL planner](../querying/sql.html) can be configured through the following properties.
|Property|Description|Default|
|--------|-----------|-------|
|`druid.sql.planner.maxSemiJoinRowsInMemory`|Maximum number of rows to keep in memory for executing two-stage semi-join queries like `SELECT * FROM Employee WHERE DeptName IN (SELECT DeptName FROM Dept)`.|100000|
|`druid.sql.planner.maxTopNLimit`|Maximum threshold for a [TopN query](../querying/topnquery.html). Higher limits will be planned as [GroupBy queries](../querying/groupbyquery.html) instead.|100000|
|`druid.sql.planner.metadataRefreshPeriod`|Throttle for metadata refreshes.|PT1M|
|`druid.sql.planner.selectPageSize`|Page size threshold for [Select queries](../querying/select-query.html). Select queries for larger resultsets will be issued back-to-back using pagination.|1000|
|`druid.sql.planner.useApproximateCountDistinct`|Whether to use an approximate cardinalty algorithm for `COUNT(DISTINCT foo)`.|true|
|`druid.sql.planner.useApproximateTopN`|Whether to use approximate [TopN queries](../querying/topnquery.html) when a SQL query could be expressed as such. If false, exact [GroupBy queries](../querying/groupbyquery.html) will be used instead.|true|
|`druid.sql.planner.useFallback`|Whether to evaluate operations on the broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|false|
### Caching
You can optionally only configure caching to be enabled on the broker by setting caching configs here.

View File

@ -2,6 +2,130 @@
layout: doc_page
---
# SQL Support for Druid
Full SQL is currently not supported with Druid. SQL libraries on top of Druid have been contributed by the community and can be found on our [libraries](../development/libraries.html) page.
The community SQL libraries are not yet as expressive as Druid's native query language.
## Built-in SQL
<div class="note caution">
Built-in SQL is an <a href="../development/experimental.html">experimental</a> feature. The API described here is
subject to change.
</div>
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.
Add "EXPLAIN PLAN FOR" to the beginning of any query to see how Druid will plan that query.
### Querying with JDBC
You can make Druid SQL queries using the [Avatica JDBC driver](https://calcite.apache.org/avatica/downloads/). Once
you've downloaded the Avatica client jar, add it to your classpath and use the connect string:
```
jdbc:avatica:remote:url=http://BROKER:8082/druid/v2/sql/avatica/
```
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");
while (resultSet.next()) {
// Do something
}
```
Table metadata is available over JDBC using `connection.getMetaData()`.
Parameterized queries don't work properly, so avoid those.
### Querying with JSON over HTTP
You can make Druid SQL queries using JSON over HTTP by POSTing to the endpoint `/druid/v2/sql/`. The request format
is:
```json
{
"query" : "SELECT COUNT(*) FROM druid.ds WHERE foo = ?"
}
```
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"}'
```
Metadata is not available over the HTTP API.
### Metadata
Druid brokers cache column type metadata for each dataSource and use it to plan SQL queries. This cache is updated
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:
```sql
SELECT * FROM metadata.COLUMNS
```
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.
### Time functions
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'`
### Semi-joins
Semi-joins involving `IN (SELECT ...)`, like the following, are planned with a special process.
```sql
SELECT x, count(*)
FROM druid.foo
WHERE x IN (SELECT x FROM druid.bar 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
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).
### 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 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`.
- A number of miscellaneous functions, like `TRIM`.
- Joins, other than semi-joins as described above.
Additionally, some Druid features are not supported by the SQL language. Some unsupported Druid features include:
- [Multi-value dimensions](multi-value-dimensions.html).
- [Query-time lookups](lookups.html).
- [Nested groupBy queries](groupbyquery.html#nested-groupbys).
- Extensions, including [approximate histograms](../development/extensions-core/approximate-histograms.html) and
[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.

42
pom.xml
View File

@ -59,6 +59,8 @@
<properties>
<apache.curator.version>2.11.1</apache.curator.version>
<avatica.version>1.9.0</avatica.version>
<calcite.version>1.10.0</calcite.version>
<guava.version>16.0.1</guava.version>
<guice.version>4.1.0</guice.version>
<jetty.version>9.2.5.v20141112</jetty.version>
@ -66,6 +68,7 @@
<!-- Watch out for Hadoop compatibility when updating to >= 2.5; see https://github.com/druid-io/druid/pull/1669 -->
<jackson.version>2.4.6</jackson.version>
<log4j.version>2.5</log4j.version>
<netty.version>4.1.6.Final</netty.version>
<slf4j.version>1.7.12</slf4j.version>
<!-- If compiling with different hadoop version also modify default hadoop coordinates in TaskConfig.java -->
<hadoop.compile.version>2.3.0</hadoop.compile.version>
@ -80,6 +83,7 @@
<module>indexing-service</module>
<module>processing</module>
<module>server</module>
<module>sql</module>
<module>services</module>
<module>integration-tests</module>
<module>benchmarks</module>
@ -246,6 +250,26 @@
<artifactId>curator-x-discovery</artifactId>
<version>${apache.curator.version}</version>
</dependency>
<dependency>
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-core</artifactId>
<version>${calcite.version}</version>
</dependency>
<dependency>
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-linq4j</artifactId>
<version>${calcite.version}</version>
</dependency>
<dependency>
<groupId>org.apache.calcite.avatica</groupId>
<artifactId>avatica-core</artifactId>
<version>${avatica.version}</version>
</dependency>
<dependency>
<groupId>org.apache.calcite.avatica</groupId>
<artifactId>avatica-server</artifactId>
<version>${avatica.version}</version>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
@ -391,6 +415,16 @@
<artifactId>jetty-proxy</artifactId>
<version>${jetty.version}</version>
</dependency>
<dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-util</artifactId>
<version>${jetty.version}</version>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
<version>${netty.version}</version>
</dependency>
<dependency>
<groupId>joda-time</groupId>
<artifactId>joda-time</artifactId>
@ -478,7 +512,7 @@
<dependency>
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
<version>2.5.0</version>
<version>3.1.0</version>
</dependency>
<dependency>
<groupId>io.tesla.aether</groupId>
@ -584,6 +618,12 @@
<artifactId>slf4j-api</artifactId>
<version>1.6.4</version>
</dependency>
<dependency>
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-core</artifactId>
<version>${calcite.version}</version>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>

View File

@ -120,6 +120,10 @@
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-server</artifactId>
</dependency>
<dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-util</artifactId>
</dependency>
<dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-proxy</artifactId>

View File

@ -22,9 +22,8 @@ package io.druid.server.initialization.jetty;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Binder;
import com.google.inject.multibindings.Multibinder;
import io.druid.java.util.common.logger.Logger;
import org.eclipse.jetty.server.Handler;
import org.eclipse.jetty.servlets.QoSFilter;
import javax.servlet.DispatcherType;
@ -52,6 +51,13 @@ public class JettyBindings
.toInstance(new QosFilterHolder(path, maxRequests));
}
public static void addHandler(Binder binder, Class<? extends Handler> handlerClass)
{
Multibinder.newSetBinder(binder, Handler.class)
.addBinding()
.to(handlerClass);
}
private static class QosFilterHolder implements ServletFilterHolder
{
private final String path;

View File

@ -59,6 +59,7 @@ import io.druid.server.metrics.MetricsModule;
import io.druid.server.metrics.MonitorsConfig;
import org.eclipse.jetty.server.ConnectionFactory;
import org.eclipse.jetty.server.Connector;
import org.eclipse.jetty.server.Handler;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.server.ServerConnector;
import org.eclipse.jetty.util.thread.QueuedThreadPool;
@ -95,8 +96,9 @@ public class JettyServerModule extends JerseyServletModule
Jerseys.addResource(binder, StatusResource.class);
binder.bind(StatusResource.class).in(LazySingleton.class);
//Adding empty binding for ServletFilterHolders so that injector returns
//an empty set when no external modules provide ServletFilterHolder impls
// Adding empty binding for ServletFilterHolders and Handlers so that injector returns an empty set if none
// are provided by extensions.
Multibinder.newSetBinder(binder, Handler.class);
Multibinder.newSetBinder(binder, ServletFilterHolder.class);
MetricsModule.register(binder, JettyMonitor.class);

View File

@ -51,6 +51,11 @@
<artifactId>druid-indexing-service</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-sql</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>io.airlift</groupId>
<artifactId>airline</artifactId>

View File

@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import com.google.inject.Module;
import com.google.inject.name.Names;
import io.airlift.airline.Command;
import io.druid.client.BrokerSegmentWatcherConfig;
import io.druid.client.BrokerServerView;
@ -51,6 +50,7 @@ import io.druid.server.http.BrokerResource;
import io.druid.server.initialization.jetty.JettyServerInitializer;
import io.druid.server.metrics.MetricsModule;
import io.druid.server.router.TieredBrokerConfig;
import io.druid.sql.guice.SqlModule;
import org.eclipse.jetty.server.Server;
import java.util.List;
@ -111,7 +111,8 @@ public class CliBroker extends ServerRunnable
LifecycleModule.register(binder, Server.class);
}
},
new LookupModule()
new LookupModule(),
new SqlModule()
);
}
}

View File

@ -19,6 +19,8 @@
package io.druid.cli;
import com.google.common.collect.ImmutableList;
import com.google.inject.Inject;
import com.google.inject.Injector;
import com.google.inject.servlet.GuiceFilter;
import io.druid.server.initialization.jetty.JettyServerInitUtils;
@ -30,10 +32,21 @@ import org.eclipse.jetty.servlet.DefaultServlet;
import org.eclipse.jetty.servlet.ServletContextHandler;
import org.eclipse.jetty.servlet.ServletHolder;
import java.util.List;
import java.util.Set;
/**
*/
public class QueryJettyServerInitializer implements JettyServerInitializer
{
private final List<Handler> extensionHandlers;
@Inject
public QueryJettyServerInitializer(Set<Handler> extensionHandlers)
{
this.extensionHandlers = ImmutableList.copyOf(extensionHandlers);
}
@Override
public void initialize(Server server, Injector injector)
{
@ -45,7 +58,13 @@ public class QueryJettyServerInitializer implements JettyServerInitializer
root.addFilter(GuiceFilter.class, "/*", null);
final HandlerList handlerList = new HandlerList();
handlerList.setHandlers(new Handler[]{JettyServerInitUtils.getJettyRequestLogHandler(), root});
final Handler[] handlers = new Handler[extensionHandlers.size() + 2];
handlers[0] = JettyServerInitUtils.getJettyRequestLogHandler();
handlers[handlers.length - 1] = root;
for (int i = 0; i < extensionHandlers.size(); i++) {
handlers[i + 1] = extensionHandlers.get(i);
}
handlerList.setHandlers(handlers);
server.setHandler(handlerList);
}
}

99
sql/pom.xml Normal file
View File

@ -0,0 +1,99 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ Licensed to Metamarkets Group Inc. (Metamarkets) under one
~ or more contributor license agreements. See the NOTICE file
~ distributed with this work for additional information
~ regarding copyright ownership. Metamarkets licenses this file
~ to you under the Apache License, Version 2.0 (the
~ "License"); you may not use this file except in compliance
~ with the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing,
~ software distributed under the License is distributed on an
~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
~ KIND, either express or implied. See the License for the
~ specific language governing permissions and limitations
~ under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>druid-sql</artifactId>
<name>druid-sql</name>
<description>Druid SQL</description>
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.9.3-SNAPSHOT</version>
</parent>
<dependencies>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-server</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-core</artifactId>
</dependency>
<dependency>
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-linq4j</artifactId>
</dependency>
<dependency>
<groupId>org.apache.calcite.avatica</groupId>
<artifactId>avatica-core</artifactId>
</dependency>
<dependency>
<groupId>org.apache.calcite.avatica</groupId>
<artifactId>avatica-server</artifactId>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
</dependency>
<!-- Tests -->
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-core</artifactId>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-processing</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>test-jar</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,183 @@
/*
* 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.collect.Maps;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import com.metamx.metrics.AbstractMonitor;
import io.druid.java.util.common.logger.Logger;
import org.apache.calcite.avatica.metrics.Counter;
import org.apache.calcite.avatica.metrics.Gauge;
import org.apache.calcite.avatica.metrics.Histogram;
import org.apache.calcite.avatica.metrics.Meter;
import org.apache.calcite.avatica.metrics.MetricsSystem;
import org.apache.calcite.avatica.metrics.Timer;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
public class AvaticaMonitor extends AbstractMonitor implements MetricsSystem
{
private static final Logger log = new Logger(AvaticaMonitor.class);
private final ConcurrentMap<String, AtomicLong> counters = Maps.newConcurrentMap();
private final ConcurrentMap<String, Gauge<?>> gauges = Maps.newConcurrentMap();
@Override
public boolean doMonitor(final ServiceEmitter emitter)
{
for (final Map.Entry<String, AtomicLong> entry : counters.entrySet()) {
final String name = entry.getKey();
final long value = entry.getValue().getAndSet(0);
emitter.emit(ServiceMetricEvent.builder().build(fullMetricName(name), value));
}
for (Map.Entry<String, Gauge<?>> entry : gauges.entrySet()) {
final String name = entry.getKey();
final Object value = entry.getValue().getValue();
if (value instanceof Number) {
emitter.emit(ServiceMetricEvent.builder().build(fullMetricName(name), (Number) value));
} else {
log.debug("Not emitting gauge[%s] since value[%s] type was[%s].", name, value, value.getClass().getName());
}
}
return true;
}
@Override
public Timer getTimer(final String name)
{
final AtomicLong counter = makeCounter(name);
return new Timer()
{
@Override
public Context start()
{
final long start = System.currentTimeMillis();
final AtomicBoolean closed = new AtomicBoolean();
return new Context()
{
@Override
public void close()
{
if (closed.compareAndSet(false, true)) {
counter.addAndGet(System.currentTimeMillis() - start);
}
}
};
}
};
}
@Override
public Histogram getHistogram(final String name)
{
// Return a dummy Histogram. We don't support Histogram metrics.
return new Histogram()
{
@Override
public void update(int i)
{
// Do nothing.
}
@Override
public void update(long l)
{
// Do nothing.
}
};
}
@Override
public Meter getMeter(final String name)
{
final AtomicLong counter = makeCounter(name);
return new Meter()
{
@Override
public void mark()
{
counter.incrementAndGet();
}
@Override
public void mark(long events)
{
counter.addAndGet(events);
}
};
}
@Override
public Counter getCounter(final String name)
{
final AtomicLong counter = makeCounter(name);
return new Counter()
{
@Override
public void increment()
{
counter.incrementAndGet();
}
@Override
public void increment(long n)
{
counter.addAndGet(n);
}
@Override
public void decrement()
{
counter.decrementAndGet();
}
@Override
public void decrement(long n)
{
counter.addAndGet(-n);
}
};
}
@Override
public <T> void register(final String name, final Gauge<T> gauge)
{
if (gauges.putIfAbsent(name, gauge) != null) {
log.warn("Ignoring gauge[%s], one with the same name was already registered.", name);
}
}
private AtomicLong makeCounter(final String name)
{
counters.putIfAbsent(name, new AtomicLong());
return counters.get(name);
}
private String fullMetricName(final String name)
{
return name.replace("org.apache.calcite.avatica", "avatica").replace(".", "/");
}
}

View File

@ -0,0 +1,77 @@
/*
* 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.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;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
public class DruidAvaticaHandler extends AvaticaJsonHandler
{
static final String AVATICA_PATH = "/druid/v2/sql/avatica/";
private final ServerConfig config;
@Inject
public DruidAvaticaHandler(
final CalciteConnection connection,
@Self final DruidNode druidNode,
final AvaticaMonitor avaticaMonitor,
final ServerConfig config
) throws InstantiationException, IllegalAccessException, InvocationTargetException
{
super(
new LocalService((Meta) CalciteMetaImpl.class.getConstructors()[0].newInstance(connection), avaticaMonitor),
avaticaMonitor
);
this.config = config;
setServerRpcMetadata(new Service.RpcMetadataResponse(druidNode.getHostAndPort()));
}
@Override
public void handle(
final String target,
final Request baseRequest,
final HttpServletRequest request,
final HttpServletResponse response
) throws IOException, ServletException
{
// This is not integrated with the experimental authorization framework.
// (Non-trivial since we don't know the dataSources up-front)
if (config.isEnableAvatica() && request.getRequestURI().equals(AVATICA_PATH)) {
super.handle(target, baseRequest, request, response);
}
}
}

View File

@ -0,0 +1,41 @@
/*
* 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;
public class ServerConfig
{
@JsonProperty
private boolean enableAvatica = false;
@JsonProperty
private boolean enableJsonOverHttp = true;
public boolean isEnableAvatica()
{
return enableAvatica;
}
public boolean isEnableJsonOverHttp()
{
return enableJsonOverHttp;
}
}

View File

@ -0,0 +1,358 @@
/*
* 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;
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;
import com.metamx.emitter.EmittingLogger;
import io.druid.client.DruidDataSource;
import io.druid.client.DruidServer;
import io.druid.client.ServerView;
import io.druid.client.TimelineServerView;
import io.druid.guice.ManageLifecycle;
import io.druid.java.util.common.concurrent.ScheduledExecutors;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.lifecycle.LifecycleStart;
import io.druid.java.util.common.lifecycle.LifecycleStop;
import io.druid.query.QuerySegmentWalker;
import io.druid.query.TableDataSource;
import io.druid.query.metadata.metadata.ColumnAnalysis;
import io.druid.query.metadata.metadata.SegmentAnalysis;
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
import io.druid.segment.column.ValueType;
import io.druid.server.coordination.DruidServerMetadata;
import io.druid.sql.calcite.planner.PlannerConfig;
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;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
@ManageLifecycle
public class DruidSchema extends AbstractSchema
{
private static final EmittingLogger log = new EmittingLogger(DruidSchema.class);
private final QuerySegmentWalker walker;
private final TimelineServerView serverView;
private final PlannerConfig config;
private final ExecutorService cacheExec;
private final ConcurrentMap<String, Table> tables;
// For awaitInitialization.
private final CountDownLatch initializationLatch = new CountDownLatch(1);
// Protects access to dataSourcesNeedingRefresh, lastRefresh, isServerViewInitialized
private final Object lock = new Object();
// List of dataSources that need metadata refreshes.
private final Set<String> dataSourcesNeedingRefresh = Sets.newHashSet();
private boolean refreshImmediately = false;
private long lastRefresh = 0L;
private boolean isServerViewInitialized = false;
@Inject
public DruidSchema(
final QuerySegmentWalker walker,
final TimelineServerView serverView,
final PlannerConfig config
)
{
this.walker = Preconditions.checkNotNull(walker, "walker");
this.serverView = Preconditions.checkNotNull(serverView, "serverView");
this.config = Preconditions.checkNotNull(config, "config");
this.cacheExec = ScheduledExecutors.fixed(1, "DruidSchema-Cache-%d");
this.tables = Maps.newConcurrentMap();
}
@LifecycleStart
public void start()
{
cacheExec.submit(
new Runnable()
{
@Override
public void run()
{
try {
while (!Thread.currentThread().isInterrupted()) {
final Set<String> dataSources = Sets.newHashSet();
try {
synchronized (lock) {
final long nextRefresh = new DateTime(lastRefresh).plus(config.getMetadataRefreshPeriod())
.getMillis();
while (!(
isServerViewInitialized
&& !dataSourcesNeedingRefresh.isEmpty()
&& (refreshImmediately || nextRefresh < System.currentTimeMillis())
)) {
lock.wait(Math.max(1, nextRefresh - System.currentTimeMillis()));
}
dataSources.addAll(dataSourcesNeedingRefresh);
dataSourcesNeedingRefresh.clear();
lastRefresh = System.currentTimeMillis();
refreshImmediately = false;
}
// Refresh dataSources.
for (final String dataSource : dataSources) {
log.debug("Refreshing metadata for dataSource[%s].", dataSource);
final long startTime = System.currentTimeMillis();
final DruidTable druidTable = computeTable(dataSource);
if (druidTable == null) {
if (tables.remove(dataSource) != null) {
log.info("Removed dataSource[%s] from the list of active dataSources.", dataSource);
}
} else {
tables.put(dataSource, druidTable);
log.info(
"Refreshed metadata for dataSource[%s] in %,dms.",
dataSource,
System.currentTimeMillis() - startTime
);
}
}
initializationLatch.countDown();
}
catch (InterruptedException e) {
// Fall through.
throw e;
}
catch (Exception e) {
log.warn(
e,
"Metadata refresh failed for dataSources[%s], trying again soon.",
Joiner.on(", ").join(dataSources)
);
synchronized (lock) {
// Add dataSources back to the refresh list.
dataSourcesNeedingRefresh.addAll(dataSources);
lock.notifyAll();
}
}
}
}
catch (InterruptedException e) {
// Just exit.
}
catch (Throwable e) {
// Throwables that fall out to here (not caught by an inner try/catch) are potentially gnarly, like
// OOMEs. Anyway, let's just emit an alert and stop refreshing metadata.
log.makeAlert(e, "Metadata refresh failed permanently").emit();
throw e;
}
finally {
log.info("Metadata refresh stopped.");
}
}
}
);
serverView.registerSegmentCallback(
MoreExecutors.sameThreadExecutor(),
new ServerView.SegmentCallback()
{
@Override
public ServerView.CallbackAction segmentViewInitialized()
{
synchronized (lock) {
isServerViewInitialized = true;
lock.notifyAll();
}
return ServerView.CallbackAction.CONTINUE;
}
@Override
public ServerView.CallbackAction segmentAdded(DruidServerMetadata server, DataSegment segment)
{
synchronized (lock) {
dataSourcesNeedingRefresh.add(segment.getDataSource());
if (!tables.containsKey(segment.getDataSource())) {
refreshImmediately = true;
}
lock.notifyAll();
}
return ServerView.CallbackAction.CONTINUE;
}
@Override
public ServerView.CallbackAction segmentRemoved(DruidServerMetadata server, DataSegment segment)
{
synchronized (lock) {
dataSourcesNeedingRefresh.add(segment.getDataSource());
lock.notifyAll();
}
return ServerView.CallbackAction.CONTINUE;
}
}
);
serverView.registerServerCallback(
MoreExecutors.sameThreadExecutor(),
new ServerView.ServerCallback()
{
@Override
public ServerView.CallbackAction serverRemoved(DruidServer server)
{
final List<String> dataSourceNames = Lists.newArrayList();
for (DruidDataSource druidDataSource : server.getDataSources()) {
dataSourceNames.add(druidDataSource.getName());
}
synchronized (lock) {
dataSourcesNeedingRefresh.addAll(dataSourceNames);
lock.notifyAll();
}
return ServerView.CallbackAction.CONTINUE;
}
}
);
}
@LifecycleStop
public void stop()
{
cacheExec.shutdownNow();
}
@VisibleForTesting
public void awaitInitialization() throws InterruptedException
{
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(
new TableDataSource(dataSource),
null,
null,
true,
null,
EnumSet.noneOf(SegmentMetadataQuery.AnalysisType.class),
null,
true
);
final Sequence<SegmentAnalysis> sequence = segmentMetadataQuery.run(walker, Maps.<String, Object>newHashMap());
final List<SegmentAnalysis> results = Sequences.toList(sequence, Lists.<SegmentAnalysis>newArrayList());
if (results.isEmpty()) {
return null;
}
final Map<String, ColumnAnalysis> columnMetadata = Iterables.getOnlyElement(results).getColumns();
final Map<String, ValueType> columnValueTypes = Maps.newHashMap();
for (Map.Entry<String, ColumnAnalysis> entry : columnMetadata.entrySet()) {
if (entry.getValue().isError()) {
// Ignore columns with metadata consistency errors.
continue;
}
final 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;
}
columnValueTypes.put(entry.getKey(), valueType);
}
return new DruidTable(
walker,
new TableDataSource(dataSource),
config,
columnValueTypes
);
}
}

View File

@ -0,0 +1,196 @@
/*
* 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.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.FilteredAggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.filter.DimFilter;
import java.util.List;
import java.util.Set;
public class Aggregation
{
private final List<AggregatorFactory> aggregatorFactories;
private final PostAggregator postAggregator;
private final PostAggregatorFactory finalizingPostAggregatorFactory;
private Aggregation(
final List<AggregatorFactory> aggregatorFactories,
final PostAggregator postAggregator,
final PostAggregatorFactory finalizingPostAggregatorFactory
)
{
this.aggregatorFactories = Preconditions.checkNotNull(aggregatorFactories, "aggregatorFactories");
this.postAggregator = postAggregator;
this.finalizingPostAggregatorFactory = finalizingPostAggregatorFactory;
if (postAggregator == null) {
Preconditions.checkArgument(aggregatorFactories.size() == 1, "aggregatorFactories.size == 1");
} else {
// Verify that there are no "useless" fields in the aggregatorFactories.
// Don't verify that the PostAggregator inputs are all present; they might not be.
final Set<String> dependentFields = postAggregator.getDependentFields();
for (AggregatorFactory aggregatorFactory : aggregatorFactories) {
if (!dependentFields.contains(aggregatorFactory.getName())) {
throw new IAE("Unused field[%s] in Aggregation", aggregatorFactory.getName());
}
}
}
}
public static Aggregation create(final AggregatorFactory aggregatorFactory)
{
return new Aggregation(ImmutableList.of(aggregatorFactory), null, null);
}
public static Aggregation create(final PostAggregator postAggregator)
{
return new Aggregation(ImmutableList.<AggregatorFactory>of(), postAggregator, null);
}
public static Aggregation create(
final List<AggregatorFactory> aggregatorFactories,
final PostAggregator postAggregator
)
{
return new Aggregation(aggregatorFactories, postAggregator, null);
}
public static Aggregation createFinalizable(
final List<AggregatorFactory> aggregatorFactories,
final PostAggregator postAggregator,
final PostAggregatorFactory finalizingPostAggregatorFactory
)
{
return new Aggregation(
aggregatorFactories,
postAggregator,
Preconditions.checkNotNull(finalizingPostAggregatorFactory, "finalizingPostAggregatorFactory")
);
}
public List<AggregatorFactory> getAggregatorFactories()
{
return aggregatorFactories;
}
public PostAggregator getPostAggregator()
{
return postAggregator;
}
public PostAggregatorFactory getFinalizingPostAggregatorFactory()
{
return finalizingPostAggregatorFactory;
}
public String getOutputName()
{
return postAggregator != null
? postAggregator.getName()
: Iterables.getOnlyElement(aggregatorFactories).getName();
}
public Aggregation filter(final DimFilter filter)
{
if (filter == null) {
return this;
}
if (postAggregator != null) {
// Verify that this Aggregation contains all inputs. If not, this "filter" call won't work right.
final Set<String> dependentFields = postAggregator.getDependentFields();
final Set<String> aggregatorNames = Sets.newHashSet();
for (AggregatorFactory aggregatorFactory : aggregatorFactories) {
aggregatorNames.add(aggregatorFactory.getName());
}
for (String field : dependentFields) {
if (!aggregatorNames.contains(field)) {
throw new ISE("Cannot filter an Aggregation that does not contain its inputs: %s", this);
}
}
}
final List<AggregatorFactory> newAggregators = Lists.newArrayList();
for (AggregatorFactory agg : aggregatorFactories) {
newAggregators.add(new FilteredAggregatorFactory(agg, filter));
}
return new Aggregation(
newAggregators,
postAggregator,
finalizingPostAggregatorFactory
);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Aggregation that = (Aggregation) o;
if (aggregatorFactories != null
? !aggregatorFactories.equals(that.aggregatorFactories)
: that.aggregatorFactories != null) {
return false;
}
if (postAggregator != null ? !postAggregator.equals(that.postAggregator) : that.postAggregator != null) {
return false;
}
return finalizingPostAggregatorFactory != null
? finalizingPostAggregatorFactory.equals(that.finalizingPostAggregatorFactory)
: that.finalizingPostAggregatorFactory == null;
}
@Override
public int hashCode()
{
int result = aggregatorFactories != null ? aggregatorFactories.hashCode() : 0;
result = 31 * result + (postAggregator != null ? postAggregator.hashCode() : 0);
result = 31 * result + (finalizingPostAggregatorFactory != null ? finalizingPostAggregatorFactory.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "Aggregation{" +
"aggregatorFactories=" + aggregatorFactories +
", postAggregator=" + postAggregator +
", finalizingPostAggregatorFactory=" + finalizingPostAggregatorFactory +
'}';
}
}

View File

@ -0,0 +1,57 @@
/*
* 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.query.aggregation.PostAggregator;
/**
* Can create PostAggregators with specific output names.
*/
public abstract class PostAggregatorFactory
{
public abstract PostAggregator factorize(final String outputName);
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
PostAggregatorFactory that = (PostAggregatorFactory) o;
return factorize(null).equals(that.factorize(null));
}
@Override
public int hashCode()
{
return factorize(null).hashCode();
}
@Override
public String toString()
{
return factorize(null).toString();
}
}

View File

@ -0,0 +1,57 @@
/*
* 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.expression;
import org.apache.calcite.sql.SqlKind;
public abstract class AbstractExpressionConversion implements ExpressionConversion
{
private final SqlKind kind;
private final String operatorName;
public AbstractExpressionConversion(SqlKind kind)
{
this(kind, null);
}
public AbstractExpressionConversion(SqlKind kind, String operatorName)
{
this.kind = kind;
this.operatorName = operatorName;
if (kind == SqlKind.OTHER_FUNCTION && operatorName == null) {
throw new NullPointerException("operatorName must be non-null for kind OTHER_FUNCTION");
} else if (kind != SqlKind.OTHER_FUNCTION && operatorName != null) {
throw new NullPointerException("operatorName must be non-null for kind " + kind);
}
}
@Override
public SqlKind sqlKind()
{
return kind;
}
@Override
public String operatorName()
{
return operatorName;
}
}

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.expression;
import io.druid.query.extraction.StrlenExtractionFn;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import java.util.List;
public class CharLengthExpressionConversion extends AbstractExpressionConversion
{
private static final CharLengthExpressionConversion INSTANCE = new CharLengthExpressionConversion();
private CharLengthExpressionConversion()
{
super(SqlKind.OTHER_FUNCTION, "CHAR_LENGTH");
}
public static CharLengthExpressionConversion instance()
{
return INSTANCE;
}
@Override
public RowExtraction convert(
final ExpressionConverter converter,
final List<String> rowOrder,
final RexNode expression
)
{
final RexCall call = (RexCall) expression;
final RowExtraction arg = converter.convert(rowOrder, call.getOperands().get(0));
if (arg == null) {
return null;
}
return RowExtraction.of(
arg.getColumn(),
ExtractionFns.compose(StrlenExtractionFn.instance(), arg.getExtractionFn())
);
}
}

View File

@ -0,0 +1,54 @@
/*
* 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.expression;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import java.util.List;
public interface ExpressionConversion
{
/**
* SQL kind that this converter knows how to convert.
*
* @return sql kind
*/
SqlKind sqlKind();
/**
* Operator name, if {@link #sqlKind()} is {@code OTHER_FUNCTION}.
*
* @return operator name, or null
*/
String operatorName();
/**
* Translate a row-expression to a Druid column reference. Note that this signature will probably need to change
* once we support extractions from multiple columns.
*
* @param converter converter that can be used to convert sub-expressions
* @param rowOrder order of fields in the Druid rows to be extracted from
* @param expression expression meant to be applied on top of the table
*
* @return (columnName, extractionFn) or null
*/
RowExtraction convert(ExpressionConverter converter, List<String> rowOrder, RexNode expression);
}

View File

@ -0,0 +1,108 @@
/*
* 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.expression;
import com.google.common.collect.Maps;
import io.druid.java.util.common.ISE;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexInputRef;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import java.util.List;
import java.util.Map;
public class ExpressionConverter
{
private final Map<SqlKind, ExpressionConversion> kindMap;
private final Map<String, ExpressionConversion> otherFunctionMap;
private ExpressionConverter(
Map<SqlKind, ExpressionConversion> kindMap,
Map<String, ExpressionConversion> otherFunctionMap
)
{
this.kindMap = kindMap;
this.otherFunctionMap = otherFunctionMap;
}
public static ExpressionConverter create(final List<ExpressionConversion> conversions)
{
final Map<SqlKind, ExpressionConversion> kindMap = Maps.newHashMap();
final Map<String, ExpressionConversion> otherFunctionMap = Maps.newHashMap();
for (final ExpressionConversion conversion : conversions) {
if (conversion.sqlKind() != SqlKind.OTHER_FUNCTION) {
if (kindMap.put(conversion.sqlKind(), conversion) != null) {
throw new ISE("Oops, can't have two conversions for sqlKind[%s]", conversion.sqlKind());
}
} else {
// kind is OTHER_FUNCTION
if (otherFunctionMap.put(conversion.operatorName(), conversion) != null) {
throw new ISE(
"Oops, can't have two conversions for sqlKind[%s], operatorName[%s]",
conversion.sqlKind(),
conversion.operatorName()
);
}
}
}
return new ExpressionConverter(kindMap, otherFunctionMap);
}
/**
* Translate a row-expression to a Druid row extraction. Note that this signature will probably need to change
* once we support extractions from multiple columns.
*
* @param rowOrder order of fields in the Druid rows to be extracted from
* @param expression expression meant to be applied on top of the table
*
* @return (columnName, extractionFn) or null
*/
public RowExtraction convert(List<String> rowOrder, RexNode expression)
{
if (expression.getKind() == SqlKind.INPUT_REF) {
final RexInputRef ref = (RexInputRef) expression;
final String columnName = rowOrder.get(ref.getIndex());
if (columnName == null) {
throw new ISE("WTF?! Expression referred to nonexistent index[%d]", ref.getIndex());
}
return RowExtraction.of(columnName, null);
} else if (expression.getKind() == SqlKind.CAST) {
// TODO(gianm): Probably not a good idea to ignore CAST like this.
return convert(rowOrder, ((RexCall) expression).getOperands().get(0));
} else {
// Try conversion using an ExpressionConversion specific to this operator.
final RowExtraction retVal;
if (expression.getKind() == SqlKind.OTHER_FUNCTION) {
final ExpressionConversion conversion = otherFunctionMap.get(((RexCall) expression).getOperator().getName());
retVal = conversion != null ? conversion.convert(this, rowOrder, expression) : null;
} else {
final ExpressionConversion conversion = kindMap.get(expression.getKind());
retVal = conversion != null ? conversion.convert(this, rowOrder, expression) : null;
}
return retVal;
}
}
}

View File

@ -0,0 +1,519 @@
/*
* 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.expression;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.io.BaseEncoding;
import com.google.common.primitives.Chars;
import io.druid.granularity.QueryGranularity;
import io.druid.java.util.common.ISE;
import io.druid.math.expr.ExprType;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.aggregation.post.ArithmeticPostAggregator;
import io.druid.query.aggregation.post.ConstantPostAggregator;
import io.druid.query.aggregation.post.ExpressionPostAggregator;
import io.druid.query.aggregation.post.FieldAccessPostAggregator;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.LikeDimFilter;
import io.druid.query.filter.NotDimFilter;
import io.druid.query.filter.OrDimFilter;
import io.druid.query.ordering.StringComparator;
import io.druid.query.ordering.StringComparators;
import io.druid.segment.column.Column;
import io.druid.sql.calcite.aggregation.PostAggregatorFactory;
import io.druid.sql.calcite.filtration.Filtration;
import io.druid.sql.calcite.table.DruidTable;
import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexInputRef;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.SqlTypeName;
import java.util.Calendar;
import java.util.List;
import java.util.Map;
/**
* A collection of functions for translating from Calcite expressions into Druid objects.
*/
public class Expressions
{
private static final ExpressionConverter EXPRESSION_CONVERTER = ExpressionConverter.create(
ImmutableList.<ExpressionConversion>of(
CharLengthExpressionConversion.instance(),
ExtractExpressionConversion.instance(),
FloorExpressionConversion.instance(),
SubstringExpressionConversion.instance()
)
);
private static final Map<String, String> MATH_FUNCTIONS = ImmutableMap.<String, String>builder()
.put("ABS", "abs")
.put("CEIL", "ceil")
.put("EXP", "exp")
.put("FLOOR", "floor")
.put("LN", "log")
.put("LOG10", "log10")
.put("POWER", "pow")
.put("SQRT", "sqrt")
.build();
private static final Map<SqlTypeName, ExprType> MATH_TYPES;
static {
final ImmutableMap.Builder<SqlTypeName, ExprType> builder = ImmutableMap.builder();
for (SqlTypeName type : SqlTypeName.APPROX_TYPES) {
builder.put(type, ExprType.DOUBLE);
}
for (SqlTypeName type : SqlTypeName.EXACT_TYPES) {
builder.put(type, ExprType.LONG);
}
for (SqlTypeName type : SqlTypeName.STRING_TYPES) {
builder.put(type, ExprType.STRING);
}
MATH_TYPES = builder.build();
}
private Expressions()
{
// No instantiation.
}
/**
* Translate a field access, possibly through a projection, to an underlying Druid table.
*
* @param druidTable underlying Druid table
* @param project projection, or null
* @param fieldNumber number of the field to access
*
* @return row expression
*/
public static RexNode fromFieldAccess(
final DruidTable druidTable,
final Project project,
final int fieldNumber
)
{
if (project == null) {
// I don't think the factory impl matters here.
return RexInputRef.of(fieldNumber, druidTable.getRowType(new JavaTypeFactoryImpl()));
} else {
return project.getChildExps().get(fieldNumber);
}
}
/**
* Translate a Calcite row-expression to a Druid row extraction. Note that this signature will probably need to
* change once we support extractions from multiple columns.
*
* @param rowOrder order of fields in the Druid rows to be extracted from
* @param expression expression meant to be applied on top of the rows
*
* @return RowExtraction or null if not possible
*/
public static RowExtraction toRowExtraction(
final List<String> rowOrder,
final RexNode expression
)
{
return EXPRESSION_CONVERTER.convert(rowOrder, expression);
}
/**
* Translate a Calcite row-expression to a Druid PostAggregator. One day, when possible, this could be folded
* into {@link #toRowExtraction(List, RexNode)}.
*
* @param name name of the PostAggregator
* @param rowOrder order of fields in the Druid rows to be extracted from
* @param finalizingPostAggregatorFactories post-aggregators that should be used for specific entries in rowOrder.
* May be empty, and individual values may be null. Missing or null values
* will lead to creation of {@link FieldAccessPostAggregator}.
* @param expression expression meant to be applied on top of the rows
*
* @return PostAggregator or null if not possible
*/
public static PostAggregator toPostAggregator(
final String name,
final List<String> rowOrder,
final List<PostAggregatorFactory> finalizingPostAggregatorFactories,
final RexNode expression
)
{
final PostAggregator retVal;
if (expression.getKind() == SqlKind.INPUT_REF) {
final RexInputRef ref = (RexInputRef) expression;
final PostAggregatorFactory finalizingPostAggregatorFactory = finalizingPostAggregatorFactories.get(ref.getIndex());
retVal = finalizingPostAggregatorFactory != null
? finalizingPostAggregatorFactory.factorize(name)
: new FieldAccessPostAggregator(name, rowOrder.get(ref.getIndex()));
} else if (expression.getKind() == SqlKind.CAST) {
// Ignore CAST when translating to PostAggregators and hope for the best. They are really loosey-goosey with
// types internally and there isn't much we can do to respect
// TODO(gianm): Probably not a good idea to ignore CAST like this.
final RexNode operand = ((RexCall) expression).getOperands().get(0);
retVal = toPostAggregator(name, rowOrder, finalizingPostAggregatorFactories, operand);
} else if (expression.getKind() == SqlKind.LITERAL
&& SqlTypeName.NUMERIC_TYPES.contains(expression.getType().getSqlTypeName())) {
retVal = new ConstantPostAggregator(name, (Number) RexLiteral.value(expression));
} else if (expression.getKind() == SqlKind.TIMES
|| expression.getKind() == SqlKind.DIVIDE
|| expression.getKind() == SqlKind.PLUS
|| expression.getKind() == SqlKind.MINUS) {
final String fnName = ImmutableMap.<SqlKind, String>builder()
.put(SqlKind.TIMES, "*")
.put(SqlKind.DIVIDE, "quotient")
.put(SqlKind.PLUS, "+")
.put(SqlKind.MINUS, "-")
.build().get(expression.getKind());
final List<PostAggregator> operands = Lists.newArrayList();
for (RexNode operand : ((RexCall) expression).getOperands()) {
final PostAggregator translatedOperand = toPostAggregator(
null,
rowOrder,
finalizingPostAggregatorFactories,
operand
);
if (translatedOperand == null) {
return null;
}
operands.add(translatedOperand);
}
retVal = new ArithmeticPostAggregator(name, fnName, operands);
} else {
// Try converting to a math expression.
final String mathExpression = Expressions.toMathExpression(rowOrder, expression);
if (mathExpression == null) {
retVal = null;
} else {
retVal = new ExpressionPostAggregator(name, mathExpression);
}
}
if (retVal != null && name != null && !name.equals(retVal.getName())) {
throw new ISE("WTF?! Was about to return a PostAggregator with bad name, [%s] != [%s]", name, retVal.getName());
}
return retVal;
}
/**
* Translate a row-expression to a Druid math expression. One day, when possible, this could be folded into
* {@link #toRowExtraction(List, RexNode)}.
*
* @param rowOrder order of fields in the Druid rows to be extracted from
* @param expression expression meant to be applied on top of the rows
*
* @return expression referring to fields in rowOrder, or null if not possible
*/
public static String toMathExpression(
final List<String> rowOrder,
final RexNode expression
)
{
final SqlKind kind = expression.getKind();
final SqlTypeName sqlTypeName = expression.getType().getSqlTypeName();
if (kind == SqlKind.INPUT_REF) {
// Translate field references.
final RexInputRef ref = (RexInputRef) expression;
final String columnName = rowOrder.get(ref.getIndex());
if (columnName == null) {
throw new ISE("WTF?! Expression referred to nonexistent index[%d]", ref.getIndex());
}
return String.format("\"%s\"", escape(columnName));
} else if (kind == SqlKind.CAST || kind == SqlKind.REINTERPRET) {
// Translate casts.
final RexNode operand = ((RexCall) expression).getOperands().get(0);
final String operandExpression = toMathExpression(rowOrder, operand);
if (operandExpression == null) {
return null;
}
final ExprType fromType = MATH_TYPES.get(operand.getType().getSqlTypeName());
final ExprType toType = MATH_TYPES.get(sqlTypeName);
if (fromType != toType) {
return String.format("CAST(%s, '%s')", operandExpression, toType.toString());
} else {
return operandExpression;
}
} else if (kind == SqlKind.TIMES || kind == SqlKind.DIVIDE || kind == SqlKind.PLUS || kind == SqlKind.MINUS) {
// Translate simple arithmetic.
final List<RexNode> operands = ((RexCall) expression).getOperands();
final String lhsExpression = toMathExpression(rowOrder, operands.get(0));
final String rhsExpression = toMathExpression(rowOrder, operands.get(1));
if (lhsExpression == null || rhsExpression == null) {
return null;
}
final String op = ImmutableMap.of(
SqlKind.TIMES, "*",
SqlKind.DIVIDE, "/",
SqlKind.PLUS, "+",
SqlKind.MINUS, "-"
).get(kind);
return String.format("(%s %s %s)", lhsExpression, op, rhsExpression);
} else if (kind == SqlKind.OTHER_FUNCTION) {
final String calciteFunction = ((RexCall) expression).getOperator().getName();
final String druidFunction = MATH_FUNCTIONS.get(calciteFunction);
final List<String> functionArgs = Lists.newArrayList();
for (final RexNode operand : ((RexCall) expression).getOperands()) {
final String operandExpression = toMathExpression(rowOrder, operand);
if (operandExpression == null) {
return null;
}
functionArgs.add(operandExpression);
}
if ("MOD".equals(calciteFunction)) {
// Special handling for MOD, which is a function in Calcite but a binary operator in Druid.
Preconditions.checkState(functionArgs.size() == 2, "WTF?! Expected 2 args for MOD.");
return String.format("(%s %s %s)", functionArgs.get(0), "%", functionArgs.get(1));
}
if (druidFunction == null) {
return null;
}
return String.format("%s(%s)", druidFunction, Joiner.on(", ").join(functionArgs));
} else if (kind == SqlKind.LITERAL) {
// Translate literal.
if (SqlTypeName.NUMERIC_TYPES.contains(sqlTypeName)) {
// Include literal numbers as-is.
return String.valueOf(RexLiteral.value(expression));
} else if (SqlTypeName.STRING_TYPES.contains(sqlTypeName)) {
// Quote literal strings.
return "\'" + escape(RexLiteral.stringValue(expression)) + "\'";
} else {
// Can't translate other literals.
return null;
}
} else {
// Can't translate other kinds of expressions.
return null;
}
}
/**
* Translates "condition" to a Druid filter, or returns null if we cannot translate the condition.
*
* @param druidTable Druid table, if the rows come from a table scan; null otherwise
* @param rowOrder order of columns in the rows to be filtered
* @param expression Calcite row expression
*/
public static DimFilter toFilter(
final DruidTable druidTable,
final List<String> rowOrder,
final RexNode expression
)
{
if (expression.getKind() == SqlKind.AND
|| expression.getKind() == SqlKind.OR
|| expression.getKind() == SqlKind.NOT) {
final List<DimFilter> filters = Lists.newArrayList();
for (final RexNode rexNode : ((RexCall) expression).getOperands()) {
final DimFilter nextFilter = toFilter(druidTable, rowOrder, rexNode);
if (nextFilter == null) {
return null;
}
filters.add(nextFilter);
}
if (expression.getKind() == SqlKind.AND) {
return new AndDimFilter(filters);
} else if (expression.getKind() == SqlKind.OR) {
return new OrDimFilter(filters);
} else {
assert expression.getKind() == SqlKind.NOT;
return new NotDimFilter(Iterables.getOnlyElement(filters));
}
} else {
// Handle filter conditions on everything else.
return toLeafFilter(druidTable, rowOrder, expression);
}
}
/**
* Translates "condition" to a Druid filter, assuming it does not contain any boolean expressions. Returns null
* if we cannot translate the condition.
*
* @param druidTable Druid table, if the rows come from a table scan; null otherwise
* @param rowOrder order of columns in the rows to be filtered
* @param expression Calcite row expression
*/
private static DimFilter toLeafFilter(
final DruidTable druidTable,
final List<String> rowOrder,
final RexNode expression
)
{
if (expression.isAlwaysTrue()) {
return Filtration.matchEverything();
} else if (expression.isAlwaysFalse()) {
return Filtration.matchNothing();
}
final SqlKind kind = expression.getKind();
if (kind == SqlKind.LIKE) {
final List<RexNode> operands = ((RexCall) expression).getOperands();
final RowExtraction rex = EXPRESSION_CONVERTER.convert(rowOrder, operands.get(0));
if (rex == null || !rex.isFilterable(druidTable)) {
return null;
}
return new LikeDimFilter(
rex.getColumn(),
RexLiteral.stringValue(operands.get(1)),
operands.size() > 2 ? RexLiteral.stringValue(operands.get(2)) : null,
rex.getExtractionFn()
);
} else if (kind == SqlKind.EQUALS
|| kind == SqlKind.NOT_EQUALS
|| kind == SqlKind.GREATER_THAN
|| kind == SqlKind.GREATER_THAN_OR_EQUAL
|| kind == SqlKind.LESS_THAN
|| kind == SqlKind.LESS_THAN_OR_EQUAL) {
final List<RexNode> operands = ((RexCall) expression).getOperands();
Preconditions.checkState(operands.size() == 2, "WTF?! Expected 2 operands, got[%,d]", operands.size());
boolean flip = false;
RexNode lhs = operands.get(0);
RexNode rhs = operands.get(1);
if (lhs.getKind() == SqlKind.LITERAL && rhs.getKind() != SqlKind.LITERAL) {
// swap lhs, rhs
RexNode x = lhs;
lhs = rhs;
rhs = x;
flip = true;
}
// rhs must be a literal
if (rhs.getKind() != SqlKind.LITERAL) {
return null;
}
// lhs must be translatable to a RowExtraction to be filterable
final RowExtraction rex = EXPRESSION_CONVERTER.convert(rowOrder, lhs);
if (rex == null || !rex.isFilterable(druidTable)) {
return null;
}
final String column = rex.getColumn();
final ExtractionFn extractionFn = rex.getExtractionFn();
if (column.equals(Column.TIME_COLUMN_NAME) && ExtractionFns.toQueryGranularity(extractionFn) != null) {
// lhs is FLOOR(__time TO gran); convert to range
final QueryGranularity gran = ExtractionFns.toQueryGranularity(extractionFn);
final long rhsMillis = ((Calendar) RexLiteral.value(rhs)).getTimeInMillis();
if (gran.truncate(rhsMillis) != rhsMillis) {
// Nothing matches.
return Filtration.matchNothing();
} else {
// Match any __time within the granular bucket.
return new BoundDimFilter(
Column.TIME_COLUMN_NAME,
String.valueOf(gran.truncate(rhsMillis)),
String.valueOf(gran.next(gran.truncate(rhsMillis))),
false,
true,
null,
null,
StringComparators.NUMERIC
);
}
}
final String val;
final RexLiteral rhsLiteral = (RexLiteral) rhs;
if (SqlTypeName.NUMERIC_TYPES.contains(rhsLiteral.getTypeName())) {
val = String.valueOf(RexLiteral.value(rhsLiteral));
} else if (rhsLiteral.getTypeName() == SqlTypeName.CHAR) {
val = String.valueOf(RexLiteral.stringValue(rhsLiteral));
} else if (SqlTypeName.DATETIME_TYPES.contains(rhsLiteral.getTypeName())) {
val = String.valueOf(((Calendar) RexLiteral.value(rhsLiteral)).getTimeInMillis());
} else {
// Hope for the best.
val = String.valueOf(RexLiteral.value(rhsLiteral));
}
// Numeric lhs needs a numeric comparison.
final boolean lhsIsNumeric = SqlTypeName.NUMERIC_TYPES.contains(lhs.getType().getSqlTypeName())
|| SqlTypeName.DATETIME_TYPES.contains(lhs.getType().getSqlTypeName());
final StringComparator comparator = lhsIsNumeric ? StringComparators.NUMERIC : StringComparators.LEXICOGRAPHIC;
final DimFilter filter;
// Always use BoundDimFilters, to simplify filter optimization later (it helps to remember the comparator).
if (kind == SqlKind.EQUALS) {
filter = new BoundDimFilter(column, val, val, false, false, null, extractionFn, comparator);
} else if (kind == SqlKind.NOT_EQUALS) {
filter = new NotDimFilter(
new BoundDimFilter(column, val, val, false, false, null, extractionFn, comparator)
);
} else if ((!flip && kind == SqlKind.GREATER_THAN) || (flip && kind == SqlKind.LESS_THAN)) {
filter = new BoundDimFilter(column, val, null, true, false, null, extractionFn, comparator);
} else if ((!flip && kind == SqlKind.GREATER_THAN_OR_EQUAL) || (flip && kind == SqlKind.LESS_THAN_OR_EQUAL)) {
filter = new BoundDimFilter(column, val, null, false, false, null, extractionFn, comparator);
} else if ((!flip && kind == SqlKind.LESS_THAN) || (flip && kind == SqlKind.GREATER_THAN)) {
filter = new BoundDimFilter(column, null, val, false, true, null, extractionFn, comparator);
} else if ((!flip && kind == SqlKind.LESS_THAN_OR_EQUAL) || (flip && kind == SqlKind.GREATER_THAN_OR_EQUAL)) {
filter = new BoundDimFilter(column, null, val, false, false, null, extractionFn, comparator);
} else {
throw new IllegalStateException("WTF?! Shouldn't have got here...");
}
return filter;
} else {
return null;
}
}
private static String escape(final String s)
{
final StringBuilder escaped = new StringBuilder();
for (int i = 0; i < s.length(); i++) {
final char c = s.charAt(i);
if (Character.isLetterOrDigit(c) || Character.isWhitespace(c)) {
escaped.append(c);
} else {
escaped.append("\\u").append(BaseEncoding.base16().encode(Chars.toByteArray(c)));
}
}
return escaped.toString();
}
}

View File

@ -0,0 +1,87 @@
/*
* 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.expression;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.TimeFormatExtractionFn;
import org.apache.calcite.avatica.util.TimeUnitRange;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import java.util.List;
public class ExtractExpressionConversion extends AbstractExpressionConversion
{
private static final ExtractExpressionConversion INSTANCE = new ExtractExpressionConversion();
private ExtractExpressionConversion()
{
super(SqlKind.EXTRACT);
}
public static ExtractExpressionConversion instance()
{
return INSTANCE;
}
@Override
public RowExtraction convert(
final ExpressionConverter converter,
final List<String> rowOrder,
final RexNode expression
)
{
// EXTRACT(timeUnit FROM expr)
final RexCall call = (RexCall) expression;
final RexLiteral flag = (RexLiteral) call.getOperands().get(0);
final TimeUnitRange timeUnit = (TimeUnitRange) flag.getValue();
final RexNode expr = call.getOperands().get(1);
final RowExtraction rex = converter.convert(rowOrder, expr);
if (rex == null) {
return null;
}
final String dateTimeFormat = TimeUnits.toDateTimeFormat(timeUnit);
if (dateTimeFormat == null) {
return null;
}
final ExtractionFn baseExtractionFn;
if (call.getOperator().getName().equals("EXTRACT_DATE")) {
// Expr will be in number of days since the epoch. Can't translate.
return null;
} else {
// Expr will be in millis since the epoch
baseExtractionFn = rex.getExtractionFn();
}
return RowExtraction.of(
rex.getColumn(),
ExtractionFns.compose(
new TimeFormatExtractionFn(dateTimeFormat, null, null, null, true),
baseExtractionFn
)
);
}
}

View File

@ -0,0 +1,87 @@
/*
* 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.expression;
import com.google.common.collect.Lists;
import io.druid.granularity.QueryGranularity;
import io.druid.query.extraction.CascadeExtractionFn;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.TimeFormatExtractionFn;
import java.util.Arrays;
import java.util.List;
public class ExtractionFns
{
/**
* Converts extractionFn to a QueryGranularity, if possible.
*
* @param extractionFn function
*
* @return
*/
public static QueryGranularity toQueryGranularity(final ExtractionFn extractionFn)
{
if (extractionFn instanceof TimeFormatExtractionFn) {
final TimeFormatExtractionFn fn = (TimeFormatExtractionFn) extractionFn;
if (fn.getFormat() == null && fn.getTimeZone() == null && fn.getLocale() == null) {
return fn.getGranularity();
}
}
return null;
}
/**
* Compose f and g, returning an ExtractionFn that computes f(g(x)). Null f or g are treated like identity functions.
*
* @param f function
* @param g function
*
* @return composed function, or null if both f and g were null
*/
public static ExtractionFn compose(final ExtractionFn f, final ExtractionFn g)
{
if (f == null) {
// Treat null like identity.
return g;
} else if (g == null) {
return f;
} else {
final List<ExtractionFn> extractionFns = Lists.newArrayList();
// Apply g, then f, unwrapping if they are already cascades.
if (g instanceof CascadeExtractionFn) {
extractionFns.addAll(Arrays.asList(((CascadeExtractionFn) g).getExtractionFns()));
} else {
extractionFns.add(g);
}
if (f instanceof CascadeExtractionFn) {
extractionFns.addAll(Arrays.asList(((CascadeExtractionFn) f).getExtractionFns()));
} else {
extractionFns.add(f);
}
return new CascadeExtractionFn(extractionFns.toArray(new ExtractionFn[extractionFns.size()]));
}
}
}

View File

@ -0,0 +1,89 @@
/*
* 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.expression;
import io.druid.granularity.QueryGranularity;
import io.druid.query.extraction.BucketExtractionFn;
import io.druid.query.extraction.TimeFormatExtractionFn;
import org.apache.calcite.avatica.util.TimeUnitRange;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import java.util.List;
public class FloorExpressionConversion extends AbstractExpressionConversion
{
private static final FloorExpressionConversion INSTANCE = new FloorExpressionConversion();
private FloorExpressionConversion()
{
super(SqlKind.FLOOR);
}
public static FloorExpressionConversion instance()
{
return INSTANCE;
}
@Override
public RowExtraction convert(
final ExpressionConverter converter,
final List<String> rowOrder,
final RexNode expression
)
{
final RexCall call = (RexCall) expression;
final RexNode arg = call.getOperands().get(0);
final RowExtraction rex = converter.convert(rowOrder, arg);
if (rex == null) {
return null;
} else if (call.getOperands().size() == 1) {
// FLOOR(expr)
return RowExtraction.of(
rex.getColumn(),
ExtractionFns.compose(new BucketExtractionFn(1.0, 0.0), rex.getExtractionFn())
);
} else if (call.getOperands().size() == 2) {
// FLOOR(expr TO timeUnit)
final RexLiteral flag = (RexLiteral) call.getOperands().get(1);
final TimeUnitRange timeUnit = (TimeUnitRange) flag.getValue();
final QueryGranularity queryGranularity = TimeUnits.toQueryGranularity(timeUnit);
if (queryGranularity != null) {
return RowExtraction.of(
rex.getColumn(),
ExtractionFns.compose(
new TimeFormatExtractionFn(null, null, null, queryGranularity, true),
rex.getExtractionFn()
)
);
} else {
// We don't have a queryGranularity for this timeUnit.
return null;
}
} else {
// WTF? FLOOR with 3 arguments?
return null;
}
}
}

View File

@ -0,0 +1,185 @@
/*
* 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.expression;
import com.google.common.base.Preconditions;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.dimension.ExtractionDimensionSpec;
import io.druid.query.extraction.ExtractionFn;
import io.druid.segment.column.Column;
import io.druid.segment.column.ValueType;
import io.druid.segment.filter.Filters;
import io.druid.sql.calcite.rel.DruidQueryBuilder;
import io.druid.sql.calcite.table.DruidTable;
/**
* Represents an extraction of a value from a Druid row. Can be used for grouping, filtering, etc.
*
* Currently this is a column plus an extractionFn, but it's expected that as time goes on, this will become more
* general and allow for variously-typed extractions from multiple columns.
*/
public class RowExtraction
{
private final String column;
private final ExtractionFn extractionFn;
public RowExtraction(String column, ExtractionFn extractionFn)
{
this.column = Preconditions.checkNotNull(column, "column");
this.extractionFn = extractionFn;
}
public static RowExtraction of(String column, ExtractionFn extractionFn)
{
return new RowExtraction(column, extractionFn);
}
public static RowExtraction fromDimensionSpec(final DimensionSpec dimensionSpec)
{
if (dimensionSpec instanceof ExtractionDimensionSpec) {
return RowExtraction.of(
dimensionSpec.getDimension(),
((ExtractionDimensionSpec) dimensionSpec).getExtractionFn()
);
} else if (dimensionSpec instanceof DefaultDimensionSpec) {
return RowExtraction.of(dimensionSpec.getDimension(), null);
} else {
return null;
}
}
public static RowExtraction fromQueryBuilder(
final DruidQueryBuilder queryBuilder,
final int fieldNumber
)
{
final String fieldName = queryBuilder.getRowOrder().get(fieldNumber);
if (queryBuilder.getGrouping() != null) {
for (DimensionSpec dimensionSpec : queryBuilder.getGrouping().getDimensions()) {
if (dimensionSpec.getOutputName().equals(fieldName)) {
return RowExtraction.fromDimensionSpec(dimensionSpec);
}
}
return null;
} else if (queryBuilder.getSelectProjection() != null) {
for (DimensionSpec dimensionSpec : queryBuilder.getSelectProjection().getDimensions()) {
if (dimensionSpec.getOutputName().equals(fieldName)) {
return RowExtraction.fromDimensionSpec(dimensionSpec);
}
}
for (String metricName : queryBuilder.getSelectProjection().getMetrics()) {
if (metricName.equals(fieldName)) {
return RowExtraction.of(metricName, null);
}
}
return null;
} else {
// No select projection or grouping.
return RowExtraction.of(queryBuilder.getRowOrder().get(fieldNumber), null);
}
}
public String getColumn()
{
return column;
}
public ExtractionFn getExtractionFn()
{
return extractionFn;
}
/**
* Check if this extraction can be used to build a filter on a Druid table. This method exists because we can't
* filter on floats (yet) and things like DruidFilterRule need to check for that.
*
* If a null table is passed in, this method always returns true.
*
* @param druidTable Druid table, or null
*
* @return whether or not this extraction is filterable; will be true if druidTable is null
*/
public boolean isFilterable(final DruidTable druidTable)
{
return druidTable == null ||
Filters.FILTERABLE_TYPES.contains(druidTable.getColumnType(druidTable.getColumnNumber(column)));
}
public DimensionSpec toDimensionSpec(final DruidTable druidTable, final String outputName)
{
final int columnNumber = druidTable.getColumnNumber(column);
if (columnNumber < 0) {
return null;
}
final ValueType columnType = druidTable.getColumnType(columnNumber);
if (columnType == ValueType.STRING || (column.equals(Column.TIME_COLUMN_NAME) && extractionFn != null)) {
return extractionFn == null
? new DefaultDimensionSpec(column, outputName)
: new ExtractionDimensionSpec(column, outputName, extractionFn);
} else {
// Can't create dimensionSpecs for non-string, non-time.
return null;
}
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
RowExtraction that = (RowExtraction) o;
if (column != null ? !column.equals(that.column) : that.column != null) {
return false;
}
return extractionFn != null ? extractionFn.equals(that.extractionFn) : that.extractionFn == null;
}
@Override
public int hashCode()
{
int result = column != null ? column.hashCode() : 0;
result = 31 * result + (extractionFn != null ? extractionFn.hashCode() : 0);
return result;
}
@Override
public String toString()
{
if (extractionFn != null) {
return String.format("%s(%s)", extractionFn, column);
} else {
return column;
}
}
}

View File

@ -0,0 +1,71 @@
/*
* 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.expression;
import io.druid.query.extraction.SubstringDimExtractionFn;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import java.util.List;
public class SubstringExpressionConversion extends AbstractExpressionConversion
{
private static final SubstringExpressionConversion INSTANCE = new SubstringExpressionConversion();
private SubstringExpressionConversion()
{
super(SqlKind.OTHER_FUNCTION, "SUBSTRING");
}
public static SubstringExpressionConversion instance()
{
return INSTANCE;
}
@Override
public RowExtraction convert(
final ExpressionConverter converter,
final List<String> rowOrder,
final RexNode expression
)
{
final RexCall call = (RexCall) expression;
final RowExtraction arg = converter.convert(rowOrder, call.getOperands().get(0));
if (arg == null) {
return null;
}
final int index = RexLiteral.intValue(call.getOperands().get(1)) - 1;
final Integer length;
if (call.getOperands().size() > 2) {
length = RexLiteral.intValue(call.getOperands().get(2));
} else {
length = null;
}
return RowExtraction.of(arg.getColumn(),
ExtractionFns.compose(
new SubstringDimExtractionFn(index, length),
arg.getExtractionFn()
)
);
}
}

View File

@ -0,0 +1,76 @@
/*
* 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.expression;
import com.google.common.collect.ImmutableMap;
import io.druid.granularity.QueryGranularities;
import io.druid.granularity.QueryGranularity;
import org.apache.calcite.avatica.util.TimeUnitRange;
import java.util.Map;
public class TimeUnits
{
private static final Map<TimeUnitRange, QueryGranularity> QUERY_GRANULARITY_MAP = ImmutableMap.<TimeUnitRange, QueryGranularity>builder()
.put(TimeUnitRange.SECOND, QueryGranularities.SECOND)
.put(TimeUnitRange.MINUTE, QueryGranularities.MINUTE)
.put(TimeUnitRange.HOUR, QueryGranularities.HOUR)
.put(TimeUnitRange.DAY, QueryGranularities.DAY)
.put(TimeUnitRange.WEEK, QueryGranularities.WEEK)
.put(TimeUnitRange.MONTH, QueryGranularities.MONTH)
.put(TimeUnitRange.QUARTER, QueryGranularities.QUARTER)
.put(TimeUnitRange.YEAR, QueryGranularities.YEAR)
.build();
// Note that QUARTER is not supported here.
private static final Map<TimeUnitRange, String> EXTRACT_FORMAT_MAP = ImmutableMap.<TimeUnitRange, String>builder()
.put(TimeUnitRange.SECOND, "s")
.put(TimeUnitRange.MINUTE, "m")
.put(TimeUnitRange.HOUR, "H")
.put(TimeUnitRange.DAY, "d")
.put(TimeUnitRange.WEEK, "w")
.put(TimeUnitRange.MONTH, "M")
.put(TimeUnitRange.YEAR, "Y")
.build();
/**
* Returns the Druid QueryGranularity corresponding to a Calcite TimeUnitRange, or null if there is none.
*
* @param timeUnitRange timeUnit
*
* @return queryGranularity, or null
*/
public static QueryGranularity toQueryGranularity(final TimeUnitRange timeUnitRange)
{
return QUERY_GRANULARITY_MAP.get(timeUnitRange);
}
/**
* Returns the Joda format string corresponding to extracting on a Calcite TimeUnitRange, or null if there is none.
*
* @param timeUnitRange timeUnit
*
* @return queryGranularity, or null
*/
public static String toDateTimeFormat(final TimeUnitRange timeUnitRange)
{
return EXTRACT_FORMAT_MAP.get(timeUnitRange);
}
}

View File

@ -0,0 +1,95 @@
/*
* 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.filtration;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.NotDimFilter;
import io.druid.query.filter.OrDimFilter;
import java.util.List;
public abstract class BottomUpTransform implements Function<Filtration, Filtration>
{
protected abstract DimFilter process(final DimFilter filter);
private DimFilter checkedProcess(final DimFilter filter)
{
final DimFilter retVal = process(Preconditions.checkNotNull(filter, "filter"));
return Preconditions.checkNotNull(retVal, "process(filter) result in %s", getClass().getSimpleName());
}
@Override
public Filtration apply(final Filtration filtration)
{
if (filtration.getDimFilter() != null) {
final Filtration retVal = Filtration.create(apply0(filtration.getDimFilter()), filtration.getIntervals());
return filtration.equals(retVal) ? retVal : apply(retVal);
} else {
return filtration;
}
}
private DimFilter apply0(final DimFilter filter)
{
if (filter instanceof AndDimFilter) {
final List<DimFilter> oldFilters = ((AndDimFilter) filter).getFields();
final List<DimFilter> newFilters = Lists.newArrayList();
for (DimFilter oldFilter : oldFilters) {
final DimFilter newFilter = apply0(oldFilter);
if (newFilter != null) {
newFilters.add(newFilter);
}
}
if (!newFilters.equals(oldFilters)) {
return checkedProcess(new AndDimFilter(newFilters));
} else {
return checkedProcess(filter);
}
} else if (filter instanceof OrDimFilter) {
final List<DimFilter> oldFilters = ((OrDimFilter) filter).getFields();
final List<DimFilter> newFilters = Lists.newArrayList();
for (DimFilter oldFilter : oldFilters) {
final DimFilter newFilter = apply0(oldFilter);
if (newFilter != null) {
newFilters.add(newFilter);
}
}
if (!newFilters.equals(oldFilters)) {
return checkedProcess(new OrDimFilter(newFilters));
} else {
return checkedProcess(filter);
}
} else if (filter instanceof NotDimFilter) {
final DimFilter oldFilter = ((NotDimFilter) filter).getField();
final DimFilter newFilter = apply0(oldFilter);
if (!oldFilter.equals(newFilter)) {
return checkedProcess(new NotDimFilter(newFilter));
} else {
return checkedProcess(filter);
}
} else {
return checkedProcess(filter);
}
}
}

View File

@ -0,0 +1,112 @@
/*
* 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.filtration;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.ordering.StringComparator;
public class BoundRefKey
{
private final String dimension;
private final ExtractionFn extractionFn;
private final StringComparator comparator;
public BoundRefKey(String dimension, ExtractionFn extractionFn, StringComparator comparator)
{
this.dimension = dimension;
this.extractionFn = extractionFn;
this.comparator = comparator;
}
public static BoundRefKey from(BoundDimFilter filter)
{
return new BoundRefKey(
filter.getDimension(),
filter.getExtractionFn(),
filter.getOrdering()
);
}
public static BoundRefKey from(SelectorDimFilter filter, StringComparator comparator)
{
return new BoundRefKey(
filter.getDimension(),
filter.getExtractionFn(),
comparator
);
}
public String getDimension()
{
return dimension;
}
public ExtractionFn getExtractionFn()
{
return extractionFn;
}
public StringComparator getComparator()
{
return comparator;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
BoundRefKey boundRefKey = (BoundRefKey) o;
if (dimension != null ? !dimension.equals(boundRefKey.dimension) : boundRefKey.dimension != null) {
return false;
}
if (extractionFn != null ? !extractionFn.equals(boundRefKey.extractionFn) : boundRefKey.extractionFn != null) {
return false;
}
return comparator != null ? comparator.equals(boundRefKey.comparator) : boundRefKey.comparator == null;
}
@Override
public int hashCode()
{
int result = dimension != null ? dimension.hashCode() : 0;
result = 31 * result + (extractionFn != null ? extractionFn.hashCode() : 0);
result = 31 * result + (comparator != null ? comparator.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "BoundRefKey{" +
"dimension='" + dimension + '\'' +
", extractionFn=" + extractionFn +
", comparator=" + comparator +
'}';
}
}

View File

@ -0,0 +1,87 @@
/*
* 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.filtration;
import io.druid.java.util.common.ISE;
import io.druid.query.ordering.StringComparator;
public class BoundValue implements Comparable<BoundValue>
{
private final String value;
private final StringComparator comparator;
public BoundValue(String value, StringComparator comparator)
{
this.value = value;
this.comparator = comparator;
}
public String getValue()
{
return value;
}
public StringComparator getComparator()
{
return comparator;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
BoundValue that = (BoundValue) o;
if (value != null ? !value.equals(that.value) : that.value != null) {
return false;
}
return comparator != null ? comparator.equals(that.comparator) : that.comparator == null;
}
@Override
public int hashCode()
{
int result = value != null ? value.hashCode() : 0;
result = 31 * result + (comparator != null ? comparator.hashCode() : 0);
return result;
}
@Override
public int compareTo(BoundValue o)
{
if (!comparator.equals(o.comparator)) {
throw new ISE("WTF?! Comparator mismatch?!");
}
return comparator.compare(value, o.value);
}
@Override
public String toString()
{
return value;
}
}

View File

@ -0,0 +1,117 @@
/*
* 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.filtration;
import com.google.common.base.Function;
import com.google.common.collect.BoundType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Range;
import io.druid.query.filter.BoundDimFilter;
import java.util.List;
public class Bounds
{
/**
* Negates single-ended Bound filters.
*
* @param bound filter
*
* @return negated filter, or null if this bound is double-ended.
*/
public static BoundDimFilter not(final BoundDimFilter bound)
{
if (bound.getUpper() != null && bound.getLower() != null) {
return null;
} else if (bound.getUpper() != null) {
return new BoundDimFilter(
bound.getDimension(),
bound.getUpper(),
null,
!bound.isUpperStrict(),
false,
null,
bound.getExtractionFn(),
bound.getOrdering()
);
} else {
// bound.getLower() != null
return new BoundDimFilter(
bound.getDimension(),
null,
bound.getLower(),
false,
!bound.isLowerStrict(),
null,
bound.getExtractionFn(),
bound.getOrdering()
);
}
}
public static Range<BoundValue> toRange(final BoundDimFilter bound)
{
final BoundValue upper = bound.getUpper() != null ? new BoundValue(bound.getUpper(), bound.getOrdering()) : null;
final BoundValue lower = bound.getLower() != null ? new BoundValue(bound.getLower(), bound.getOrdering()) : null;
if (lower == null) {
return bound.isUpperStrict() ? Range.lessThan(upper) : Range.atMost(upper);
} else if (upper == null) {
return bound.isLowerStrict() ? Range.greaterThan(lower) : Range.atLeast(lower);
} else {
return Range.range(
lower, bound.isLowerStrict() ? BoundType.OPEN : BoundType.CLOSED,
upper, bound.isUpperStrict() ? BoundType.OPEN : BoundType.CLOSED
);
}
}
public static List<Range<BoundValue>> toRanges(final List<BoundDimFilter> bounds)
{
return ImmutableList.copyOf(
Lists.transform(
bounds,
new Function<BoundDimFilter, Range<BoundValue>>()
{
@Override
public Range<BoundValue> apply(BoundDimFilter bound)
{
return toRange(bound);
}
}
)
);
}
public static BoundDimFilter toFilter(final BoundRefKey boundRefKey, final Range<BoundValue> range)
{
return new BoundDimFilter(
boundRefKey.getDimension(),
range.hasLowerBound() ? range.lowerEndpoint().getValue() : null,
range.hasUpperBound() ? range.upperEndpoint().getValue() : null,
range.hasLowerBound() && range.lowerBoundType() == BoundType.OPEN,
range.hasUpperBound() && range.upperBoundType() == BoundType.OPEN,
null,
boundRefKey.getExtractionFn(),
boundRefKey.getComparator()
);
}
}

View File

@ -0,0 +1,230 @@
/*
* 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.filtration;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Range;
import com.google.common.collect.RangeSet;
import io.druid.java.util.common.ISE;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.NotDimFilter;
import io.druid.query.filter.OrDimFilter;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
public class CombineAndSimplifyBounds extends BottomUpTransform
{
private static final CombineAndSimplifyBounds INSTANCE = new CombineAndSimplifyBounds();
private CombineAndSimplifyBounds()
{
}
public static CombineAndSimplifyBounds instance()
{
return INSTANCE;
}
@Override
public DimFilter process(DimFilter filter)
{
if (filter instanceof AndDimFilter) {
final List<DimFilter> children = ((AndDimFilter) filter).getFields();
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 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());
} else {
return filter;
}
}
private static DimFilter doSimplifyAnd(final List<DimFilter> children)
{
return doSimplify(children, false);
}
private static DimFilter doSimplifyOr(final List<DimFilter> children)
{
return doSimplify(children, true);
}
/**
* Simplify BoundDimFilters that are children of an OR or an AND.
*
* @param children the filters
* @param disjunction true for disjunction, false for conjunction
*
* @return simplified filters
*/
private static DimFilter doSimplify(final List<DimFilter> children, boolean disjunction)
{
// Copy children list
final List<DimFilter> newChildren = Lists.newArrayList(children);
// Group Bound filters by dimension, extractionFn, and comparator and compute a RangeSet for each one.
final Map<BoundRefKey, List<BoundDimFilter>> bounds = Maps.newHashMap();
final Iterator<DimFilter> iterator = newChildren.iterator();
while (iterator.hasNext()) {
final DimFilter child = iterator.next();
if (child.equals(Filtration.matchNothing())) {
// Child matches nothing, equivalent to FALSE
// OR with FALSE => ignore
// AND with FALSE => always false, short circuit
if (disjunction) {
iterator.remove();
} else {
return Filtration.matchNothing();
}
} else if (child.equals(Filtration.matchEverything())) {
// Child matches everything, equivalent to TRUE
// OR with TRUE => always true, short circuit
// AND with TRUE => ignore
if (disjunction) {
return Filtration.matchEverything();
} else {
iterator.remove();
}
} else if (child instanceof BoundDimFilter) {
final BoundDimFilter bound = (BoundDimFilter) child;
final BoundRefKey boundRefKey = BoundRefKey.from(bound);
List<BoundDimFilter> filterList = bounds.get(boundRefKey);
if (filterList == null) {
filterList = Lists.newArrayList();
bounds.put(boundRefKey, filterList);
}
filterList.add(bound);
}
}
// Try to simplify filters within each group.
for (Map.Entry<BoundRefKey, List<BoundDimFilter>> entry : bounds.entrySet()) {
final BoundRefKey boundRefKey = entry.getKey();
final List<BoundDimFilter> filterList = entry.getValue();
// Create a RangeSet for this group.
final RangeSet<BoundValue> rangeSet = disjunction
? RangeSets.unionRanges(Bounds.toRanges(filterList))
: RangeSets.intersectRanges(Bounds.toRanges(filterList));
if (rangeSet.asRanges().size() < filterList.size()) {
// We found a simplification. Remove the old filters and add new ones.
for (final BoundDimFilter bound : filterList) {
if (!newChildren.remove(bound)) {
throw new ISE("WTF?! Tried to remove bound but couldn't?");
}
}
if (rangeSet.asRanges().isEmpty()) {
// range set matches nothing, equivalent to FALSE
// OR with FALSE => ignore
// AND with FALSE => always false, short circuit
if (disjunction) {
newChildren.add(Filtration.matchNothing());
} else {
return Filtration.matchNothing();
}
}
for (final Range<BoundValue> range : rangeSet.asRanges()) {
if (!range.hasLowerBound() && !range.hasUpperBound()) {
// range matches all, equivalent to TRUE
// AND with TRUE => ignore
// OR with TRUE => always true; short circuit
if (disjunction) {
return Filtration.matchEverything();
} else {
newChildren.add(Filtration.matchEverything());
}
} else {
newChildren.add(Bounds.toFilter(boundRefKey, range));
}
}
}
}
Preconditions.checkState(newChildren.size() > 0, "newChildren.size > 0");
if (newChildren.size() == 1) {
return newChildren.get(0);
} else {
return disjunction ? new OrDimFilter(newChildren) : new AndDimFilter(newChildren);
}
}
private static DimFilter negate(final DimFilter filter)
{
if (Filtration.matchEverything().equals(filter)) {
return Filtration.matchNothing();
} else if (Filtration.matchNothing().equals(filter)) {
return Filtration.matchEverything();
} else if (filter instanceof NotDimFilter) {
return ((NotDimFilter) filter).getField();
} else if (filter instanceof BoundDimFilter) {
final BoundDimFilter negated = Bounds.not((BoundDimFilter) filter);
return negated != null ? negated : new NotDimFilter(filter);
} else {
return new NotDimFilter(filter);
}
}
private static List<DimFilter> negateAll(final List<DimFilter> children)
{
final List<DimFilter> newChildren = Lists.newArrayListWithCapacity(children.size());
for (final DimFilter child : children) {
newChildren.add(negate(child));
}
return newChildren;
}
private static int computeCost(final DimFilter filter)
{
if (filter instanceof NotDimFilter) {
return computeCost(((NotDimFilter) filter).getField());
} else if (filter instanceof AndDimFilter) {
int cost = 0;
for (DimFilter field : ((AndDimFilter) filter).getFields()) {
cost += computeCost(field);
}
return cost;
} else if (filter instanceof OrDimFilter) {
int cost = 0;
for (DimFilter field : ((OrDimFilter) filter).getFields()) {
cost += computeCost(field);
}
return cost;
} else {
return 1;
}
}
}

View File

@ -0,0 +1,72 @@
/*
* 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.filtration;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.ordering.StringComparator;
import io.druid.sql.calcite.expression.RowExtraction;
import io.druid.sql.calcite.table.DruidTable;
import io.druid.sql.calcite.table.DruidTables;
public class ConvertBoundsToSelectors extends BottomUpTransform
{
private final DruidTable druidTable;
private ConvertBoundsToSelectors(final DruidTable druidTable)
{
this.druidTable = druidTable;
}
public static ConvertBoundsToSelectors create(final DruidTable druidTable)
{
return new ConvertBoundsToSelectors(druidTable);
}
@Override
public DimFilter process(DimFilter filter)
{
if (filter instanceof BoundDimFilter) {
final BoundDimFilter bound = (BoundDimFilter) filter;
final StringComparator naturalStringComparator = DruidTables.naturalStringComparator(
druidTable,
RowExtraction.of(bound.getDimension(), bound.getExtractionFn())
);
if (bound.hasUpperBound()
&& bound.hasLowerBound()
&& bound.getUpper().equals(bound.getLower())
&& !bound.isUpperStrict()
&& !bound.isLowerStrict()
&& bound.getOrdering().equals(naturalStringComparator)) {
return new SelectorDimFilter(
bound.getDimension(),
bound.getUpper(),
bound.getExtractionFn()
);
} else {
return filter;
}
} else {
return filter;
}
}
}

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.filtration;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import io.druid.java.util.common.ISE;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.InDimFilter;
import io.druid.query.filter.OrDimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.sql.calcite.expression.RowExtraction;
import io.druid.sql.calcite.table.DruidTable;
import io.druid.sql.calcite.table.DruidTables;
import java.util.List;
import java.util.Map;
public class ConvertSelectorsToIns extends BottomUpTransform
{
private final DruidTable druidTable;
private ConvertSelectorsToIns(final DruidTable druidTable)
{
this.druidTable = druidTable;
}
public static ConvertSelectorsToIns create(final DruidTable druidTable)
{
return new ConvertSelectorsToIns(druidTable);
}
@Override
public DimFilter process(DimFilter filter)
{
if (filter instanceof OrDimFilter) {
// Copy children list
final List<DimFilter> children = Lists.newArrayList(((OrDimFilter) filter).getFields());
// Group filters by dimension and extractionFn.
final Map<BoundRefKey, List<SelectorDimFilter>> selectors = Maps.newHashMap();
for (DimFilter child : children) {
if (child instanceof SelectorDimFilter) {
final SelectorDimFilter selector = (SelectorDimFilter) child;
final BoundRefKey boundRefKey = BoundRefKey.from(
selector,
DruidTables.naturalStringComparator(
druidTable,
RowExtraction.of(selector.getDimension(), selector.getExtractionFn())
)
);
List<SelectorDimFilter> filterList = selectors.get(boundRefKey);
if (filterList == null) {
filterList = Lists.newArrayList();
selectors.put(boundRefKey, filterList);
}
filterList.add(selector);
}
}
// Emit IN filters for each group of size > 1.
for (Map.Entry<BoundRefKey, List<SelectorDimFilter>> entry : selectors.entrySet()) {
final List<SelectorDimFilter> filterList = entry.getValue();
if (filterList.size() > 1) {
// We found a simplification. Remove the old filters and add new ones.
final List<String> values = Lists.newArrayList();
for (final SelectorDimFilter selector : filterList) {
values.add(selector.getValue());
if (!children.remove(selector)) {
throw new ISE("WTF?! Tried to remove selector but couldn't?");
}
}
children.add(new InDimFilter(entry.getKey().getDimension(), values, entry.getKey().getExtractionFn()));
}
}
if (!children.equals(((OrDimFilter) filter).getFields())) {
return children.size() == 1 ? children.get(0) : new OrDimFilter(children);
} else {
return filter;
}
} else {
return filter;
}
}
}

View File

@ -0,0 +1,189 @@
/*
* 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.filtration;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import io.druid.common.utils.JodaUtils;
import io.druid.java.util.common.ISE;
import io.druid.js.JavaScriptConfig;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.JavaScriptDimFilter;
import io.druid.query.spec.MultipleIntervalSegmentSpec;
import io.druid.query.spec.QuerySegmentSpec;
import io.druid.sql.calcite.table.DruidTable;
import org.joda.time.Interval;
import java.util.List;
public class Filtration
{
private static final Interval ETERNITY = new Interval(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT);
private static final DimFilter MATCH_NOTHING = new JavaScriptDimFilter(
"dummy", "function(x){return false;}", null, JavaScriptConfig.getDefault()
);
private static final DimFilter MATCH_EVERYTHING = new JavaScriptDimFilter(
"dummy", "function(x){return true;}", null, JavaScriptConfig.getDefault()
);
// 1) If "dimFilter" is null, it should be ignored and not affect filtration.
// 2) There is an implicit AND between "intervals" and "dimFilter" (if dimFilter is non-null).
// 3) There is an implicit OR between the intervals in "intervals".
private final DimFilter dimFilter;
private final List<Interval> intervals;
private Filtration(final DimFilter dimFilter, final List<Interval> intervals)
{
this.intervals = intervals != null ? intervals : ImmutableList.of(ETERNITY);
this.dimFilter = dimFilter;
}
public static Interval eternity()
{
return ETERNITY;
}
public static DimFilter matchNothing()
{
return MATCH_NOTHING;
}
public static DimFilter matchEverything()
{
return MATCH_EVERYTHING;
}
public static Filtration create(final DimFilter dimFilter)
{
return new Filtration(dimFilter, null);
}
public static Filtration create(final DimFilter dimFilter, final List<Interval> intervals)
{
return new Filtration(dimFilter, intervals);
}
private static Filtration transform(final Filtration filtration, final List<Function<Filtration, Filtration>> fns)
{
Filtration retVal = filtration;
for (Function<Filtration, Filtration> fn : fns) {
retVal = fn.apply(retVal);
}
return retVal;
}
public QuerySegmentSpec getQuerySegmentSpec()
{
return new MultipleIntervalSegmentSpec(intervals);
}
public List<Interval> getIntervals()
{
return intervals;
}
public DimFilter getDimFilter()
{
return dimFilter;
}
/**
* Optimize a Filtration for querying, possibly pulling out intervals and simplifying the dimFilter in the process.
*
* @return equivalent Filtration
*/
public Filtration optimize(final DruidTable druidTable)
{
return transform(
this,
ImmutableList.of(
CombineAndSimplifyBounds.instance(),
MoveTimeFiltersToIntervals.instance(),
ConvertBoundsToSelectors.create(druidTable),
ConvertSelectorsToIns.create(druidTable),
MoveMarkerFiltersToIntervals.instance(),
ValidateNoMarkerFiltersRemain.instance()
)
);
}
/**
* Optimize a Filtration containing only a DimFilter, avoiding pulling out intervals.
*
* @return equivalent Filtration
*/
public Filtration optimizeFilterOnly(final DruidTable druidTable)
{
if (!intervals.equals(ImmutableList.of(eternity()))) {
throw new ISE("Cannot optimizeFilterOnly when intervals are set");
}
final Filtration transformed = transform(
this,
ImmutableList.<Function<Filtration, Filtration>>of(
CombineAndSimplifyBounds.instance(),
ConvertBoundsToSelectors.create(druidTable),
ConvertSelectorsToIns.create(druidTable)
)
);
if (!transformed.getIntervals().equals(ImmutableList.of(eternity()))) {
throw new ISE("WTF?! optimizeFilterOnly was about to return filtration with intervals?!");
}
return transformed;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Filtration that = (Filtration) o;
if (intervals != null ? !intervals.equals(that.intervals) : that.intervals != null) {
return false;
}
return dimFilter != null ? dimFilter.equals(that.dimFilter) : that.dimFilter == null;
}
@Override
public int hashCode()
{
int result = intervals != null ? intervals.hashCode() : 0;
result = 31 * result + (dimFilter != null ? dimFilter.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "Filtration{" +
"intervals=" + intervals +
", dimFilter=" + dimFilter +
'}';
}
}

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.calcite.filtration;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import org.joda.time.Interval;
public class MoveMarkerFiltersToIntervals implements Function<Filtration, Filtration>
{
private static final MoveMarkerFiltersToIntervals INSTANCE = new MoveMarkerFiltersToIntervals();
private MoveMarkerFiltersToIntervals()
{
}
public static MoveMarkerFiltersToIntervals instance()
{
return INSTANCE;
}
@Override
public Filtration apply(final Filtration filtration)
{
if (Filtration.matchEverything().equals(filtration.getDimFilter())) {
return Filtration.create(null, filtration.getIntervals());
} else if (Filtration.matchNothing().equals(filtration.getDimFilter())) {
return Filtration.create(null, ImmutableList.<Interval>of());
} else {
return filtration;
}
}
}

View File

@ -0,0 +1,172 @@
/*
* 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.filtration;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Range;
import com.google.common.collect.RangeSet;
import io.druid.java.util.common.Pair;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.NotDimFilter;
import io.druid.query.filter.OrDimFilter;
import io.druid.query.ordering.StringComparators;
import io.druid.segment.column.Column;
import java.util.List;
public class MoveTimeFiltersToIntervals implements Function<Filtration, Filtration>
{
private static final MoveTimeFiltersToIntervals INSTANCE = new MoveTimeFiltersToIntervals();
private static final BoundRefKey TIME_BOUND_REF_KEY = new BoundRefKey(
Column.TIME_COLUMN_NAME,
null,
StringComparators.NUMERIC
);
private MoveTimeFiltersToIntervals()
{
}
public static MoveTimeFiltersToIntervals instance()
{
return INSTANCE;
}
@Override
public Filtration apply(final Filtration filtration)
{
if (filtration.getDimFilter() == null) {
return filtration;
}
// Convert existing filtration intervals to a RangeSet.
final RangeSet<Long> rangeSet = RangeSets.fromIntervals(filtration.getIntervals());
// Remove anything outside eternity.
rangeSet.removeAll(RangeSets.fromIntervals(ImmutableList.of(Filtration.eternity())).complement());
// Extract time bounds from the dimFilter.
final Pair<DimFilter, RangeSet<Long>> pair = extractConvertibleTimeBounds(filtration.getDimFilter());
if (pair.rhs != null) {
rangeSet.removeAll(pair.rhs.complement());
}
return Filtration.create(pair.lhs, RangeSets.toIntervals(rangeSet));
}
/**
* Extract bound filters on __time that can be converted to query-level "intervals".
*
* @return pair of new dimFilter + RangeSet of __time that should be ANDed together. Either can be null but not both.
*/
private static Pair<DimFilter, RangeSet<Long>> extractConvertibleTimeBounds(final DimFilter filter)
{
if (filter instanceof AndDimFilter) {
final List<DimFilter> children = ((AndDimFilter) filter).getFields();
final List<DimFilter> newChildren = Lists.newArrayList();
final List<RangeSet<Long>> rangeSets = Lists.newArrayList();
for (DimFilter child : children) {
final Pair<DimFilter, RangeSet<Long>> pair = extractConvertibleTimeBounds(child);
if (pair.lhs != null) {
newChildren.add(pair.lhs);
}
if (pair.rhs != null) {
rangeSets.add(pair.rhs);
}
}
final DimFilter newFilter;
if (newChildren.size() == 0) {
newFilter = null;
} else if (newChildren.size() == 1) {
newFilter = newChildren.get(0);
} else {
newFilter = new AndDimFilter(newChildren);
}
return Pair.of(
newFilter,
rangeSets.isEmpty() ? null : RangeSets.intersectRangeSets(rangeSets)
);
} else if (filter instanceof OrDimFilter) {
final List<DimFilter> children = ((OrDimFilter) filter).getFields();
final List<RangeSet<Long>> rangeSets = Lists.newArrayList();
boolean allCompletelyConverted = true;
boolean allHadIntervals = true;
for (DimFilter child : children) {
final Pair<DimFilter, RangeSet<Long>> pair = extractConvertibleTimeBounds(child);
if (pair.lhs != null) {
allCompletelyConverted = false;
}
if (pair.rhs != null) {
rangeSets.add(pair.rhs);
} else {
allHadIntervals = false;
}
}
if (allCompletelyConverted) {
return Pair.of(null, RangeSets.unionRangeSets(rangeSets));
} else {
return Pair.of(filter, allHadIntervals ? RangeSets.unionRangeSets(rangeSets) : null);
}
} else if (filter instanceof NotDimFilter) {
final DimFilter child = ((NotDimFilter) filter).getField();
final Pair<DimFilter, RangeSet<Long>> pair = extractConvertibleTimeBounds(child);
if (pair.rhs != null && pair.lhs == null) {
return Pair.of(null, pair.rhs.complement());
} else {
return Pair.of(filter, null);
}
} else if (filter instanceof BoundDimFilter) {
final BoundDimFilter bound = (BoundDimFilter) filter;
if (BoundRefKey.from(bound).equals(TIME_BOUND_REF_KEY)) {
return Pair.of(null, RangeSets.of(toLongRange(Bounds.toRange(bound))));
} else {
return Pair.of(filter, null);
}
} else {
return Pair.of(filter, null);
}
}
private static Range<Long> toLongRange(final Range<BoundValue> range)
{
if (!range.hasUpperBound() && !range.hasLowerBound()) {
return Range.all();
} else if (range.hasUpperBound() && !range.hasLowerBound()) {
return Range.upTo(Long.parseLong(range.upperEndpoint().getValue()), range.upperBoundType());
} else if (!range.hasUpperBound() && range.hasLowerBound()) {
return Range.downTo(Long.parseLong(range.lowerEndpoint().getValue()), range.lowerBoundType());
} else {
return Range.range(
Long.parseLong(range.lowerEndpoint().getValue()), range.lowerBoundType(),
Long.parseLong(range.upperEndpoint().getValue()), range.upperBoundType()
);
}
}
}

View File

@ -0,0 +1,136 @@
/*
* 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.filtration;
import com.google.common.collect.BoundType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Range;
import com.google.common.collect.RangeSet;
import com.google.common.collect.TreeRangeSet;
import org.joda.time.Interval;
import java.util.List;
public class RangeSets
{
public static <T extends Comparable<T>> RangeSet<T> of(final Range<T> range)
{
return unionRanges(ImmutableList.of(range));
}
/**
* Unions a set of ranges, or returns null if the set is empty.
*/
public static <T extends Comparable<T>> RangeSet<T> unionRanges(final Iterable<Range<T>> ranges)
{
RangeSet<T> rangeSet = null;
for (Range<T> range : ranges) {
if (rangeSet == null) {
rangeSet = TreeRangeSet.create();
}
rangeSet.add(range);
}
return rangeSet;
}
/**
* Unions a set of rangeSets, or returns null if the set is empty.
*/
public static <T extends Comparable<T>> RangeSet<T> unionRangeSets(final Iterable<RangeSet<T>> rangeSets)
{
final RangeSet<T> rangeSet = TreeRangeSet.create();
for (RangeSet<T> set : rangeSets) {
rangeSet.addAll(set);
}
return rangeSet;
}
/**
* Intersects a set of ranges, or returns null if the set is empty.
*/
public static <T extends Comparable<T>> RangeSet<T> intersectRanges(final Iterable<Range<T>> ranges)
{
RangeSet<T> rangeSet = null;
for (final Range<T> range : ranges) {
if (rangeSet == null) {
rangeSet = TreeRangeSet.create();
rangeSet.add(range);
} else {
rangeSet = TreeRangeSet.create(rangeSet.subRangeSet(range));
}
}
return rangeSet;
}
/**
* Intersects a set of rangeSets, or returns null if the set is empty.
*/
public static <T extends Comparable<T>> RangeSet<T> intersectRangeSets(final Iterable<RangeSet<T>> rangeSets)
{
RangeSet<T> rangeSet = null;
for (final RangeSet<T> set : rangeSets) {
if (rangeSet == null) {
rangeSet = TreeRangeSet.create();
rangeSet.addAll(set);
} else {
rangeSet.removeAll(set.complement());
}
}
return rangeSet;
}
public static RangeSet<Long> fromIntervals(final Iterable<Interval> intervals)
{
final RangeSet<Long> retVal = TreeRangeSet.create();
for (Interval interval : intervals) {
retVal.add(Range.closedOpen(interval.getStartMillis(), interval.getEndMillis()));
}
return retVal;
}
public static List<Interval> toIntervals(final RangeSet<Long> rangeSet)
{
final List<Interval> retVal = Lists.newArrayList();
for (Range<Long> range : rangeSet.asRanges()) {
final long start;
final long end;
if (range.hasLowerBound()) {
final long millis = range.lowerEndpoint();
start = millis + (range.lowerBoundType() == BoundType.OPEN ? 1 : 0);
} else {
start = Filtration.eternity().getStartMillis();
}
if (range.hasUpperBound()) {
final long millis = range.upperEndpoint();
end = millis + (range.upperBoundType() == BoundType.OPEN ? 0 : 1);
} else {
end = Filtration.eternity().getEndMillis();
}
retVal.add(new Interval(start, end));
}
return retVal;
}
}

View File

@ -0,0 +1,47 @@
/*
* 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.filtration;
import io.druid.java.util.common.ISE;
import io.druid.query.filter.DimFilter;
public class ValidateNoMarkerFiltersRemain extends BottomUpTransform
{
private static final ValidateNoMarkerFiltersRemain INSTANCE = new ValidateNoMarkerFiltersRemain();
private ValidateNoMarkerFiltersRemain()
{
}
public static ValidateNoMarkerFiltersRemain instance()
{
return INSTANCE;
}
@Override
protected DimFilter process(DimFilter filter)
{
if (Filtration.matchNothing().equals(filter) || Filtration.matchEverything().equals(filter)) {
throw new ISE("Marker filters shouldn't exist in the final filter, but found: %s", filter);
}
return filter;
}
}

View File

@ -0,0 +1,98 @@
/*
* 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.Predicates;
import com.google.common.collect.ImmutableList;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.rel.core.Aggregate;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Values;
import org.apache.calcite.rel.logical.LogicalValues;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import java.math.BigDecimal;
/**
* Rule that applies Aggregate to Values. Currently only applies to empty Values.
*
* This is still useful because PruneEmptyRules doesn't handle Aggregate, which is in turn because
* Aggregate of empty relations need some special handling: a single row will be generated, where
* each column's value depends on the specific aggregate calls (e.g. COUNT is 0, SUM is NULL).
* Sample query where this matters: <code>SELECT COUNT(*) FROM s.foo WHERE 1 = 0</code>.
*
* Can be replaced by AggregateValuesRule in Calcite 1.11.0, when released.
*/
public class AggregateValuesRule extends RelOptRule
{
public static final AggregateValuesRule INSTANCE = new AggregateValuesRule();
private AggregateValuesRule()
{
super(
operand(Aggregate.class, null, Predicates.not(Aggregate.IS_NOT_GRAND_TOTAL),
operand(Values.class, null, Values.IS_EMPTY, none())
)
);
}
@Override
public void onMatch(RelOptRuleCall call)
{
final Aggregate aggregate = call.rel(0);
final Values values = call.rel(1);
final ImmutableList.Builder<RexLiteral> literals = ImmutableList.builder();
final RexBuilder rexBuilder = call.builder().getRexBuilder();
for (final AggregateCall aggregateCall : aggregate.getAggCallList()) {
switch (aggregateCall.getAggregation().getKind()) {
case COUNT:
case SUM0:
literals.add((RexLiteral) rexBuilder.makeLiteral(
BigDecimal.ZERO, aggregateCall.getType(), false));
break;
case MIN:
case MAX:
case SUM:
literals.add(rexBuilder.constantNull());
break;
default:
// Unknown what this aggregate call should do on empty Values. Bail out to be safe.
return;
}
}
call.transformTo(
LogicalValues.create(
values.getCluster(),
aggregate.getRowType(),
ImmutableList.of(literals.build())
)
);
// New plan is absolutely better than old plan.
call.getPlanner().setImportance(aggregate, 0.0);
}
}

View File

@ -0,0 +1,89 @@
/*
* 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.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 org.apache.calcite.schema.Schema;
import org.apache.calcite.schema.SchemaPlus;
import java.sql.SQLException;
import java.util.Properties;
/**
* Entry points for Calcite.
*/
public class Calcites
{
private static final String DRUID_SCHEMA_NAME = "druid";
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
{
final Properties props = new Properties();
props.setProperty("caseSensitive", "true");
props.setProperty("unquotedCasing", "UNCHANGED");
final CalciteJdbc41Factory jdbcFactory = new CalciteJdbc41Factory();
final Function0<CalcitePrepare> prepareFactory = new Function0<CalcitePrepare>()
{
@Override
public CalcitePrepare apply()
{
return new DruidPlannerImpl(plannerConfig);
}
};
final Driver driver = new Driver()
{
@Override
protected Function0<CalcitePrepare> createPrepareFactory()
{
return prepareFactory;
}
};
final CalciteConnection calciteConnection = (CalciteConnection) jdbcFactory.newConnection(
driver,
jdbcFactory,
"jdbc:calcite:",
props
);
final SchemaPlus druidSchemaPlus = calciteConnection.getRootSchema().add(DRUID_SCHEMA_NAME, druidSchema);
druidSchemaPlus.setCacheEnabled(false);
return calciteConnection;
}
}

View File

@ -0,0 +1,59 @@
/*
* 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.rex.RexNode;
import org.apache.calcite.sql.SqlCall;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql2rel.SqlRexContext;
import org.apache.calcite.sql2rel.SqlRexConvertlet;
import org.apache.calcite.sql2rel.SqlRexConvertletTable;
import org.apache.calcite.sql2rel.StandardConvertletTable;
public class DruidConvertletTable implements SqlRexConvertletTable
{
private static final DruidConvertletTable INSTANCE = new DruidConvertletTable();
private DruidConvertletTable()
{
}
public static DruidConvertletTable instance()
{
return INSTANCE;
}
@Override
public SqlRexConvertlet get(SqlCall call)
{
if (call.getKind() == SqlKind.EXTRACT && call.getOperandList().get(1).getKind() != SqlKind.LITERAL) {
return new SqlRexConvertlet()
{
@Override
public RexNode convertCall(SqlRexContext cx, SqlCall call)
{
return StandardConvertletTable.INSTANCE.convertCall(cx, call);
}
};
} else {
return StandardConvertletTable.INSTANCE.get(call);
}
}
}

View File

@ -0,0 +1,68 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.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;
/**
* 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;
}
}

View File

@ -0,0 +1,145 @@
/*
* 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.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Period;
public class PlannerConfig
{
@JsonProperty
private Period metadataRefreshPeriod = new Period("PT1M");
@JsonProperty
private int maxSemiJoinRowsInMemory = 100000;
@JsonProperty
private int maxTopNLimit = 100000;
@JsonProperty
private int selectThreshold = 1000;
@JsonProperty
private boolean useApproximateCountDistinct = true;
@JsonProperty
private boolean useApproximateTopN = true;
@JsonProperty
private boolean useFallback = false;
public Period getMetadataRefreshPeriod()
{
return metadataRefreshPeriod;
}
public int getMaxSemiJoinRowsInMemory()
{
return maxSemiJoinRowsInMemory;
}
public int getMaxTopNLimit()
{
return maxTopNLimit;
}
public int getSelectThreshold()
{
return selectThreshold;
}
public boolean isUseApproximateCountDistinct()
{
return useApproximateCountDistinct;
}
public boolean isUseApproximateTopN()
{
return useApproximateTopN;
}
public boolean isUseFallback()
{
return useFallback;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
PlannerConfig that = (PlannerConfig) o;
if (maxSemiJoinRowsInMemory != that.maxSemiJoinRowsInMemory) {
return false;
}
if (maxTopNLimit != that.maxTopNLimit) {
return false;
}
if (selectThreshold != that.selectThreshold) {
return false;
}
if (useApproximateCountDistinct != that.useApproximateCountDistinct) {
return false;
}
if (useApproximateTopN != that.useApproximateTopN) {
return false;
}
if (useFallback != that.useFallback) {
return false;
}
return metadataRefreshPeriod != null
? metadataRefreshPeriod.equals(that.metadataRefreshPeriod)
: that.metadataRefreshPeriod == null;
}
@Override
public int hashCode()
{
int result = metadataRefreshPeriod != null ? metadataRefreshPeriod.hashCode() : 0;
result = 31 * result + maxSemiJoinRowsInMemory;
result = 31 * result + maxTopNLimit;
result = 31 * result + selectThreshold;
result = 31 * result + (useApproximateCountDistinct ? 1 : 0);
result = 31 * result + (useApproximateTopN ? 1 : 0);
result = 31 * result + (useFallback ? 1 : 0);
return result;
}
@Override
public String toString()
{
return "PlannerConfig{" +
"metadataRefreshPeriod=" + metadataRefreshPeriod +
", maxSemiJoinRowsInMemory=" + maxSemiJoinRowsInMemory +
", maxTopNLimit=" + maxTopNLimit +
", selectThreshold=" + selectThreshold +
", useApproximateCountDistinct=" + useApproximateCountDistinct +
", useApproximateTopN=" + useApproximateTopN +
", useFallback=" + useFallback +
'}';
}
}

View File

@ -0,0 +1,214 @@
/*
* 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.ImmutableList;
import io.druid.sql.calcite.rule.DruidBindableConverterRule;
import io.druid.sql.calcite.rule.DruidFilterRule;
import io.druid.sql.calcite.rule.DruidSelectProjectionRule;
import io.druid.sql.calcite.rule.DruidSelectSortRule;
import io.druid.sql.calcite.rule.DruidSemiJoinRule;
import io.druid.sql.calcite.rule.GroupByRules;
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;
import org.apache.calcite.rel.rules.AggregateJoinTransposeRule;
import org.apache.calcite.rel.rules.AggregateProjectMergeRule;
import org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule;
import org.apache.calcite.rel.rules.AggregateRemoveRule;
import org.apache.calcite.rel.rules.AggregateStarTableRule;
import org.apache.calcite.rel.rules.CalcRemoveRule;
import org.apache.calcite.rel.rules.DateRangeRules;
import org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
import org.apache.calcite.rel.rules.FilterJoinRule;
import org.apache.calcite.rel.rules.FilterMergeRule;
import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
import org.apache.calcite.rel.rules.FilterTableScanRule;
import org.apache.calcite.rel.rules.JoinCommuteRule;
import org.apache.calcite.rel.rules.JoinPushExpressionsRule;
import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
import org.apache.calcite.rel.rules.ProjectMergeRule;
import org.apache.calcite.rel.rules.ProjectRemoveRule;
import org.apache.calcite.rel.rules.ProjectTableScanRule;
import org.apache.calcite.rel.rules.ProjectToWindowRule;
import org.apache.calcite.rel.rules.ProjectWindowTransposeRule;
import org.apache.calcite.rel.rules.PruneEmptyRules;
import org.apache.calcite.rel.rules.ReduceExpressionsRule;
import org.apache.calcite.rel.rules.SemiJoinRule;
import org.apache.calcite.rel.rules.SortJoinTransposeRule;
import org.apache.calcite.rel.rules.SortProjectTransposeRule;
import org.apache.calcite.rel.rules.SortRemoveRule;
import org.apache.calcite.rel.rules.SortUnionTransposeRule;
import org.apache.calcite.rel.rules.TableScanRule;
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 java.util.List;
public class Rules
{
// Rules from CalcitePrepareImpl's DEFAULT_RULES, minus AggregateExpandDistinctAggregatesRule
// and AggregateReduceFunctionsRule.
private static final List<RelOptRule> DEFAULT_RULES =
ImmutableList.of(
AggregateStarTableRule.INSTANCE,
AggregateStarTableRule.INSTANCE2,
TableScanRule.INSTANCE,
ProjectMergeRule.INSTANCE,
FilterTableScanRule.INSTANCE,
ProjectFilterTransposeRule.INSTANCE,
FilterProjectTransposeRule.INSTANCE,
FilterJoinRule.FILTER_ON_JOIN,
JoinPushExpressionsRule.INSTANCE,
FilterAggregateTransposeRule.INSTANCE,
ProjectWindowTransposeRule.INSTANCE,
JoinCommuteRule.INSTANCE,
JoinPushThroughJoinRule.RIGHT,
JoinPushThroughJoinRule.LEFT,
SortProjectTransposeRule.INSTANCE,
SortJoinTransposeRule.INSTANCE,
SortUnionTransposeRule.INSTANCE
);
// Rules from CalcitePrepareImpl's createPlanner.
private static final List<RelOptRule> MISCELLANEOUS_RULES =
ImmutableList.of(
Bindables.BINDABLE_TABLE_SCAN_RULE,
ProjectTableScanRule.INSTANCE,
ProjectTableScanRule.INTERPRETER,
EnumerableInterpreterRule.INSTANCE,
EnumerableRules.ENUMERABLE_VALUES_RULE
);
// Rules from CalcitePrepareImpl's CONSTANT_REDUCTION_RULES.
private static final List<RelOptRule> CONSTANT_REDUCTION_RULES =
ImmutableList.of(
ReduceExpressionsRule.PROJECT_INSTANCE,
ReduceExpressionsRule.CALC_INSTANCE,
ReduceExpressionsRule.JOIN_INSTANCE,
ReduceExpressionsRule.FILTER_INSTANCE,
ValuesReduceRule.FILTER_INSTANCE,
ValuesReduceRule.PROJECT_FILTER_INSTANCE,
ValuesReduceRule.PROJECT_INSTANCE,
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(
FilterJoinRule.FILTER_ON_JOIN,
FilterJoinRule.JOIN,
AbstractConverter.ExpandConversionRule.INSTANCE,
JoinCommuteRule.INSTANCE,
SemiJoinRule.INSTANCE,
AggregateRemoveRule.INSTANCE,
UnionToDistinctRule.INSTANCE,
ProjectRemoveRule.INSTANCE,
AggregateJoinTransposeRule.INSTANCE,
AggregateProjectMergeRule.INSTANCE,
CalcRemoveRule.INSTANCE,
SortRemoveRule.INSTANCE
);
// Rules from RelOptUtil's registerAbstractRels.
private static final List<RelOptRule> RELOPTUTIL_ABSTRACT_RULES =
ImmutableList.of(
AggregateProjectPullUpConstantsRule.INSTANCE2,
UnionPullUpConstantsRule.INSTANCE,
PruneEmptyRules.UNION_INSTANCE,
PruneEmptyRules.PROJECT_INSTANCE,
PruneEmptyRules.FILTER_INSTANCE,
PruneEmptyRules.SORT_INSTANCE,
PruneEmptyRules.AGGREGATE_INSTANCE,
PruneEmptyRules.JOIN_LEFT_INSTANCE,
PruneEmptyRules.JOIN_RIGHT_INSTANCE,
PruneEmptyRules.SORT_FETCH_ZERO_INSTANCE,
UnionMergeRule.INSTANCE,
ProjectToWindowRule.PROJECT,
FilterMergeRule.INSTANCE,
DateRangeRules.FILTER_INSTANCE
);
private Rules()
{
// No instantiation.
}
public static List<RelOptRule> ruleSet(final PlannerConfig plannerConfig)
{
final ImmutableList.Builder<RelOptRule> rules = ImmutableList.builder();
// Calcite rules.
rules.addAll(DEFAULT_RULES);
rules.addAll(MISCELLANEOUS_RULES);
rules.addAll(CONSTANT_REDUCTION_RULES);
rules.addAll(VOLCANO_ABSTRACT_RULES);
rules.addAll(RELOPTUTIL_ABSTRACT_RULES);
if (plannerConfig.isUseFallback()) {
rules.addAll(ENUMERABLE_RULES);
}
// Druid-specific rules.
rules.add(DruidFilterRule.instance());
rules.add(DruidSelectSortRule.instance());
rules.add(DruidSelectProjectionRule.instance());
if (plannerConfig.getMaxSemiJoinRowsInMemory() > 0) {
rules.add(DruidSemiJoinRule.instance());
}
rules.addAll(GroupByRules.rules(plannerConfig));
// Allow conversion of Druid queries to Bindable convention.
rules.add(DruidBindableConverterRule.instance());
return rules.build();
}
}

View File

@ -0,0 +1,84 @@
/*
* 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.rel;
import org.apache.calcite.plan.Convention;
import org.apache.calcite.plan.RelOptPlanner;
import org.apache.calcite.plan.RelTrait;
import org.apache.calcite.plan.RelTraitDef;
import org.apache.calcite.plan.RelTraitSet;
public class DruidConvention implements Convention
{
private static final DruidConvention INSTANCE = new DruidConvention();
private DruidConvention()
{
}
public static DruidConvention instance()
{
return INSTANCE;
}
@Override
public Class getInterface()
{
return null;
}
@Override
public String getName()
{
return null;
}
@Override
public boolean canConvertConvention(Convention toConvention)
{
return false;
}
@Override
public boolean useAbstractConvertersForConversion(
RelTraitSet fromTraits, RelTraitSet toTraits
)
{
return false;
}
@Override
public RelTraitDef getTraitDef()
{
return null;
}
@Override
public boolean satisfies(RelTrait trait)
{
return false;
}
@Override
public void register(RelOptPlanner planner)
{
}
}

View File

@ -0,0 +1,388 @@
/*
* 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.rel;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import io.druid.granularity.QueryGranularities;
import io.druid.granularity.QueryGranularity;
import io.druid.java.util.common.ISE;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.filter.DimFilter;
import io.druid.query.groupby.orderby.DefaultLimitSpec;
import io.druid.query.groupby.orderby.OrderByColumnSpec;
import io.druid.query.topn.DimensionTopNMetricSpec;
import io.druid.query.topn.InvertedTopNMetricSpec;
import io.druid.query.topn.NumericTopNMetricSpec;
import io.druid.query.topn.TopNMetricSpec;
import io.druid.segment.column.Column;
import io.druid.sql.calcite.expression.ExtractionFns;
import io.druid.sql.calcite.planner.PlannerConfig;
import io.druid.sql.calcite.table.DruidTable;
import org.apache.calcite.interpreter.Row;
import org.apache.calcite.plan.RelTrait;
import org.apache.calcite.rel.RelCollations;
import org.apache.calcite.rel.RelFieldCollation;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelDataTypeField;
import java.util.List;
public class DruidQueryBuilder
{
private final DimFilter filter;
private final SelectProjection selectProjection;
private final Grouping grouping;
private final DimFilter having;
private final DefaultLimitSpec limitSpec;
private final RelDataType rowType;
private final List<String> rowOrder;
private DruidQueryBuilder(
final DimFilter filter,
final SelectProjection selectProjection,
final Grouping grouping,
final DimFilter having,
final DefaultLimitSpec limitSpec,
final RelDataType rowType,
final List<String> rowOrder
)
{
this.filter = filter;
this.selectProjection = selectProjection;
this.grouping = grouping;
this.having = having;
this.limitSpec = limitSpec;
this.rowType = Preconditions.checkNotNull(rowType, "rowType");
this.rowOrder = Preconditions.checkNotNull(ImmutableList.copyOf(rowOrder), "rowOrder");
if (selectProjection != null && grouping != null) {
throw new ISE("Cannot have both selectProjection and grouping");
}
}
public static DruidQueryBuilder fullScan(final DruidTable druidTable, final RelDataTypeFactory relDataTypeFactory)
{
final RelDataType rowType = druidTable.getRowType(relDataTypeFactory);
final List<String> rowOrder = Lists.newArrayListWithCapacity(rowType.getFieldCount());
for (RelDataTypeField field : rowType.getFieldList()) {
rowOrder.add(field.getName());
}
return new DruidQueryBuilder(null, null, null, null, null, rowType, rowOrder);
}
public DruidQueryBuilder withFilter(final DimFilter newFilter)
{
Preconditions.checkNotNull(newFilter, "newFilter");
return new DruidQueryBuilder(newFilter, selectProjection, grouping, having, limitSpec, rowType, rowOrder);
}
public DruidQueryBuilder withSelectProjection(final SelectProjection newProjection, final List<String> newRowOrder)
{
Preconditions.checkState(selectProjection == null, "cannot project twice");
Preconditions.checkState(grouping == null, "cannot project after grouping");
Preconditions.checkNotNull(newProjection, "newProjection");
Preconditions.checkState(
newProjection.getProject().getChildExps().size() == newRowOrder.size(),
"project size[%,d] != rowOrder size[%,d]",
newProjection.getProject().getChildExps().size(),
newRowOrder.size()
);
return new DruidQueryBuilder(
filter,
newProjection,
grouping,
having,
limitSpec,
newProjection.getProject().getRowType(),
newRowOrder
);
}
public DruidQueryBuilder withGrouping(
final Grouping newGrouping,
final RelDataType newRowType,
final List<String> newRowOrder
)
{
Preconditions.checkState(grouping == null, "cannot add grouping twice");
Preconditions.checkState(having == null, "cannot add grouping after having");
Preconditions.checkState(limitSpec == null, "cannot add grouping after limitSpec");
Preconditions.checkNotNull(newGrouping, "newGrouping");
// Set selectProjection to null now that we're grouping. Grouping subsumes select projection.
return new DruidQueryBuilder(filter, null, newGrouping, having, limitSpec, newRowType, newRowOrder);
}
public DruidQueryBuilder withAdjustedGrouping(
final Grouping newGrouping,
final RelDataType newRowType,
final List<String> newRowOrder
)
{
// Like withGrouping, but without any sanity checks. It's assumed that callers will pass something that makes sense.
// This is used when adjusting the Grouping while pushing down a post-Aggregate Project or Sort.
Preconditions.checkNotNull(newGrouping, "newGrouping");
return new DruidQueryBuilder(filter, null, newGrouping, having, limitSpec, newRowType, newRowOrder);
}
public DruidQueryBuilder withHaving(final DimFilter newHaving)
{
Preconditions.checkState(having == null, "cannot add having twice");
Preconditions.checkState(limitSpec == null, "cannot add having after limitSpec");
Preconditions.checkState(grouping != null, "cannot add having before grouping");
Preconditions.checkNotNull(newHaving, "newHaving");
return new DruidQueryBuilder(filter, selectProjection, grouping, newHaving, limitSpec, rowType, rowOrder);
}
public DruidQueryBuilder withLimitSpec(final DefaultLimitSpec newLimitSpec)
{
Preconditions.checkState(limitSpec == null, "cannot add limitSpec twice");
Preconditions.checkNotNull(newLimitSpec, "newLimitSpec");
return new DruidQueryBuilder(filter, selectProjection, grouping, having, newLimitSpec, rowType, rowOrder);
}
public DimFilter getFilter()
{
return filter;
}
public SelectProjection getSelectProjection()
{
return selectProjection;
}
public Grouping getGrouping()
{
return grouping;
}
public DimFilter getHaving()
{
return having;
}
public DefaultLimitSpec getLimitSpec()
{
return limitSpec;
}
public RelDataType getRowType()
{
return rowType;
}
public List<String> getRowOrder()
{
return rowOrder;
}
public RelTrait[] getRelTraits()
{
final List<RelFieldCollation> collations = Lists.newArrayList();
if (limitSpec != null) {
for (OrderByColumnSpec orderBy : limitSpec.getColumns()) {
final int i = rowOrder.indexOf(orderBy.getDimension());
final RelFieldCollation.Direction direction = orderBy.getDirection() == OrderByColumnSpec.Direction.ASCENDING
? RelFieldCollation.Direction.ASCENDING
: RelFieldCollation.Direction.DESCENDING;
collations.add(new RelFieldCollation(i, direction));
}
}
if (!collations.isEmpty()) {
return new RelTrait[]{RelCollations.of(collations)};
} else {
return new RelTrait[]{};
}
}
public void accumulate(
final DruidTable druidTable,
final Function<Row, Void> sink
)
{
final PlannerConfig config = druidTable.getPlannerConfig();
if (grouping == null) {
QueryMaker.executeSelect(druidTable, this, sink);
} else if (asQueryGranularityIfTimeseries() != null) {
QueryMaker.executeTimeseries(druidTable, this, sink);
} else if (asTopNMetricSpecIfTopN(config.getMaxTopNLimit(), config.isUseApproximateTopN()) != null) {
QueryMaker.executeTopN(druidTable, this, sink);
} else {
QueryMaker.executeGroupBy(druidTable, this, sink);
}
}
/**
* Determine if this query can be run as a Timeseries query, and if so, return the query granularity.
*
* @return query granularity, or null
*/
public QueryGranularity asQueryGranularityIfTimeseries()
{
if (grouping == null) {
return null;
}
final List<DimensionSpec> dimensions = grouping.getDimensions();
if (dimensions.isEmpty()) {
return QueryGranularities.ALL;
} else if (dimensions.size() == 1) {
final DimensionSpec dimensionSpec = Iterables.getOnlyElement(dimensions);
final QueryGranularity gran = ExtractionFns.toQueryGranularity(dimensionSpec.getExtractionFn());
if (gran == null || !dimensionSpec.getDimension().equals(Column.TIME_COLUMN_NAME)) {
// Timeseries only applies if the single dimension is granular __time.
return null;
}
if (having != null) {
// Timeseries does not offer HAVING.
return null;
}
// Timeseries only applies if sort is null, or if sort is on the time dimension.
final boolean sortingOnTime =
limitSpec == null || limitSpec.getColumns().isEmpty()
|| (limitSpec.getLimit() == Integer.MAX_VALUE
&& limitSpec.getColumns().size() == 1
&& limitSpec.getColumns().get(0).getDimension().equals(dimensionSpec.getOutputName())
&& limitSpec.getColumns().get(0).getDirection() == OrderByColumnSpec.Direction.ASCENDING);
if (sortingOnTime) {
return ExtractionFns.toQueryGranularity(dimensionSpec.getExtractionFn());
}
}
return null;
}
/**
* Determine if this query can be run as a topN query, and if so, returns the metric spec for ordering.
*
* @param maxTopNLimit maximum limit to consider for conversion to a topN
* @param useApproximateTopN true if we should allow approximate topNs, false otherwise
*
* @return metric spec, or null
*/
public TopNMetricSpec asTopNMetricSpecIfTopN(
final int maxTopNLimit,
final boolean useApproximateTopN
)
{
// Must have GROUP BY one column, ORDER BY one column, limit less than maxTopNLimit, and no HAVING.
if (grouping == null
|| grouping.getDimensions().size() != 1
|| limitSpec == null
|| limitSpec.getColumns().size() != 1
|| limitSpec.getLimit() > maxTopNLimit
|| having != null) {
return null;
}
final DimensionSpec dimensionSpec = Iterables.getOnlyElement(grouping.getDimensions());
final OrderByColumnSpec limitColumn = Iterables.getOnlyElement(limitSpec.getColumns());
if (limitColumn.getDimension().equals(dimensionSpec.getOutputName())) {
// DimensionTopNMetricSpec is exact; always return it even if allowApproximate is false.
final DimensionTopNMetricSpec baseMetricSpec = new DimensionTopNMetricSpec(
null,
limitColumn.getDimensionComparator()
);
return limitColumn.getDirection() == OrderByColumnSpec.Direction.ASCENDING
? baseMetricSpec
: new InvertedTopNMetricSpec(baseMetricSpec);
} else if (useApproximateTopN) {
// ORDER BY metric
final NumericTopNMetricSpec baseMetricSpec = new NumericTopNMetricSpec(limitColumn.getDimension());
return limitColumn.getDirection() == OrderByColumnSpec.Direction.ASCENDING
? new InvertedTopNMetricSpec(baseMetricSpec)
: baseMetricSpec;
} else {
return null;
}
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
DruidQueryBuilder that = (DruidQueryBuilder) o;
if (filter != null ? !filter.equals(that.filter) : that.filter != null) {
return false;
}
if (selectProjection != null ? !selectProjection.equals(that.selectProjection) : that.selectProjection != null) {
return false;
}
if (grouping != null ? !grouping.equals(that.grouping) : that.grouping != null) {
return false;
}
if (having != null ? !having.equals(that.having) : that.having != null) {
return false;
}
if (limitSpec != null ? !limitSpec.equals(that.limitSpec) : that.limitSpec != null) {
return false;
}
if (rowType != null ? !rowType.equals(that.rowType) : that.rowType != null) {
return false;
}
return rowOrder != null ? rowOrder.equals(that.rowOrder) : that.rowOrder == null;
}
@Override
public int hashCode()
{
int result = filter != null ? filter.hashCode() : 0;
result = 31 * result + (selectProjection != null ? selectProjection.hashCode() : 0);
result = 31 * result + (grouping != null ? grouping.hashCode() : 0);
result = 31 * result + (having != null ? having.hashCode() : 0);
result = 31 * result + (limitSpec != null ? limitSpec.hashCode() : 0);
result = 31 * result + (rowType != null ? rowType.hashCode() : 0);
result = 31 * result + (rowOrder != null ? rowOrder.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "DruidQueryBuilder{" +
"filter=" + filter +
", selectProjection=" + selectProjection +
", grouping=" + grouping +
", having=" + having +
", limitSpec=" + limitSpec +
", rowOrder=" + rowOrder +
'}';
}
}

View File

@ -0,0 +1,168 @@
/*
* 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.rel;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import io.druid.sql.calcite.filtration.Filtration;
import io.druid.sql.calcite.table.DruidTable;
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;
import org.apache.calcite.plan.RelOptTable;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelWriter;
import org.apache.calcite.rel.metadata.RelMetadataQuery;
import org.apache.calcite.rel.type.RelDataType;
public class DruidQueryRel extends DruidRel<DruidQueryRel>
{
private final RelOptTable table;
private final DruidTable druidTable;
private final DruidQueryBuilder queryBuilder;
private DruidQueryRel(
final RelOptCluster cluster,
final RelTraitSet traitSet,
final RelOptTable table,
final DruidTable druidTable,
final DruidQueryBuilder queryBuilder
)
{
super(cluster, traitSet);
this.table = Preconditions.checkNotNull(table, "table");
this.druidTable = Preconditions.checkNotNull(druidTable, "druidTable");
this.queryBuilder = Preconditions.checkNotNull(queryBuilder, "queryBuilder");
}
/**
* Create a DruidQueryRel representing a full scan.
*/
public static DruidQueryRel fullScan(
final RelOptCluster cluster,
final RelTraitSet traitSet,
final RelOptTable table,
final DruidTable druidTable
)
{
return new DruidQueryRel(
cluster,
traitSet,
table,
druidTable,
DruidQueryBuilder.fullScan(druidTable, cluster.getTypeFactory())
);
}
public DruidQueryRel asBindable()
{
return new DruidQueryRel(
getCluster(),
getTraitSet().plus(BindableConvention.INSTANCE),
table,
druidTable,
queryBuilder
);
}
public DruidTable getDruidTable()
{
return druidTable;
}
public DruidQueryBuilder getQueryBuilder()
{
return queryBuilder;
}
public DruidQueryRel withQueryBuilder(final DruidQueryBuilder newQueryBuilder)
{
return new DruidQueryRel(
getCluster(),
getTraitSet().plusAll(newQueryBuilder.getRelTraits()),
table,
druidTable,
newQueryBuilder
);
}
@Override
public void accumulate(final Function<Row, Void> sink)
{
queryBuilder.accumulate(druidTable, sink);
}
@Override
public RelOptTable getTable()
{
return table;
}
@Override
public Class<Object[]> getElementType()
{
return Object[].class;
}
@Override
protected RelDataType deriveRowType()
{
return queryBuilder.getRowType();
}
@Override
public RelWriter explainTerms(final RelWriter pw)
{
pw.item("dataSource", druidTable.getDataSource());
if (queryBuilder != null) {
final Filtration filtration = Filtration.create(queryBuilder.getFilter()).optimize(druidTable);
if (!filtration.getIntervals().equals(ImmutableList.of(Filtration.eternity()))) {
pw.item("intervals", filtration.getIntervals());
}
if (filtration.getDimFilter() != null) {
pw.item("filter", filtration.getDimFilter());
}
if (queryBuilder.getSelectProjection() != null) {
pw.item("selectDimensions", queryBuilder.getSelectProjection().getDimensions());
pw.item("selectMetrics", queryBuilder.getSelectProjection().getMetrics());
}
if (queryBuilder.getGrouping() != null) {
pw.item("dimensions", queryBuilder.getGrouping().getDimensions());
pw.item("aggregations", queryBuilder.getGrouping().getAggregations());
}
if (queryBuilder.getHaving() != null) {
pw.item("having", queryBuilder.getHaving());
}
if (queryBuilder.getLimitSpec() != null) {
pw.item("limitSpec", queryBuilder.getLimitSpec());
}
}
return pw;
}
@Override
public RelOptCost computeSelfCost(final RelOptPlanner planner, final RelMetadataQuery mq)
{
return super.computeSelfCost(planner, mq).multiplyBy(0.1);
}
}

View File

@ -0,0 +1,70 @@
/*
* 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.rel;
import com.google.common.base.Function;
import io.druid.sql.calcite.table.DruidTable;
import org.apache.calcite.DataContext;
import org.apache.calcite.interpreter.BindableRel;
import org.apache.calcite.interpreter.Node;
import org.apache.calcite.interpreter.Row;
import org.apache.calcite.interpreter.Sink;
import org.apache.calcite.linq4j.Enumerable;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.AbstractRelNode;
public abstract class DruidRel<T extends DruidRel> extends AbstractRelNode implements BindableRel
{
public DruidRel(RelOptCluster cluster, RelTraitSet traitSet)
{
super(cluster, traitSet);
}
public abstract DruidTable getDruidTable();
public abstract DruidQueryBuilder getQueryBuilder();
public abstract void accumulate(Function<Row, Void> sink);
public abstract T withQueryBuilder(DruidQueryBuilder newQueryBuilder);
public abstract T asBindable();
@Override
public Node implement(InterpreterImplementor implementor)
{
final Sink sink = implementor.interpreter.sink(this);
return new Node()
{
@Override
public void run() throws InterruptedException
{
accumulate(QueryMaker.sinkFunction(sink));
}
};
}
@Override
public Enumerable<Object[]> bind(final DataContext dataContext)
{
throw new UnsupportedOperationException();
}
}

View File

@ -0,0 +1,317 @@
/*
* 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.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.query.ResourceLimitExceededException;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DimFilter;
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.DruidTable;
import io.druid.sql.calcite.table.DruidTables;
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;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelWriter;
import org.apache.calcite.rel.core.SemiJoin;
import org.apache.calcite.rel.metadata.RelMetadataQuery;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rex.RexNode;
import java.util.List;
import java.util.Set;
public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
{
private final SemiJoin semiJoin;
private final DruidRel left;
private final DruidRel right;
private final RexNode condition;
private final List<RowExtraction> leftRowExtractions;
private final List<Integer> rightKeys;
private final int maxSemiJoinRowsInMemory;
private DruidSemiJoin(
final RelOptCluster cluster,
final RelTraitSet traitSet,
final SemiJoin semiJoin,
final DruidRel left,
final DruidRel right,
final RexNode condition,
final List<RowExtraction> leftRowExtractions,
final List<Integer> rightKeys,
final int maxSemiJoinRowsInMemory
)
{
super(cluster, traitSet);
this.semiJoin = semiJoin;
this.left = left;
this.right = right;
this.condition = condition;
this.leftRowExtractions = ImmutableList.copyOf(leftRowExtractions);
this.rightKeys = ImmutableList.copyOf(rightKeys);
this.maxSemiJoinRowsInMemory = maxSemiJoinRowsInMemory;
}
public static DruidSemiJoin from(
final SemiJoin semiJoin,
final RelTraitSet traitSet,
final DruidRel left,
final DruidRel right
)
{
if (semiJoin.getLeftKeys().size() != semiJoin.getRightKeys().size()) {
throw new ISE("WTF?! SemiJoin with different left/right key count?");
}
final ImmutableList.Builder<RowExtraction> listBuilder = ImmutableList.builder();
for (Integer key : semiJoin.getLeftKeys()) {
final RowExtraction rex = RowExtraction.fromQueryBuilder(left.getQueryBuilder(), key);
if (rex == null) {
// Can't figure out what to filter the left-hand side on...
return null;
}
listBuilder.add(rex);
}
return new DruidSemiJoin(
semiJoin.getCluster(),
traitSet,
semiJoin,
left,
right,
semiJoin.getCondition(),
listBuilder.build(),
semiJoin.getRightKeys(),
right.getDruidTable().getPlannerConfig().getMaxSemiJoinRowsInMemory()
);
}
@Override
public Class<Object[]> getElementType()
{
return Object[].class;
}
@Override
public DruidTable getDruidTable()
{
return left.getDruidTable();
}
@Override
public DruidQueryBuilder getQueryBuilder()
{
return left.getQueryBuilder();
}
@Override
public DruidSemiJoin withQueryBuilder(final DruidQueryBuilder newQueryBuilder)
{
return new DruidSemiJoin(
getCluster(),
getTraitSet().plusAll(newQueryBuilder.getRelTraits()),
semiJoin,
left.withQueryBuilder(newQueryBuilder),
right,
condition,
leftRowExtractions,
rightKeys,
maxSemiJoinRowsInMemory
);
}
@Override
public DruidSemiJoin asBindable()
{
return new DruidSemiJoin(
getCluster(),
getTraitSet().plus(BindableConvention.INSTANCE),
semiJoin,
left,
right,
condition,
leftRowExtractions,
rightKeys,
maxSemiJoinRowsInMemory
);
}
@Override
public void accumulate(final Function<Row, Void> sink)
{
final Pair<DruidQueryBuilder, List<Integer>> pair = getRightQueryBuilderWithGrouping();
final DruidQueryBuilder rightQueryBuilderAdjusted = 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();
rightQueryBuilderAdjusted.accumulate(
right.getDruidTable(),
new Function<Row, Void>()
{
@Override
public Void apply(final Row row)
{
final List<String> values = Lists.newArrayListWithCapacity(rightKeysAdjusted.size());
for (int i : rightKeysAdjusted) {
final Object value = row.getObject(i);
final String stringValue = value != null ? String.valueOf(value) : "";
values.add(stringValue);
if (values.size() > maxSemiJoinRowsInMemory) {
throw new ResourceLimitExceededException(
String.format("maxSemiJoinRowsInMemory[%,d] exceeded", maxSemiJoinRowsInMemory)
);
}
}
if (valuess.add(values)) {
final List<DimFilter> bounds = Lists.newArrayList();
for (int i = 0; i < values.size(); i++) {
bounds.add(
new BoundDimFilter(
leftRowExtractions.get(i).getColumn(),
values.get(i),
values.get(i),
false,
false,
null,
leftRowExtractions.get(i).getExtractionFn(),
DruidTables.naturalStringComparator(getDruidTable(), leftRowExtractions.get(i))
)
);
}
filters.add(new AndDimFilter(bounds));
}
return null;
}
}
);
valuess.clear();
if (!filters.isEmpty()) {
// Add a filter to the left side. Use OR of singleton Bound filters so they can be simplified later.
final DimFilter semiJoinFilter = new OrDimFilter(filters);
final DimFilter newFilter = left.getQueryBuilder().getFilter() == null
? semiJoinFilter
: new AndDimFilter(
ImmutableList.of(
semiJoinFilter,
left.getQueryBuilder().getFilter()
)
);
left.getQueryBuilder().withFilter(newFilter).accumulate(
left.getDruidTable(),
sink
);
}
}
@Override
public Enumerable<Object[]> bind(final DataContext dataContext)
{
throw new UnsupportedOperationException();
}
@Override
protected RelDataType deriveRowType()
{
return left.getRowType();
}
@Override
public RelWriter explainTerms(RelWriter pw)
{
final Pair<DruidQueryBuilder, List<Integer>> rightQueryBuilderWithGrouping = getRightQueryBuilderWithGrouping();
return pw
.item("leftDataSource", left.getDruidTable().getDataSource())
.item("leftRowExtractions", leftRowExtractions)
.item("leftQuery", left.getQueryBuilder())
.item("rightDataSource", right.getDruidTable().getDataSource())
.item("rightKeysAdjusted", rightQueryBuilderWithGrouping.rhs)
.item("rightQuery", rightQueryBuilderWithGrouping.lhs);
}
@Override
public RelOptCost computeSelfCost(final RelOptPlanner planner, final RelMetadataQuery mq)
{
return semiJoin.computeSelfCost(planner, mq).multiplyBy(0.1);
}
private Pair<DruidQueryBuilder, List<Integer>> getRightQueryBuilderWithGrouping()
{
if (right.getQueryBuilder().getGrouping() != null) {
return Pair.of(right.getQueryBuilder(), rightKeys);
} else {
// Add grouping on the join key to limit resultset from data nodes.
final List<DimensionSpec> dimensionSpecs = Lists.newArrayList();
final List<RelDataType> rowTypes = Lists.newArrayList();
final List<String> rowOrder = Lists.newArrayList();
final List<Integer> rightKeysAdjusted = Lists.newArrayList();
int counter = 0;
for (final int key : rightKeys) {
final String keyDimensionOutputName = "v" + key;
final RowExtraction rex = RowExtraction.fromQueryBuilder(right.getQueryBuilder(), key);
if (rex == null) {
throw new ISE("WTF?! Can't find dimensionSpec to group on!");
}
final DimensionSpec dimensionSpec = rex.toDimensionSpec(left.getDruidTable(), keyDimensionOutputName);
if (dimensionSpec == null) {
throw new ISE("WTF?! Can't translate row expression to dimensionSpec: %s", rex);
}
dimensionSpecs.add(dimensionSpec);
rowTypes.add(right.getQueryBuilder().getRowType().getFieldList().get(key).getType());
rowOrder.add(dimensionSpec.getOutputName());
rightKeysAdjusted.add(counter++);
}
final DruidQueryBuilder newQueryBuilder = right
.getQueryBuilder()
.withGrouping(
Grouping.create(dimensionSpecs, ImmutableList.<Aggregation>of()),
getCluster().getTypeFactory().createStructType(rowTypes, rowOrder),
rowOrder
);
return Pair.of(newQueryBuilder, rightKeysAdjusted);
}
}
}

View File

@ -0,0 +1,139 @@
/*
* 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.rel;
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.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.dimension.DimensionSpec;
import io.druid.sql.calcite.aggregation.Aggregation;
import java.util.List;
import java.util.Set;
public class Grouping
{
private final List<DimensionSpec> dimensions;
private final List<Aggregation> aggregations;
private Grouping(
final List<DimensionSpec> dimensions,
final List<Aggregation> aggregations
)
{
this.dimensions = ImmutableList.copyOf(dimensions);
this.aggregations = ImmutableList.copyOf(aggregations);
// Verify no collisions.
final Set<String> seen = Sets.newHashSet();
for (DimensionSpec dimensionSpec : dimensions) {
if (!seen.add(dimensionSpec.getOutputName())) {
throw new ISE("Duplicate field name: %s", dimensionSpec.getOutputName());
}
}
for (Aggregation aggregation : aggregations) {
for (AggregatorFactory aggregatorFactory : aggregation.getAggregatorFactories()) {
if (!seen.add(aggregatorFactory.getName())) {
throw new ISE("Duplicate field name: %s", aggregatorFactory.getName());
}
}
if (aggregation.getPostAggregator() != null && !seen.add(aggregation.getPostAggregator().getName())) {
throw new ISE("Duplicate field name in rowOrder: %s", aggregation.getPostAggregator().getName());
}
}
}
public static Grouping create(
final List<DimensionSpec> dimensions,
final List<Aggregation> aggregations
)
{
return new Grouping(dimensions, aggregations);
}
public List<DimensionSpec> getDimensions()
{
return dimensions;
}
public List<Aggregation> getAggregations()
{
return aggregations;
}
public List<AggregatorFactory> getAggregatorFactories()
{
final List<AggregatorFactory> retVal = Lists.newArrayList();
for (final Aggregation aggregation : aggregations) {
retVal.addAll(aggregation.getAggregatorFactories());
}
return retVal;
}
public List<PostAggregator> getPostAggregators()
{
final List<PostAggregator> retVal = Lists.newArrayList();
for (final Aggregation aggregation : aggregations) {
if (aggregation.getPostAggregator() != null) {
retVal.add(aggregation.getPostAggregator());
}
}
return retVal;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Grouping grouping = (Grouping) o;
if (dimensions != null ? !dimensions.equals(grouping.dimensions) : grouping.dimensions != null) {
return false;
}
return aggregations != null ? aggregations.equals(grouping.aggregations) : grouping.aggregations == null;
}
@Override
public int hashCode()
{
int result = dimensions != null ? dimensions.hashCode() : 0;
result = 31 * result + (aggregations != null ? aggregations.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "Grouping{" +
"dimensions=" + dimensions +
", aggregations=" + aggregations +
'}';
}
}

View File

@ -0,0 +1,435 @@
/*
* 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.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.ImmutableList;
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.granularity.QueryGranularities;
import io.druid.granularity.QueryGranularity;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.guava.Accumulator;
import io.druid.java.util.common.logger.Logger;
import io.druid.query.Result;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.groupby.GroupByQuery;
import io.druid.query.groupby.having.DimFilterHavingSpec;
import io.druid.query.groupby.orderby.OrderByColumnSpec;
import io.druid.query.select.EventHolder;
import io.druid.query.select.PagingSpec;
import io.druid.query.select.SelectQuery;
import io.druid.query.select.SelectResultValue;
import io.druid.query.timeseries.TimeseriesQuery;
import io.druid.query.timeseries.TimeseriesResultValue;
import io.druid.query.topn.DimensionAndMetricValueExtractor;
import io.druid.query.topn.TopNMetricSpec;
import io.druid.query.topn.TopNQuery;
import io.druid.query.topn.TopNResultValue;
import io.druid.segment.column.Column;
import io.druid.sql.calcite.filtration.Filtration;
import io.druid.sql.calcite.table.DruidTable;
import org.apache.calcite.interpreter.Row;
import org.apache.calcite.interpreter.Sink;
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.Calendar;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
public class QueryMaker
{
private final static Logger log = new Logger(QueryMaker.class);
private QueryMaker()
{
// No instantiation.
}
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 static void executeSelect(
final DruidTable druidTable,
final DruidQueryBuilder queryBuilder,
final Function<Row, Void> sink
)
{
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 Filtration filtration = Filtration.create(queryBuilder.getFilter()).optimize(druidTable);
final SelectProjection selectProjection = queryBuilder.getSelectProjection();
final Integer limit;
final boolean descending;
if (queryBuilder.getLimitSpec() != null) {
limit = queryBuilder.getLimitSpec().getLimit();
// Safe to assume limitSpec has zero or one entry; DruidSelectSortRule wouldn't push in anything else.
if (queryBuilder.getLimitSpec().getColumns().size() > 0) {
final OrderByColumnSpec orderBy = Iterables.getOnlyElement(queryBuilder.getLimitSpec().getColumns());
if (!orderBy.getDimension().equals(Column.TIME_COLUMN_NAME)) {
throw new ISE("WTF?! Got select with non-time orderBy[%s]", orderBy);
}
descending = orderBy.getDirection() == OrderByColumnSpec.Direction.DESCENDING;
} else {
descending = false;
}
} else {
limit = null;
descending = false;
}
// 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 = new SelectQuery(
druidTable.getDataSource(),
filtration.getQuerySegmentSpec(),
descending,
filtration.getDimFilter(),
QueryGranularities.ALL,
selectProjection != null ? selectProjection.getDimensions() : ImmutableList.<DimensionSpec>of(),
selectProjection != null ? selectProjection.getMetrics() : ImmutableList.<String>of(),
null,
new PagingSpec(pagingIdentifiers.get(), druidTable.getPlannerConfig().getSelectThreshold(), true),
null
);
Hook.QUERY_PLAN.run(query);
morePages.set(false);
final AtomicBoolean gotResult = new AtomicBoolean();
query.run(druidTable.getQuerySegmentWalker(), Maps.<String, Object>newHashMap()).accumulate(
null,
new Accumulator<Object, Result<SelectResultValue>>()
{
@Override
public Object accumulate(final Object accumulated, 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());
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();
}
return null;
}
}
);
}
}
public static void executeTimeseries(
final DruidTable druidTable,
final DruidQueryBuilder queryBuilder,
final Function<Row, Void> sink
)
{
final QueryGranularity queryGranularity = queryBuilder.asQueryGranularityIfTimeseries();
if (queryGranularity == null) {
throw new ISE("WTF?! executeTimeseries called on query that cannot become a timeseries?!");
}
final String timeOutputName = queryBuilder.getGrouping().getDimensions().size() == 1
? queryBuilder.getGrouping().getDimensions().get(0).getOutputName()
: null;
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
final Row.RowBuilder rowBuilder = Row.newBuilder(fieldList.size());
final Filtration filtration = Filtration.create(queryBuilder.getFilter()).optimize(druidTable);
final Map<String, Object> context = Maps.newHashMap();
context.put("skipEmptyBuckets", true);
final TimeseriesQuery query = new TimeseriesQuery(
druidTable.getDataSource(),
filtration.getQuerySegmentSpec(),
false,
filtration.getDimFilter(),
queryGranularity,
queryBuilder.getGrouping().getAggregatorFactories(),
queryBuilder.getGrouping().getPostAggregators(),
context
);
Hook.QUERY_PLAN.run(query);
query.run(druidTable.getQuerySegmentWalker(), Maps.<String, Object>newHashMap()).accumulate(
null,
new Accumulator<Object, Result<TimeseriesResultValue>>()
{
@Override
public Object accumulate(final Object accumulated, final Result<TimeseriesResultValue> result)
{
final Map<String, Object> row = result.getValue().getBaseObject();
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()));
} else {
rowBuilder.set(field.getIndex(), coerce(row.get(outputName), field.getType().getSqlTypeName()));
}
}
sink.apply(rowBuilder.build());
rowBuilder.reset();
return null;
}
}
);
}
public static void executeTopN(
final DruidTable druidTable,
final DruidQueryBuilder queryBuilder,
final Function<Row, Void> sink
)
{
// OK to hard-code permissive values here; this method is only called if we really do want a topN.
final TopNMetricSpec topNMetricSpec = queryBuilder.asTopNMetricSpecIfTopN(Integer.MAX_VALUE, true);
if (topNMetricSpec == null) {
throw new ISE("WTF?! executeTopN called on query that cannot become a topN?!");
}
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
final Row.RowBuilder rowBuilder = Row.newBuilder(fieldList.size());
final Filtration filtration = Filtration.create(queryBuilder.getFilter()).optimize(druidTable);
final TopNQuery query = new TopNQuery(
druidTable.getDataSource(),
Iterables.getOnlyElement(queryBuilder.getGrouping().getDimensions()),
topNMetricSpec,
queryBuilder.getLimitSpec().getLimit(),
filtration.getQuerySegmentSpec(),
filtration.getDimFilter(),
QueryGranularities.ALL,
queryBuilder.getGrouping().getAggregatorFactories(),
queryBuilder.getGrouping().getPostAggregators(),
null
);
Hook.QUERY_PLAN.run(query);
query.run(druidTable.getQuerySegmentWalker(), 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();
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()));
}
sink.apply(rowBuilder.build());
rowBuilder.reset();
}
return null;
}
}
);
}
public static void executeGroupBy(
final DruidTable druidTable,
final DruidQueryBuilder queryBuilder,
final Function<Row, Void> sink
)
{
Preconditions.checkState(queryBuilder.getGrouping() != null, "grouping must be non-null");
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
final Row.RowBuilder rowBuilder = Row.newBuilder(fieldList.size());
final Filtration filtration = Filtration.create(queryBuilder.getFilter()).optimize(druidTable);
final GroupByQuery query = new GroupByQuery(
druidTable.getDataSource(),
filtration.getQuerySegmentSpec(),
filtration.getDimFilter(),
QueryGranularities.ALL,
queryBuilder.getGrouping().getDimensions(),
queryBuilder.getGrouping().getAggregatorFactories(),
queryBuilder.getGrouping().getPostAggregators(),
queryBuilder.getHaving() != null ? new DimFilterHavingSpec(queryBuilder.getHaving()) : null,
queryBuilder.getLimitSpec(),
null
);
Hook.QUERY_PLAN.run(query);
query.run(druidTable.getQuerySegmentWalker(), Maps.<String, Object>newHashMap()).accumulate(
null,
new Accumulator<Object, io.druid.data.input.Row>()
{
@Override
public Object accumulate(final Object accumulated, final io.druid.data.input.Row row)
{
for (RelDataTypeField field : fieldList) {
rowBuilder.set(
field.getIndex(),
coerce(
row.getRaw(queryBuilder.getRowOrder().get(field.getIndex())),
field.getType().getSqlTypeName()
)
);
}
sink.apply(rowBuilder.build());
rowBuilder.reset();
return null;
}
}
);
}
private static Object coerce(final Object value, final SqlTypeName sqlType)
{
final Object coercedValue;
if (SqlTypeName.CHAR_TYPES.contains(sqlType)) {
if (value == null || value instanceof String) {
coercedValue = Strings.nullToEmpty((String) value);
} else if (value instanceof NlsString) {
coercedValue = ((NlsString) value).getValue();
} else {
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
}
} else if (value == null) {
coercedValue = null;
} else if (sqlType == SqlTypeName.DATE) {
final Long millis = (Long) coerce(value, SqlTypeName.TIMESTAMP);
if (millis == null) {
return null;
} else {
return new DateTime(millis.longValue()).dayOfMonth().roundFloorCopy().getMillis();
}
} else if (sqlType == SqlTypeName.TIMESTAMP) {
if (value instanceof Number) {
coercedValue = new DateTime(((Number) value).longValue()).getMillis();
} else if (value instanceof String) {
coercedValue = Long.parseLong((String) value);
} else if (value instanceof Calendar) {
coercedValue = ((Calendar) value).getTimeInMillis();
} else if (value instanceof DateTime) {
coercedValue = ((DateTime) value).getMillis();
} else {
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
}
} else if (sqlType == SqlTypeName.INTEGER) {
if (value instanceof String) {
coercedValue = Ints.tryParse((String) value);
} else if (value instanceof Number) {
coercedValue = ((Number) value).intValue();
} else {
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
}
} else if (sqlType == SqlTypeName.BIGINT) {
if (value instanceof String) {
coercedValue = GuavaUtils.tryParseLong((String) value);
} else if (value instanceof Number) {
coercedValue = ((Number) value).longValue();
} else {
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
}
} else if (sqlType == SqlTypeName.FLOAT || sqlType == SqlTypeName.DOUBLE) {
if (value instanceof String) {
coercedValue = Doubles.tryParse((String) value);
} else if (value instanceof Number) {
coercedValue = ((Number) value).doubleValue();
} else {
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
}
} else {
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
}
return coercedValue;
}
}

View File

@ -0,0 +1,116 @@
/*
* 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.rel;
import com.google.common.collect.Sets;
import io.druid.java.util.common.ISE;
import io.druid.query.dimension.DimensionSpec;
import io.druid.segment.column.Column;
import org.apache.calcite.rel.core.Project;
import java.util.List;
import java.util.Set;
public class SelectProjection
{
private final Project project;
private final List<DimensionSpec> dimensions;
private final List<String> metrics;
public SelectProjection(
final Project project,
final List<DimensionSpec> dimensions,
final List<String> metrics
)
{
this.project = project;
this.dimensions = dimensions;
this.metrics = metrics;
// Verify no collisions. Start with TIME_COLUMN_NAME because QueryMaker.executeSelect hard-codes it.
final Set<String> seen = Sets.newHashSet(Column.TIME_COLUMN_NAME);
for (DimensionSpec dimensionSpec : dimensions) {
if (!seen.add(dimensionSpec.getOutputName())) {
throw new ISE("Duplicate field name: %s", dimensionSpec.getOutputName());
}
}
for (String fieldName : metrics) {
if (!seen.add(fieldName)) {
throw new ISE("Duplicate field name: %s", fieldName);
}
}
}
public Project getProject()
{
return project;
}
public List<DimensionSpec> getDimensions()
{
return dimensions;
}
public List<String> getMetrics()
{
return metrics;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SelectProjection that = (SelectProjection) o;
if (project != null ? !project.equals(that.project) : that.project != null) {
return false;
}
if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) {
return false;
}
return metrics != null ? metrics.equals(that.metrics) : that.metrics == null;
}
@Override
public int hashCode()
{
int result = project != null ? project.hashCode() : 0;
result = 31 * result + (dimensions != null ? dimensions.hashCode() : 0);
result = 31 * result + (metrics != null ? metrics.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "SelectProjection{" +
"project=" + project +
", dimensions=" + dimensions +
", metrics=" + metrics +
'}';
}
}

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.rule;
import io.druid.sql.calcite.rel.DruidRel;
import org.apache.calcite.interpreter.BindableConvention;
import org.apache.calcite.plan.Convention;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.convert.ConverterRule;
public class DruidBindableConverterRule extends ConverterRule
{
private static DruidBindableConverterRule INSTANCE = new DruidBindableConverterRule();
private DruidBindableConverterRule()
{
super(
DruidRel.class,
Convention.NONE,
BindableConvention.INSTANCE,
DruidBindableConverterRule.class.getSimpleName()
);
}
public static DruidBindableConverterRule instance()
{
return INSTANCE;
}
@Override
public RelNode convert(RelNode rel)
{
final DruidRel druidRel = (DruidRel) rel;
return druidRel.asBindable();
}
}

View File

@ -0,0 +1,66 @@
/*
* 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.query.filter.DimFilter;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.rel.DruidRel;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.rel.core.Filter;
public class DruidFilterRule extends RelOptRule
{
private static final DruidFilterRule INSTANCE = new DruidFilterRule();
private DruidFilterRule()
{
super(operand(Filter.class, operand(DruidRel.class, none())));
}
public static DruidFilterRule instance()
{
return INSTANCE;
}
@Override
public void onMatch(RelOptRuleCall call)
{
final Filter filter = call.rel(0);
final DruidRel druidRel = call.rel(1);
if (druidRel.getQueryBuilder().getFilter() != null
|| druidRel.getQueryBuilder().getSelectProjection() != null
|| druidRel.getQueryBuilder().getGrouping() != null) {
return;
}
final DimFilter dimFilter = Expressions.toFilter(
druidRel.getDruidTable(),
druidRel.getQueryBuilder().getRowOrder(),
filter.getCondition()
);
if (dimFilter != null) {
call.transformTo(
druidRel.withQueryBuilder(druidRel.getQueryBuilder().withFilter(dimFilter))
);
}
}
}

View File

@ -0,0 +1,125 @@
/*
* 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 com.google.common.collect.Lists;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.dimension.ExtractionDimensionSpec;
import io.druid.query.extraction.ExtractionFn;
import io.druid.segment.column.Column;
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.rel.DruidRel;
import io.druid.sql.calcite.rel.SelectProjection;
import io.druid.sql.calcite.table.DruidTables;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.rel.core.Project;
import java.util.List;
public class DruidSelectProjectionRule extends RelOptRule
{
private static final DruidSelectProjectionRule INSTANCE = new DruidSelectProjectionRule();
private DruidSelectProjectionRule()
{
super(operand(Project.class, operand(DruidRel.class, none())));
}
public static DruidSelectProjectionRule instance()
{
return INSTANCE;
}
@Override
public void onMatch(RelOptRuleCall call)
{
final Project project = call.rel(0);
final DruidRel druidRel = call.rel(1);
if (druidRel.getQueryBuilder().getSelectProjection() != null
|| druidRel.getQueryBuilder().getGrouping() != null
|| druidRel.getQueryBuilder().getLimitSpec() != null) {
return;
}
// Only push in projections that can be used by the Select query.
// Leave anything more complicated to DruidAggregateProjectRule for possible handling in a GroupBy query.
final List<DimensionSpec> dimensions = Lists.newArrayList();
final List<String> metrics = Lists.newArrayList();
final List<String> rowOrder = Lists.newArrayList();
int dimOutputNameCounter = 0;
for (int i = 0; i < project.getRowType().getFieldCount(); i++) {
final RowExtraction rex = Expressions.toRowExtraction(
DruidTables.rowOrder(druidRel.getDruidTable()),
project.getChildExps().get(i)
);
if (rex == null) {
return;
}
final String column = rex.getColumn();
final ExtractionFn extractionFn = rex.getExtractionFn();
// Check if this field should be a dimension, a metric, or a reference to __time.
final ValueType columnType = druidRel.getDruidTable()
.getColumnType(druidRel.getDruidTable().getColumnNumber(column));
if (columnType == ValueType.STRING || (column.equals(Column.TIME_COLUMN_NAME) && extractionFn != null)) {
// Add to dimensions.
do {
dimOutputNameCounter++;
} while (druidRel.getDruidTable().getColumnNumber(GroupByRules.dimOutputName(dimOutputNameCounter)) >= 0);
final String outputName = GroupByRules.dimOutputName(dimOutputNameCounter);
final DimensionSpec dimensionSpec = extractionFn == null
? new DefaultDimensionSpec(column, outputName)
: new ExtractionDimensionSpec(column, outputName, extractionFn);
dimensions.add(dimensionSpec);
rowOrder.add(outputName);
} else if (extractionFn == null && !column.equals(Column.TIME_COLUMN_NAME)) {
// Add to metrics.
metrics.add(column);
rowOrder.add(column);
} else if (extractionFn == null && column.equals(Column.TIME_COLUMN_NAME)) {
// This is __time.
rowOrder.add(Column.TIME_COLUMN_NAME);
} else {
// Don't know what to do!
return;
}
}
call.transformTo(
druidRel.withQueryBuilder(
druidRel.getQueryBuilder()
.withSelectProjection(
new SelectProjection(project, dimensions, metrics),
rowOrder
)
)
);
}
}

View File

@ -0,0 +1,74 @@
/*
* 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.query.groupby.orderby.DefaultLimitSpec;
import io.druid.query.groupby.orderby.OrderByColumnSpec;
import io.druid.segment.column.Column;
import io.druid.sql.calcite.rel.DruidRel;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.rel.core.Sort;
import java.util.List;
public class DruidSelectSortRule extends RelOptRule
{
private static final DruidSelectSortRule INSTANCE = new DruidSelectSortRule();
private DruidSelectSortRule()
{
super(operand(Sort.class, operand(DruidRel.class, none())));
}
public static DruidSelectSortRule instance()
{
return INSTANCE;
}
@Override
public void onMatch(RelOptRuleCall call)
{
final Sort sort = call.rel(0);
final DruidRel druidRel = call.rel(1);
if (druidRel.getQueryBuilder().getGrouping() != null
|| druidRel.getQueryBuilder().getLimitSpec() != null) {
return;
}
final DefaultLimitSpec limitSpec = GroupByRules.toLimitSpec(druidRel.getQueryBuilder().getRowOrder(), sort);
if (limitSpec == null) {
return;
}
// Only push in sorts that can be used by the Select query.
final List<OrderByColumnSpec> orderBys = limitSpec.getColumns();
if (orderBys.isEmpty() ||
(orderBys.size() == 1 && orderBys.get(0).getDimension().equals(Column.TIME_COLUMN_NAME))) {
call.transformTo(
druidRel.withQueryBuilder(
druidRel.getQueryBuilder()
.withLimitSpec(limitSpec)
)
);
}
}
}

View File

@ -0,0 +1,59 @@
/*
* 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.DruidRel;
import io.druid.sql.calcite.rel.DruidSemiJoin;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.rel.core.SemiJoin;
public class DruidSemiJoinRule extends RelOptRule
{
private static final DruidSemiJoinRule INSTANCE = new DruidSemiJoinRule();
private DruidSemiJoinRule()
{
super(
operand(
SemiJoin.class,
operand(DruidRel.class, none()),
operand(DruidRel.class, none())
)
);
}
public static DruidSemiJoinRule instance()
{
return INSTANCE;
}
@Override
public void onMatch(RelOptRuleCall call)
{
final SemiJoin semiJoin = call.rel(0);
final DruidRel left = call.rel(1);
final DruidRel right = call.rel(2);
final DruidSemiJoin druidSemiJoin = DruidSemiJoin.from(semiJoin, semiJoin.getTraitSet(), left, right);
if (druidSemiJoin != null) {
call.transformTo(druidSemiJoin);
}
}
}

View File

@ -0,0 +1,787 @@
/*
* 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 com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import io.druid.java.util.common.ISE;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
import io.druid.query.aggregation.DoubleMinAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
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.DefaultDimensionSpec;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.dimension.ExtractionDimensionSpec;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.NotDimFilter;
import io.druid.query.groupby.orderby.DefaultLimitSpec;
import io.druid.query.groupby.orderby.OrderByColumnSpec;
import io.druid.query.ordering.StringComparator;
import io.druid.query.ordering.StringComparators;
import io.druid.segment.column.Column;
import io.druid.segment.column.ValueType;
import io.druid.sql.calcite.aggregation.Aggregation;
import io.druid.sql.calcite.aggregation.PostAggregatorFactory;
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.PlannerConfig;
import io.druid.sql.calcite.rel.DruidRel;
import io.druid.sql.calcite.rel.Grouping;
import io.druid.sql.calcite.table.DruidTable;
import io.druid.sql.calcite.table.DruidTables;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.rel.RelFieldCollation;
import org.apache.calcite.rel.core.Aggregate;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Filter;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rel.core.Sort;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexInputRef;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.util.ImmutableBitSet;
import java.util.List;
import java.util.Map;
public class GroupByRules
{
private GroupByRules()
{
// No instantiation.
}
public static List<RelOptRule> rules(final PlannerConfig plannerConfig)
{
return ImmutableList.of(
new DruidAggregateRule(plannerConfig.isUseApproximateCountDistinct()),
new DruidAggregateProjectRule(plannerConfig.isUseApproximateCountDistinct()),
new DruidProjectAfterAggregationRule(),
new DruidFilterAfterAggregationRule(),
new DruidGroupBySortRule()
);
}
/**
* Used to represent inputs to aggregators. Ideally this should be folded into {@link RowExtraction}, but we
* can't do that until RowExtractions are a bit more versatile.
*/
private static class FieldOrExpression
{
private final String fieldName;
private final String expression;
public FieldOrExpression(String fieldName, String expression)
{
this.fieldName = fieldName;
this.expression = expression;
Preconditions.checkArgument(fieldName == null ^ expression == null, "must have either fieldName or expression");
}
public static FieldOrExpression fromRexNode(final List<String> rowOrder, final RexNode rexNode)
{
final RowExtraction rex = Expressions.toRowExtraction(rowOrder, rexNode);
if (rex != null && rex.getExtractionFn() == null) {
// This was a simple field access.
return fieldName(rex.getColumn());
}
// Try as a math expression.
final String mathExpression = Expressions.toMathExpression(rowOrder, rexNode);
if (mathExpression != null) {
return expression(mathExpression);
}
return null;
}
public static FieldOrExpression fieldName(final String fieldName)
{
return new FieldOrExpression(fieldName, null);
}
public static FieldOrExpression expression(final String expression)
{
return new FieldOrExpression(null, expression);
}
public String getFieldName()
{
return fieldName;
}
public String getExpression()
{
return expression;
}
}
public static class DruidAggregateRule extends RelOptRule
{
final boolean approximateCountDistinct;
private DruidAggregateRule(final boolean approximateCountDistinct)
{
super(operand(Aggregate.class, operand(DruidRel.class, none())));
this.approximateCountDistinct = approximateCountDistinct;
}
@Override
public void onMatch(RelOptRuleCall call)
{
final Aggregate aggregate = call.rel(0);
final DruidRel druidRel = call.rel(1);
final DruidRel newDruidRel = GroupByRules.applyAggregate(
druidRel,
null,
aggregate,
approximateCountDistinct
);
if (newDruidRel != null) {
call.transformTo(newDruidRel);
}
}
}
public static class DruidAggregateProjectRule extends RelOptRule
{
final boolean approximateCountDistinct;
private DruidAggregateProjectRule(final boolean approximateCountDistinct)
{
super(operand(Aggregate.class, operand(Project.class, operand(DruidRel.class, none()))));
this.approximateCountDistinct = approximateCountDistinct;
}
@Override
public void onMatch(RelOptRuleCall call)
{
final Aggregate aggregate = call.rel(0);
final Project project = call.rel(1);
final DruidRel druidRel = call.rel(2);
final DruidRel newDruidRel = GroupByRules.applyAggregate(
druidRel,
project,
aggregate,
approximateCountDistinct
);
if (newDruidRel != null) {
call.transformTo(newDruidRel);
}
}
}
public static class DruidProjectAfterAggregationRule extends RelOptRule
{
private DruidProjectAfterAggregationRule()
{
super(operand(Project.class, operand(DruidRel.class, none())));
}
@Override
public void onMatch(RelOptRuleCall call)
{
final Project postProject = call.rel(0);
final DruidRel druidRel = call.rel(1);
final DruidRel newDruidRel = GroupByRules.applyProjectAfterAggregate(druidRel, postProject);
if (newDruidRel != null) {
call.transformTo(newDruidRel);
}
}
}
public static class DruidFilterAfterAggregationRule extends RelOptRule
{
private DruidFilterAfterAggregationRule()
{
super(operand(Filter.class, operand(DruidRel.class, none())));
}
@Override
public void onMatch(RelOptRuleCall call)
{
final Filter postFilter = call.rel(0);
final DruidRel druidRel = call.rel(1);
final DruidRel newDruidRel = GroupByRules.applyFilterAfterAggregate(druidRel, postFilter);
if (newDruidRel != null) {
call.transformTo(newDruidRel);
}
}
}
public static class DruidGroupBySortRule extends RelOptRule
{
private DruidGroupBySortRule()
{
super(operand(Sort.class, operand(DruidRel.class, none())));
}
@Override
public void onMatch(RelOptRuleCall call)
{
final Sort sort = call.rel(0);
final DruidRel druidRel = call.rel(1);
final DruidRel newDruidRel = GroupByRules.applySort(druidRel, sort);
if (newDruidRel != null) {
call.transformTo(newDruidRel);
}
}
}
private static DruidRel applyAggregate(
final DruidRel druidRel,
final Project project0,
final Aggregate aggregate,
final boolean approximateCountDistinct
)
{
if ((project0 != null && druidRel.getQueryBuilder().getSelectProjection() != null /* can't project twice */)
|| druidRel.getQueryBuilder().getGrouping() != null
|| aggregate.indicator
|| aggregate.getGroupSets().size() != 1) {
return null;
}
final Project project;
if (project0 != null) {
project = project0;
} else if (druidRel.getQueryBuilder().getSelectProjection() != null) {
project = druidRel.getQueryBuilder().getSelectProjection().getProject();
} else {
project = null;
}
final List<DimensionSpec> dimensions = Lists.newArrayList();
final List<Aggregation> aggregations = Lists.newArrayList();
final List<String> rowOrder = Lists.newArrayList();
// Translate groupSet.
final ImmutableBitSet groupSet = aggregate.getGroupSet();
int dimOutputNameCounter = 0;
for (int i : groupSet) {
if (project != null && project.getChildExps().get(i) instanceof RexLiteral) {
// Ignore literals in GROUP BY, so a user can write e.g. "GROUP BY 'dummy'" to group everything into a single
// row. Add dummy rowOrder entry so NULLs come out. This is not strictly correct but it works as long as
// nobody actually expects to see the literal.
rowOrder.add(dimOutputName(dimOutputNameCounter++));
} else {
final DimensionSpec dimensionSpec = toDimensionSpec(
druidRel.getDruidTable(),
Expressions.toRowExtraction(
DruidTables.rowOrder(druidRel.getDruidTable()),
Expressions.fromFieldAccess(druidRel.getDruidTable(), project, i)
),
dimOutputName(dimOutputNameCounter++)
);
if (dimensionSpec == null) {
return null;
}
dimensions.add(dimensionSpec);
rowOrder.add(dimensionSpec.getOutputName());
}
}
// Translate aggregates.
for (int i = 0; i < aggregate.getAggCallList().size(); i++) {
final AggregateCall aggCall = aggregate.getAggCallList().get(i);
final Aggregation aggregation = translateAggregateCall(
druidRel,
project,
aggCall,
i,
approximateCountDistinct
);
if (aggregation == null) {
return null;
}
aggregations.add(aggregation);
rowOrder.add(aggregation.getOutputName());
}
return druidRel.withQueryBuilder(
druidRel.getQueryBuilder()
.withGrouping(
Grouping.create(dimensions, aggregations),
aggregate.getRowType(),
rowOrder
)
);
}
private static DruidRel applyProjectAfterAggregate(
final DruidRel druidRel,
final Project postProject
)
{
if (druidRel.getQueryBuilder().getGrouping() == null || druidRel.getQueryBuilder().getLimitSpec() != null) {
return null;
}
final List<String> rowOrder = druidRel.getQueryBuilder().getRowOrder();
final Grouping grouping = druidRel.getQueryBuilder().getGrouping();
final List<Aggregation> newAggregations = Lists.newArrayList(grouping.getAggregations());
final List<PostAggregatorFactory> finalizingPostAggregatorFactories = Lists.newArrayList();
final List<String> newRowOrder = Lists.newArrayList();
// Build list of finalizingPostAggregatorFactories.
final Map<String, Aggregation> aggregationMap = Maps.newHashMap();
for (final Aggregation aggregation : grouping.getAggregations()) {
aggregationMap.put(aggregation.getOutputName(), aggregation);
}
for (final String field : rowOrder) {
final Aggregation aggregation = aggregationMap.get(field);
finalizingPostAggregatorFactories.add(
aggregation == null
? null
: aggregation.getFinalizingPostAggregatorFactory()
);
}
// Walk through the postProject expressions.
for (final RexNode projectExpression : postProject.getChildExps()) {
if (projectExpression.isA(SqlKind.INPUT_REF)) {
final RexInputRef ref = (RexInputRef) projectExpression;
final String fieldName = rowOrder.get(ref.getIndex());
newRowOrder.add(fieldName);
finalizingPostAggregatorFactories.add(null);
} else {
// Attempt to convert to PostAggregator.
final String postAggregatorName = aggOutputName(newAggregations.size());
final PostAggregator postAggregator = Expressions.toPostAggregator(
postAggregatorName,
rowOrder,
finalizingPostAggregatorFactories,
projectExpression
);
if (postAggregator != null) {
newAggregations.add(Aggregation.create(postAggregator));
newRowOrder.add(postAggregator.getName());
finalizingPostAggregatorFactories.add(null);
} else {
return null;
}
}
}
return druidRel.withQueryBuilder(
druidRel.getQueryBuilder()
.withAdjustedGrouping(
Grouping.create(grouping.getDimensions(), newAggregations),
postProject.getRowType(),
newRowOrder
)
);
}
private static DruidRel applyFilterAfterAggregate(
final DruidRel druidRel,
final Filter postFilter
)
{
if (druidRel.getQueryBuilder().getGrouping() == null
|| druidRel.getQueryBuilder().getHaving() != null
|| druidRel.getQueryBuilder().getLimitSpec() != null) {
return null;
}
final DimFilter dimFilter = Expressions.toFilter(
null, // null table; this filter is being applied as a HAVING on result rows
druidRel.getQueryBuilder().getRowOrder(),
postFilter.getCondition()
);
if (dimFilter != null) {
return druidRel.withQueryBuilder(
druidRel.getQueryBuilder()
.withHaving(dimFilter)
);
} else {
return null;
}
}
private static DruidRel applySort(
final DruidRel druidRel,
final Sort sort
)
{
if (druidRel.getQueryBuilder().getGrouping() == null || druidRel.getQueryBuilder().getLimitSpec() != null) {
// Can only sort when grouping and not already sorting.
return null;
}
final Grouping grouping = druidRel.getQueryBuilder().getGrouping();
final DefaultLimitSpec limitSpec = toLimitSpec(druidRel.getQueryBuilder().getRowOrder(), sort);
if (limitSpec == null) {
return null;
}
final List<OrderByColumnSpec> orderBys = limitSpec.getColumns();
final List<DimensionSpec> newDimensions = Lists.newArrayList(grouping.getDimensions());
// Reorder dimensions, maybe, to allow groupBy to consider pushing down sorting (see DefaultLimitSpec).
if (!orderBys.isEmpty()) {
final Map<String, Integer> dimensionOrderByOutputName = Maps.newHashMap();
for (int i = 0; i < newDimensions.size(); i++) {
dimensionOrderByOutputName.put(newDimensions.get(i).getOutputName(), i);
}
for (int i = 0; i < orderBys.size(); i++) {
final OrderByColumnSpec orderBy = orderBys.get(i);
final Integer dimensionOrder = dimensionOrderByOutputName.get(orderBy.getDimension());
if (dimensionOrder != null
&& dimensionOrder != i
&& orderBy.getDirection() == OrderByColumnSpec.Direction.ASCENDING
&& orderBy.getDimensionComparator().equals(StringComparators.LEXICOGRAPHIC)) {
final DimensionSpec tmp = newDimensions.get(i);
newDimensions.set(i, newDimensions.get(dimensionOrder));
newDimensions.set(dimensionOrder, tmp);
dimensionOrderByOutputName.put(newDimensions.get(i).getOutputName(), i);
dimensionOrderByOutputName.put(newDimensions.get(dimensionOrder).getOutputName(), dimensionOrder);
}
}
}
if (!orderBys.isEmpty() || limitSpec.getLimit() < Integer.MAX_VALUE) {
return druidRel.withQueryBuilder(
druidRel.getQueryBuilder()
.withAdjustedGrouping(
Grouping.create(newDimensions, grouping.getAggregations()),
druidRel.getQueryBuilder().getRowType(),
druidRel.getQueryBuilder().getRowOrder()
)
.withLimitSpec(limitSpec)
);
} else {
return druidRel;
}
}
public static DefaultLimitSpec toLimitSpec(
final List<String> rowOrder,
final Sort sort
)
{
final Integer limit = sort.fetch != null ? RexLiteral.intValue(sort.fetch) : null;
final List<OrderByColumnSpec> orderBys = Lists.newArrayListWithCapacity(sort.getChildExps().size());
if (sort.offset != null) {
// LimitSpecs don't accept offsets.
return null;
}
// Extract orderBy column specs.
for (int sortKey = 0; sortKey < sort.getChildExps().size(); sortKey++) {
final RexNode sortExpression = sort.getChildExps().get(sortKey);
final RelFieldCollation collation = sort.getCollation().getFieldCollations().get(sortKey);
final OrderByColumnSpec.Direction direction;
final StringComparator comparator;
if (collation.getDirection() == RelFieldCollation.Direction.ASCENDING) {
direction = OrderByColumnSpec.Direction.ASCENDING;
} else if (collation.getDirection() == RelFieldCollation.Direction.DESCENDING) {
direction = OrderByColumnSpec.Direction.DESCENDING;
} else {
throw new ISE("WTF?! Don't know what to do with direction[%s]", collation.getDirection());
}
if (SqlTypeName.NUMERIC_TYPES.contains(sortExpression.getType().getSqlTypeName())
|| SqlTypeName.DATETIME_TYPES.contains(sortExpression.getType().getSqlTypeName())) {
comparator = StringComparators.NUMERIC;
} else {
comparator = StringComparators.LEXICOGRAPHIC;
}
if (sortExpression.isA(SqlKind.INPUT_REF)) {
final RexInputRef ref = (RexInputRef) sortExpression;
final String fieldName = rowOrder.get(ref.getIndex());
orderBys.add(new OrderByColumnSpec(fieldName, direction, comparator));
} else {
// We don't support sorting by anything other than refs which actually appear in the query result.
return null;
}
}
return new DefaultLimitSpec(orderBys, limit);
}
private static DimensionSpec toDimensionSpec(
final DruidTable druidTable,
final RowExtraction rex,
final String name
)
{
if (rex == null) {
return null;
}
final int columnNumber = druidTable.getColumnNumber(rex.getColumn());
if (columnNumber < 0) {
return null;
}
final ValueType columnType = druidTable.getColumnType(columnNumber);
if (columnType == ValueType.STRING ||
(rex.getColumn().equals(Column.TIME_COLUMN_NAME) && rex.getExtractionFn() != null)) {
return rex.getExtractionFn() == null
? new DefaultDimensionSpec(rex.getColumn(), name)
: new ExtractionDimensionSpec(rex.getColumn(), name, rex.getExtractionFn());
} else {
// Can't create dimensionSpecs for non-string, non-time.
return null;
}
}
/**
* Translate an AggregateCall to Druid equivalents.
*
* @return translated aggregation, or null if translation failed.
*/
private static Aggregation translateAggregateCall(
final DruidRel druidRel,
final Project project,
final AggregateCall call,
final int aggNumber,
final boolean approximateCountDistinct
)
{
final List<DimFilter> filters = Lists.newArrayList();
final List<String> rowOrder = DruidTables.rowOrder(druidRel.getDruidTable());
final String name = aggOutputName(aggNumber);
final SqlKind kind = call.getAggregation().getKind();
final SqlTypeName outputType = call.getType().getSqlTypeName();
final Aggregation retVal;
if (call.filterArg >= 0) {
// AGG(xxx) FILTER(WHERE yyy)
if (project == null) {
// We need some kind of projection to support filtered aggregations.
return null;
}
final RexNode expression = project.getChildExps().get(call.filterArg);
final DimFilter filter = Expressions.toFilter(druidRel.getDruidTable(), rowOrder, expression);
if (filter == null) {
return null;
}
filters.add(filter);
}
if (call.getAggregation().getKind() == SqlKind.COUNT && call.getArgList().isEmpty()) {
// COUNT(*)
retVal = Aggregation.create(new CountAggregatorFactory(name));
} else if (call.getAggregation().getKind() == SqlKind.COUNT && call.isDistinct() && approximateCountDistinct) {
// COUNT(DISTINCT x)
final DimensionSpec dimensionSpec = toDimensionSpec(
druidRel.getDruidTable(),
Expressions.toRowExtraction(
rowOrder,
Expressions.fromFieldAccess(
druidRel.getDruidTable(),
project,
Iterables.getOnlyElement(call.getArgList())
)
),
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(*)
boolean forceCount = false;
final FieldOrExpression input;
final int inputField = Iterables.getOnlyElement(call.getArgList());
final RexNode rexNode = Expressions.fromFieldAccess(druidRel.getDruidTable(), project, inputField);
final FieldOrExpression foe = FieldOrExpression.fromRexNode(rowOrder, rexNode);
if (foe != null) {
input = foe;
} else if (rexNode.getKind() == SqlKind.CASE && ((RexCall) rexNode).getOperands().size() == 3) {
// Possibly a CASE-style filtered aggregation. Styles supported:
// A: SUM(CASE WHEN x = 'foo' THEN cnt END) => operands (x = 'foo', cnt, null)
// B: SUM(CASE WHEN x = 'foo' THEN 1 ELSE 0 END) => operands (x = 'foo', 1, 0)
// C: COUNT(CASE WHEN x = 'foo' THEN 'dummy' END) => operands (x = 'foo', 'dummy', null)
// If the null and non-null args are switched, "flip" is set, which negates the filter.
final RexCall caseCall = (RexCall) rexNode;
final boolean flip = RexLiteral.isNullLiteral(caseCall.getOperands().get(1))
&& !RexLiteral.isNullLiteral(caseCall.getOperands().get(2));
final RexNode arg1 = caseCall.getOperands().get(flip ? 2 : 1);
final RexNode arg2 = caseCall.getOperands().get(flip ? 1 : 2);
// Operand 1: Filter
final DimFilter filter = Expressions.toFilter(
druidRel.getDruidTable(),
rowOrder,
caseCall.getOperands().get(0)
);
if (filter == null) {
return null;
} else {
filters.add(flip ? new NotDimFilter(filter) : filter);
}
if (call.getAggregation().getKind() == SqlKind.COUNT
&& arg1 instanceof RexLiteral
&& !RexLiteral.isNullLiteral(arg1)
&& RexLiteral.isNullLiteral(arg2)) {
// Case C
forceCount = true;
input = null;
} else if (call.getAggregation().getKind() == SqlKind.SUM
&& arg1 instanceof RexLiteral
&& ((Number) RexLiteral.value(arg1)).intValue() == 1
&& arg2 instanceof RexLiteral
&& ((Number) RexLiteral.value(arg2)).intValue() == 0) {
// Case B
forceCount = true;
input = null;
} else if (RexLiteral.isNullLiteral(arg2)) {
// Maybe case A
input = FieldOrExpression.fromRexNode(rowOrder, arg1);
if (input == null) {
return null;
}
} else {
// Can't translate CASE into a filter.
return null;
}
} else {
// Can't translate aggregator expression.
return null;
}
if (!forceCount) {
Preconditions.checkNotNull(input, "WTF?! input was null for non-COUNT aggregation");
}
if (forceCount || kind == SqlKind.COUNT) {
// COUNT(x)
retVal = Aggregation.create(new CountAggregatorFactory(name));
} else {
// All aggregators other than COUNT expect a single argument with no extractionFn.
final String fieldName = input.getFieldName();
final String expression = input.getExpression();
final boolean isLong = SqlTypeName.INT_TYPES.contains(outputType)
|| SqlTypeName.DATETIME_TYPES.contains(outputType);
if (kind == SqlKind.SUM || kind == SqlKind.SUM0) {
retVal = isLong
? Aggregation.create(new LongSumAggregatorFactory(name, fieldName, expression))
: Aggregation.create(new DoubleSumAggregatorFactory(name, fieldName, expression));
} else if (kind == SqlKind.MIN) {
retVal = isLong
? Aggregation.create(new LongMinAggregatorFactory(name, fieldName, expression))
: Aggregation.create(new DoubleMinAggregatorFactory(name, fieldName, expression));
} else if (kind == SqlKind.MAX) {
retVal = isLong
? Aggregation.create(new LongMaxAggregatorFactory(name, fieldName, expression))
: Aggregation.create(new DoubleMaxAggregatorFactory(name, fieldName, expression));
} else if (kind == SqlKind.AVG) {
final String sumName = aggInternalName(aggNumber, "sum");
final String countName = aggInternalName(aggNumber, "count");
final AggregatorFactory sum = isLong
? new LongSumAggregatorFactory(sumName, fieldName, expression)
: new DoubleSumAggregatorFactory(sumName, fieldName, expression);
final AggregatorFactory count = new CountAggregatorFactory(countName);
retVal = Aggregation.create(
ImmutableList.of(sum, count),
new ArithmeticPostAggregator(
name,
"quotient",
ImmutableList.<PostAggregator>of(
new FieldAccessPostAggregator(null, sumName),
new FieldAccessPostAggregator(null, countName)
)
)
);
} else {
retVal = null;
}
}
} else {
retVal = null;
}
final DimFilter filter = filters.isEmpty()
? null
: Filtration.create(new AndDimFilter(filters))
.optimizeFilterOnly(druidRel.getDruidTable())
.getDimFilter();
return retVal != null ? retVal.filter(filter) : null;
}
public static String dimOutputName(final int dimNumber)
{
return "d" + dimNumber;
}
private static String aggOutputName(final int aggNumber)
{
return "a" + aggNumber;
}
private static String aggInternalName(final int aggNumber, final String key)
{
return "A" + aggNumber + ":" + key;
}
}

View File

@ -0,0 +1,210 @@
/*
* 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.table;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSortedMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import io.druid.java.util.common.ISE;
import io.druid.query.DataSource;
import io.druid.query.QuerySegmentWalker;
import io.druid.segment.column.Column;
import io.druid.segment.column.ValueType;
import io.druid.sql.calcite.planner.PlannerConfig;
import io.druid.sql.calcite.rel.DruidQueryRel;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptTable;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.schema.Schema;
import org.apache.calcite.schema.Statistic;
import org.apache.calcite.schema.Statistics;
import org.apache.calcite.schema.TranslatableTable;
import org.apache.calcite.sql.type.SqlTypeName;
import java.util.List;
import java.util.Map;
public class DruidTable implements TranslatableTable
{
private final QuerySegmentWalker walker;
private final DataSource dataSource;
private final PlannerConfig config;
private final Map<String, Integer> columnNumbers;
private final List<ValueType> columnTypes;
private final List<String> columnNames;
public DruidTable(
final QuerySegmentWalker walker,
final DataSource dataSource,
final PlannerConfig config,
final Map<String, ValueType> columns
)
{
this.walker = Preconditions.checkNotNull(walker, "walker");
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
this.config = Preconditions.checkNotNull(config, "config");
this.columnNumbers = Maps.newLinkedHashMap();
this.columnTypes = Lists.newArrayList();
this.columnNames = Lists.newArrayList();
int i = 0;
for (Map.Entry<String, ValueType> entry : ImmutableSortedMap.copyOf(columns).entrySet()) {
columnNumbers.put(entry.getKey(), i++);
columnTypes.add(entry.getValue());
columnNames.add(entry.getKey());
}
}
public QuerySegmentWalker getQuerySegmentWalker()
{
return walker;
}
public DataSource getDataSource()
{
return dataSource;
}
public PlannerConfig getPlannerConfig()
{
return config;
}
public int getColumnCount()
{
return columnNames.size();
}
public int getColumnNumber(final String name)
{
final Integer number = columnNumbers.get(name);
return number != null ? number : -1;
}
public String getColumnName(final int n)
{
return columnNames.get(n);
}
public ValueType getColumnType(final int n)
{
return columnTypes.get(n);
}
@Override
public Schema.TableType getJdbcTableType()
{
return Schema.TableType.TABLE;
}
@Override
public Statistic getStatistic()
{
return Statistics.UNKNOWN;
}
@Override
public RelDataType getRowType(final RelDataTypeFactory typeFactory)
{
final RelDataTypeFactory.FieldInfoBuilder builder = typeFactory.builder();
for (Map.Entry<String, Integer> entry : columnNumbers.entrySet()) {
final RelDataType sqlTypeName;
if (entry.getKey().equals(Column.TIME_COLUMN_NAME)) {
sqlTypeName = typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
} else {
final ValueType valueType = columnTypes.get(entry.getValue());
switch (valueType) {
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);
break;
case LONG:
sqlTypeName = typeFactory.createSqlType(SqlTypeName.BIGINT);
break;
case FLOAT:
sqlTypeName = typeFactory.createSqlType(SqlTypeName.FLOAT);
break;
default:
throw new ISE("WTF?! valueType[%s] not translatable?", valueType);
}
}
builder.add(entry.getKey(), sqlTypeName);
}
return builder.build();
}
@Override
public RelNode toRel(final RelOptTable.ToRelContext context, final RelOptTable table)
{
final RelOptCluster cluster = context.getCluster();
return DruidQueryRel.fullScan(
cluster,
cluster.traitSet(),
table,
this
);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
DruidTable that = (DruidTable) o;
if (dataSource != null ? !dataSource.equals(that.dataSource) : that.dataSource != null) {
return false;
}
if (columnNumbers != null ? !columnNumbers.equals(that.columnNumbers) : that.columnNumbers != null) {
return false;
}
return columnTypes != null ? columnTypes.equals(that.columnTypes) : that.columnTypes == null;
}
@Override
public int hashCode()
{
int result = dataSource != null ? dataSource.hashCode() : 0;
result = 31 * result + (columnNumbers != null ? columnNumbers.hashCode() : 0);
result = 31 * result + (columnTypes != null ? columnTypes.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "DruidTable{" +
"dataSource=" + dataSource +
", columnNumbers=" + columnNumbers +
", columnTypes=" + columnTypes +
'}';
}
}

View File

@ -0,0 +1,76 @@
/*
* 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.table;
import com.google.common.collect.Lists;
import io.druid.query.ordering.StringComparator;
import io.druid.query.ordering.StringComparators;
import io.druid.segment.column.ValueType;
import io.druid.sql.calcite.expression.RowExtraction;
import java.util.List;
public class DruidTables
{
private DruidTables()
{
// No instantiation.
}
/**
* Returns the "natural" rowOrder for a Druid table. This is the order that a scan without projection would return.
*
* @param druidTable druid table
*
* @return natural row order
*/
public static List<String> rowOrder(
final DruidTable druidTable
)
{
final List<String> rowOrder = Lists.newArrayListWithCapacity(druidTable.getColumnCount());
for (int i = 0; i < druidTable.getColumnCount(); i++) {
rowOrder.add(druidTable.getColumnName(i));
}
return rowOrder;
}
/**
* Return the "natural" {@link StringComparator} for an extraction from a Druid table. This will be a lexicographic
* comparator for String types and a numeric comparator for Number types.
*
* @param druidTable underlying Druid table
* @param rowExtraction extraction from the table
*
* @return natural comparator
*/
public static StringComparator naturalStringComparator(
final DruidTable druidTable,
final RowExtraction rowExtraction
)
{
if (rowExtraction.getExtractionFn() != null
|| druidTable.getColumnType(druidTable.getColumnNumber(rowExtraction.getColumn())) == ValueType.STRING) {
return StringComparators.LEXICOGRAPHIC;
} else {
return StringComparators.NUMERIC;
}
}
}

View File

@ -0,0 +1,88 @@
/*
* 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.common.base.Preconditions;
import com.google.inject.Binder;
import com.google.inject.Inject;
import com.google.inject.Module;
import com.google.inject.Provides;
import io.druid.guice.Jerseys;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.LazySingleton;
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.DruidAvaticaHandler;
import io.druid.sql.avatica.ServerConfig;
import io.druid.sql.calcite.DruidSchema;
import io.druid.sql.calcite.planner.Calcites;
import io.druid.sql.calcite.planner.PlannerConfig;
import io.druid.sql.http.SqlResource;
import org.apache.calcite.jdbc.CalciteConnection;
import java.sql.SQLException;
import java.util.Properties;
public class SqlModule implements Module
{
private static final String PROPERTY_SQL_ENABLE = "druid.sql.enable";
@Inject
private Properties props;
public SqlModule()
{
}
@Override
public void configure(Binder binder)
{
if (isEnabled()) {
JsonConfigProvider.bind(binder, "druid.sql.server", ServerConfig.class);
JsonConfigProvider.bind(binder, "druid.sql.planner", PlannerConfig.class);
Jerseys.addResource(binder, SqlResource.class);
binder.bind(AvaticaMonitor.class).in(LazySingleton.class);
JettyBindings.addHandler(binder, DruidAvaticaHandler.class);
MetricsModule.register(binder, AvaticaMonitor.class);
LifecycleModule.register(binder, DruidSchema.class);
}
}
@Provides
public CalciteConnection createCalciteConnection(
final DruidSchema druidSchema,
final PlannerConfig plannerConfig
) throws SQLException
{
if (isEnabled()) {
return Calcites.jdbc(druidSchema, plannerConfig);
} else {
throw new IllegalStateException("Cannot provide CalciteConnection when SQL is disabled.");
}
}
private boolean isEnabled()
{
Preconditions.checkNotNull(props, "props");
return Boolean.valueOf(props.getProperty(PROPERTY_SQL_ENABLE, "false"));
}
}

View File

@ -0,0 +1,72 @@
/*
* 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.http;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
public class SqlQuery
{
private final String query;
@JsonCreator
public SqlQuery(
@JsonProperty("query") final String query
)
{
this.query = Preconditions.checkNotNull(query, "query");
}
@JsonProperty
public String getQuery()
{
return query;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SqlQuery sqlQuery = (SqlQuery) o;
return query != null ? query.equals(sqlQuery.query) : sqlQuery.query == null;
}
@Override
public int hashCode()
{
return query != null ? query.hashCode() : 0;
}
@Override
public String toString()
{
return "SqlQuery{" +
"query='" + query + '\'' +
'}';
}
}

View File

@ -0,0 +1,152 @@
/*
* 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.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.logger.Logger;
import io.druid.query.QueryInterruptedException;
import org.apache.calcite.jdbc.CalciteConnection;
import org.joda.time.DateTime;
import javax.ws.rs.Consumes;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
import javax.ws.rs.Produces;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.MediaType;
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;
@Path("/druid/v2/sql/")
public class SqlResource
{
private static final Logger log = new Logger(SqlResource.class);
private final ObjectMapper jsonMapper;
private final Connection connection;
@Inject
public SqlResource(
@Json ObjectMapper jsonMapper,
CalciteConnection connection
)
{
this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper");
this.connection = Preconditions.checkNotNull(connection, "connection");
}
@POST
@Produces(MediaType.APPLICATION_JSON)
@Consumes(MediaType.APPLICATION_JSON)
public Response doPost(final SqlQuery sqlQuery) throws SQLException, IOException
{
// 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();
// 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.getColumnName(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();
}
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();
} else {
maybeUnwrapped = e;
}
return Response.serverError()
.type(MediaType.APPLICATION_JSON_TYPE)
.entity(jsonMapper.writeValueAsBytes(QueryInterruptedException.wrapIfNeeded(maybeUnwrapped)))
.build();
}
}
}

View File

@ -0,0 +1,268 @@
/*
* 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.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
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.PlannerConfig;
import io.druid.sql.calcite.util.CalciteTests;
import org.apache.calcite.jdbc.CalciteConnection;
import org.eclipse.jetty.server.Server;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import java.net.InetSocketAddress;
import java.sql.Connection;
import java.sql.DatabaseMetaData;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.ResultSetMetaData;
import java.sql.SQLException;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
public class DruidAvaticaHandlerTest
{
@Rule
public ExpectedException expectedException = ExpectedException.none();
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private CalciteConnection serverConnection;
private Server server;
private Connection client;
@Before
public void setUp() throws Exception
{
final PlannerConfig plannerConfig = new PlannerConfig();
serverConnection = Calcites.jdbc(
CalciteTests.createMockSchema(
CalciteTests.createWalker(temporaryFolder.newFolder()),
plannerConfig
),
plannerConfig
);
final ServerConfig serverConfig = new ServerConfig()
{
@Override
public boolean isEnableAvatica()
{
return true;
}
};
final DruidAvaticaHandler handler = new DruidAvaticaHandler(
serverConnection,
new DruidNode("dummy", "dummy", 1),
new AvaticaMonitor(),
serverConfig
);
final int port = new Random().nextInt(9999) + 10000;
server = new Server(new InetSocketAddress("127.0.0.1", port));
server.setHandler(handler);
server.start();
final String url = String.format(
"jdbc:avatica:remote:url=http://127.0.0.1:%d%s",
port,
DruidAvaticaHandler.AVATICA_PATH
);
client = DriverManager.getConnection(url);
}
@After
public void tearDown() throws Exception
{
client.close();
server.stop();
serverConnection.close();
client = null;
server = null;
serverConnection = null;
}
@Test
public void testSelectCount() throws Exception
{
final ResultSet resultSet = client.createStatement().executeQuery("SELECT COUNT(*) AS cnt FROM druid.foo");
final List<Map<String, Object>> rows = getRows(resultSet);
Assert.assertEquals(
ImmutableList.of(
ImmutableMap.of("cnt", 6L)
),
rows
);
}
@Test
public void testExplainSelectCount() throws Exception
{
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"
)
),
rows
);
}
@Test
public void testDatabaseMetaDataSchemas() throws Exception
{
final DatabaseMetaData metaData = client.getMetaData();
Assert.assertEquals(
ImmutableList.of(
ROW(Pair.of("TABLE_CATALOG", null), Pair.of("TABLE_SCHEM", "druid"))
),
getRows(metaData.getSchemas(null, "druid"))
);
}
@Test
public void testDatabaseMetaDataTables() throws Exception
{
final DatabaseMetaData metaData = client.getMetaData();
Assert.assertEquals(
ImmutableList.of(
ROW(
Pair.of("TABLE_CAT", null),
Pair.of("TABLE_NAME", "foo"),
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_TYPE", "TABLE")
)
),
getRows(
metaData.getTables(null, "druid", "%", null),
ImmutableSet.of("TABLE_CAT", "TABLE_NAME", "TABLE_SCHEM", "TABLE_TYPE")
)
);
}
@Test
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("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("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("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("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("IS_NULLABLE", "NO")
)
),
getRows(
metaData.getColumns(null, "druid", "foo", "%"),
ImmutableSet.of("IS_NULLABLE", "TABLE_NAME", "TABLE_SCHEM", "COLUMN_NAME", "DATA_TYPE", "TYPE_NAME")
)
);
}
private static List<Map<String, Object>> getRows(final ResultSet resultSet) throws SQLException
{
return getRows(resultSet, null);
}
private static List<Map<String, Object>> getRows(final ResultSet resultSet, final Set<String> returnKeys)
throws SQLException
{
try {
final ResultSetMetaData metaData = resultSet.getMetaData();
final List<Map<String, Object>> rows = Lists.newArrayList();
while (resultSet.next()) {
final Map<String, Object> row = Maps.newHashMap();
for (int i = 0; i < metaData.getColumnCount(); i++) {
if (returnKeys == null || returnKeys.contains(metaData.getColumnName(i + 1))) {
row.put(metaData.getColumnName(i + 1), resultSet.getObject(i + 1));
}
}
rows.add(row);
}
return rows;
}
finally {
resultSet.close();
}
}
private static Map<String, Object> ROW(final Pair<String, ?>... entries)
{
final Map<String, Object> m = Maps.newHashMap();
for (Pair<String, ?> entry : entries) {
m.put(entry.lhs, entry.rhs);
}
return m;
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,133 @@
/*
* 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;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import io.druid.sql.calcite.planner.PlannerConfig;
import io.druid.sql.calcite.table.DruidTable;
import io.druid.sql.calcite.util.CalciteTests;
import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import io.druid.sql.calcite.util.TestServerInventoryView;
import org.apache.calcite.jdbc.CalciteConnection;
import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeField;
import org.apache.calcite.schema.Table;
import org.apache.calcite.sql.type.SqlTypeName;
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.sql.Connection;
import java.sql.DriverManager;
import java.util.List;
import java.util.Map;
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();
private SpecificSegmentsQuerySegmentWalker walker = null;
private DruidSchema schema = null;
private Connection connection = null;
@Before
public void setUp() throws Exception
{
walker = CalciteTests.createWalker(temporaryFolder.newFolder());
Properties props = new Properties();
props.setProperty("caseSensitive", "true");
props.setProperty("unquotedCasing", "UNCHANGED");
connection = DriverManager.getConnection("jdbc:calcite:", props);
CalciteConnection calciteConnection = connection.unwrap(CalciteConnection.class);
schema = new DruidSchema(
walker,
new TestServerInventoryView(walker.getSegments()),
PLANNER_CONFIG_DEFAULT
);
calciteConnection.getRootSchema().add("s", schema);
schema.start();
schema.awaitInitialization();
}
@After
public void tearDown() throws Exception
{
schema.stop();
walker.close();
connection.close();
}
@Test
public void testGetTableMap()
{
Assert.assertEquals(ImmutableSet.of("foo"), schema.getTableNames());
final Map<String, Table> tableMap = schema.getTableMap();
Assert.assertEquals(1, tableMap.size());
Assert.assertEquals("foo", Iterables.getOnlyElement(tableMap.keySet()));
final DruidTable druidTable = (DruidTable) Iterables.getOnlyElement(tableMap.values());
final RelDataType rowType = druidTable.getRowType(new JavaTypeFactoryImpl());
final List<RelDataTypeField> fields = rowType.getFieldList();
Assert.assertEquals(5, fields.size());
Assert.assertEquals("__time", fields.get(0).getName());
Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName());
Assert.assertEquals("cnt", fields.get(1).getName());
Assert.assertEquals(SqlTypeName.BIGINT, fields.get(1).getType().getSqlTypeName());
Assert.assertEquals("dim1", fields.get(2).getName());
Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(2).getType().getSqlTypeName());
Assert.assertEquals("dim2", fields.get(3).getName());
Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(3).getType().getSqlTypeName());
Assert.assertEquals("m1", fields.get(4).getName());
Assert.assertEquals(SqlTypeName.FLOAT, fields.get(4).getType().getSqlTypeName());
}
}

View File

@ -0,0 +1,62 @@
/*
* 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.filtration;
import com.google.common.collect.ImmutableList;
import io.druid.query.filter.IntervalDimFilter;
import io.druid.query.filter.NotDimFilter;
import io.druid.segment.column.Column;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
public class FiltrationTest
{
@Test
public void testNotIntervals()
{
final Filtration filtration = Filtration.create(
new NotDimFilter(
new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
ImmutableList.of(new Interval("2000/2001"), new Interval("2002/2003")),
null
)
),
null
).optimize(null);
Assert.assertEquals(
ImmutableList.of(Filtration.eternity()),
filtration.getIntervals()
);
Assert.assertEquals(
new NotDimFilter(
new IntervalDimFilter(
Column.TIME_COLUMN_NAME,
ImmutableList.of(new Interval("2000/2001"), new Interval("2002/2003")),
null
)
),
filtration.getDimFilter()
);
}
}

View File

@ -0,0 +1,161 @@
/*
* 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.http;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
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.PlannerConfig;
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.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.StreamingOutput;
import java.io.ByteArrayOutputStream;
import java.util.List;
import java.util.Map;
public class SqlResourceTest
{
private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper();
@Rule
public ExpectedException expectedException = ExpectedException.none();
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private CalciteConnection connection;
private SqlResource resource;
@Before
public void setUp() throws Exception
{
final PlannerConfig plannerConfig = new PlannerConfig();
connection = Calcites.jdbc(
CalciteTests.createMockSchema(
CalciteTests.createWalker(temporaryFolder.newFolder()),
plannerConfig
),
plannerConfig
);
resource = new SqlResource(JSON_MAPPER, connection);
}
@After
public void tearDown() throws Exception
{
connection.close();
connection = null;
}
@Test
public void testCountStar() throws Exception
{
final List<Map<String, Object>> rows = doPost(
new SqlQuery("SELECT COUNT(*) AS cnt FROM druid.foo")
);
Assert.assertEquals(
ImmutableList.of(
ImmutableMap.of("cnt", 6)
),
rows
);
}
@Test
public void testTimestampsInResponse() throws Exception
{
final List<Map<String, Object>> rows = doPost(
new SqlQuery("SELECT __time FROM druid.foo LIMIT 1")
);
Assert.assertEquals(
ImmutableList.of(
ImmutableMap.of("__time", "2000-01-01T00:00:00.000Z")
),
rows
);
}
@Test
public void testExplainCountStar() throws Exception
{
final List<Map<String, Object>> rows = doPost(
new SqlQuery("EXPLAIN PLAN FOR SELECT COUNT(*) AS cnt FROM druid.foo")
);
Assert.assertEquals(
ImmutableList.of(
ImmutableMap.<String, Object>of(
"PLAN",
"EnumerableInterpreter\n"
+ " DruidQueryRel(dataSource=[foo], dimensions=[[]], aggregations=[[Aggregation{aggregatorFactories=[CountAggregatorFactory{name='a0'}], postAggregator=null, finalizingPostAggregatorFactory=null}]])\n"
)
),
rows
);
}
@Test
public void testCannotPlan() throws Exception
{
expectedException.expect(QueryInterruptedException.class);
expectedException.expectMessage("Column 'dim3' not found in any table");
doPost(
new SqlQuery("SELECT dim3 FROM druid.foo")
);
Assert.fail();
}
private List<Map<String, Object>> doPost(final SqlQuery query) throws Exception
{
final Response response = resource.doPost(query);
if (response.getStatus() == 200) {
final StreamingOutput output = (StreamingOutput) response.getEntity();
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
output.write(baos);
return JSON_MAPPER.readValue(
baos.toByteArray(),
new TypeReference<List<Map<String, Object>>>()
{
}
);
} else {
throw JSON_MAPPER.readValue((byte[]) response.getEntity(), QueryInterruptedException.class);
}
}
}

View File

@ -0,0 +1,247 @@
/*
* 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.google.common.base.Supplier;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.druid.collections.StupidPool;
import io.druid.data.input.InputRow;
import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.InputRowParser;
import io.druid.data.input.impl.MapInputRowParser;
import io.druid.data.input.impl.TimeAndDimsParseSpec;
import io.druid.data.input.impl.TimestampSpec;
import io.druid.query.DefaultQueryRunnerFactoryConglomerate;
import io.druid.query.Query;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.QuerySegmentWalker;
import io.druid.query.TableDataSource;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import io.druid.query.groupby.GroupByQuery;
import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryRunnerTest;
import io.druid.query.groupby.strategy.GroupByStrategySelector;
import io.druid.query.metadata.SegmentMetadataQueryConfig;
import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest;
import io.druid.query.metadata.SegmentMetadataQueryRunnerFactory;
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
import io.druid.query.select.SelectQuery;
import io.druid.query.select.SelectQueryEngine;
import io.druid.query.select.SelectQueryQueryToolChest;
import io.druid.query.select.SelectQueryRunnerFactory;
import io.druid.query.timeseries.TimeseriesQuery;
import io.druid.query.timeseries.TimeseriesQueryEngine;
import io.druid.query.timeseries.TimeseriesQueryQueryToolChest;
import io.druid.query.timeseries.TimeseriesQueryRunnerFactory;
import io.druid.query.topn.TopNQuery;
import io.druid.query.topn.TopNQueryConfig;
import io.druid.query.topn.TopNQueryQueryToolChest;
import io.druid.query.topn.TopNQueryRunnerFactory;
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.incremental.IncrementalIndexSchema;
import io.druid.sql.calcite.planner.PlannerConfig;
import io.druid.sql.calcite.table.DruidTable;
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 java.io.File;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
/**
* Utility functions for Calcite tests.
*/
public class CalciteTests
{
public static final String DATASOURCE = "foo";
private static final String TIMESTAMP_COLUMN = "t";
private static final InputRowParser<Map<String, Object>> PARSER = new MapInputRowParser(
new TimeAndDimsParseSpec(
new TimestampSpec(TIMESTAMP_COLUMN, "iso", null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2")),
null,
null
)
)
);
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 Map<String, ValueType> COLUMN_TYPES = ImmutableMap.of(
"__time", ValueType.LONG,
"cnt", ValueType.LONG,
"dim1", ValueType.STRING,
"dim2", ValueType.STRING,
"m1", ValueType.FLOAT
);
private CalciteTests()
{
// No instantiation.
}
public static SpecificSegmentsQuerySegmentWalker createWalker(final File tmpDir)
{
return createWalker(tmpDir, ROWS);
}
public static SpecificSegmentsQuerySegmentWalker createWalker(final File tmpDir, final List<InputRow> rows)
{
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 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<>(
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(
new GroupByQueryConfig()
{
@Override
public String getDefaultStrategy()
{
return GroupByStrategySelector.STRATEGY_V2;
}
}
)
)
.build()
);
return new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
DataSegment.builder()
.dataSource(DATASOURCE)
.interval(index.getDataInterval())
.version("1")
.shardSpec(new LinearShardSpec(0))
.build(),
index
);
}
public static DruidTable createDruidTable(final QuerySegmentWalker walker, final PlannerConfig plannerConfig)
{
return new DruidTable(walker, new TableDataSource(DATASOURCE), plannerConfig, COLUMN_TYPES);
}
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);
return new AbstractSchema()
{
@Override
protected Map<String, Table> getTableMap()
{
return tableMap;
}
};
}
private static InputRow ROW(final ImmutableMap<String, ?> map)
{
return PARSER.parse((Map<String, Object>) map);
}
}

View File

@ -0,0 +1,217 @@
/*
* 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.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.io.Closeables;
import com.google.common.util.concurrent.MoreExecutors;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.guava.FunctionalIterable;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.NoopQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.query.QuerySegmentWalker;
import io.druid.query.QueryToolChest;
import io.druid.query.SegmentDescriptor;
import io.druid.query.TableDataSource;
import io.druid.query.spec.SpecificSegmentQueryRunner;
import io.druid.query.spec.SpecificSegmentSpec;
import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexSegment;
import io.druid.segment.Segment;
import io.druid.timeline.DataSegment;
import io.druid.timeline.TimelineObjectHolder;
import io.druid.timeline.VersionedIntervalTimeline;
import io.druid.timeline.partition.PartitionChunk;
import io.druid.timeline.partition.PartitionHolder;
import org.joda.time.Interval;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class SpecificSegmentsQuerySegmentWalker implements QuerySegmentWalker, Closeable
{
private final QueryRunnerFactoryConglomerate conglomerate;
private final Map<String, VersionedIntervalTimeline<String, Segment>> timelines = Maps.newHashMap();
private final List<Closeable> closeables = Lists.newArrayList();
private final List<DataSegment> segments = Lists.newArrayList();
public SpecificSegmentsQuerySegmentWalker(QueryRunnerFactoryConglomerate conglomerate)
{
this.conglomerate = conglomerate;
}
public SpecificSegmentsQuerySegmentWalker add(
final DataSegment descriptor,
final QueryableIndex index
)
{
final Segment segment = new QueryableIndexSegment(descriptor.getIdentifier(), index);
if (!timelines.containsKey(descriptor.getDataSource())) {
timelines.put(descriptor.getDataSource(), new VersionedIntervalTimeline<String, Segment>(Ordering.natural()));
}
final VersionedIntervalTimeline<String, Segment> timeline = timelines.get(descriptor.getDataSource());
timeline.add(descriptor.getInterval(), descriptor.getVersion(), descriptor.getShardSpec().createChunk(segment));
segments.add(descriptor);
return this;
}
public List<DataSegment> getSegments()
{
return segments;
}
@Override
public <T> QueryRunner<T> getQueryRunnerForIntervals(
final Query<T> query,
final Iterable<Interval> intervals
)
{
final VersionedIntervalTimeline<String, Segment> timeline = getTimeline(query);
if (timeline == null) {
return new NoopQueryRunner<>();
}
final Iterable<SegmentDescriptor> specs = FunctionalIterable
.create(intervals)
.transformCat(
new Function<Interval, Iterable<TimelineObjectHolder<String, Segment>>>()
{
@Override
public Iterable<TimelineObjectHolder<String, Segment>> apply(final Interval interval)
{
return timeline.lookup(interval);
}
}
)
.transformCat(
new Function<TimelineObjectHolder<String, Segment>, Iterable<SegmentDescriptor>>()
{
@Override
public Iterable<SegmentDescriptor> apply(final TimelineObjectHolder<String, Segment> holder)
{
return FunctionalIterable
.create(holder.getObject())
.transform(
new Function<PartitionChunk<Segment>, SegmentDescriptor>()
{
@Override
public SegmentDescriptor apply(final PartitionChunk<Segment> chunk)
{
return new SegmentDescriptor(
holder.getInterval(),
holder.getVersion(),
chunk.getChunkNumber()
);
}
}
);
}
}
);
return getQueryRunnerForSegments(query, specs);
}
@Override
public <T> QueryRunner<T> getQueryRunnerForSegments(
final Query<T> query,
final Iterable<SegmentDescriptor> specs
)
{
final VersionedIntervalTimeline<String, Segment> timeline = getTimeline(query);
if (timeline == null) {
return new NoopQueryRunner<>();
}
final QueryRunnerFactory<T, Query<T>> factory = conglomerate.findFactory(query);
if (factory == null) {
throw new ISE("Unknown query type[%s].", query.getClass());
}
final QueryToolChest<T, Query<T>> toolChest = factory.getToolchest();
return new FinalizeResultsQueryRunner<>(
toolChest.mergeResults(
factory.mergeRunners(
MoreExecutors.sameThreadExecutor(),
FunctionalIterable
.create(specs)
.transformCat(
new Function<SegmentDescriptor, Iterable<QueryRunner<T>>>()
{
@Override
public Iterable<QueryRunner<T>> apply(final SegmentDescriptor descriptor)
{
final PartitionHolder<Segment> holder = timeline.findEntry(
descriptor.getInterval(),
descriptor.getVersion()
);
return Iterables.transform(
holder,
new Function<PartitionChunk<Segment>, QueryRunner<T>>()
{
@Override
public QueryRunner<T> apply(PartitionChunk<Segment> chunk)
{
return new SpecificSegmentQueryRunner<T>(
factory.createRunner(chunk.getObject()),
new SpecificSegmentSpec(descriptor)
);
}
}
);
}
}
)
)
),
toolChest
);
}
@Override
public void close() throws IOException
{
for (Closeable closeable : closeables) {
Closeables.close(closeable, true);
}
}
private <T> VersionedIntervalTimeline<String, Segment> getTimeline(Query<T> query)
{
if (query.getDataSource() instanceof TableDataSource) {
return timelines.get(((TableDataSource) query.getDataSource()).getName());
} else {
return null;
}
}
}

View File

@ -0,0 +1,95 @@
/*
* 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.google.common.collect.ImmutableList;
import io.druid.client.DruidServer;
import io.druid.client.ServerView;
import io.druid.client.TimelineServerView;
import io.druid.client.selector.ServerSelector;
import io.druid.query.DataSource;
import io.druid.query.QueryRunner;
import io.druid.server.coordination.DruidServerMetadata;
import io.druid.timeline.DataSegment;
import io.druid.timeline.TimelineLookup;
import java.util.List;
import java.util.concurrent.Executor;
public class TestServerInventoryView implements TimelineServerView
{
private final List<DataSegment> segments;
public TestServerInventoryView(List<DataSegment> segments)
{
this.segments = ImmutableList.copyOf(segments);
}
@Override
public TimelineLookup<String, ServerSelector> getTimeline(DataSource dataSource)
{
throw new UnsupportedOperationException();
}
@Override
public void registerSegmentCallback(Executor exec, final SegmentCallback callback)
{
final DruidServerMetadata dummyServer = new DruidServerMetadata("dummy", "dummy", 0, "dummy", "dummy", 0);
for (final DataSegment segment : segments) {
exec.execute(
new Runnable()
{
@Override
public void run()
{
callback.segmentAdded(dummyServer, segment);
}
}
);
}
exec.execute(
new Runnable()
{
@Override
public void run()
{
callback.segmentViewInitialized();
}
}
);
}
@Override
public <T> QueryRunner<T> getQueryRunner(DruidServer server)
{
throw new UnsupportedOperationException();
}
@Override
public void registerServerCallback(
Executor exec,
ServerView.ServerCallback callback
)
{
// Do nothing
}
}