Skip to content

Commit

Permalink
SQL: Make row extractions extensible and add one for lookups. (apache…
Browse files Browse the repository at this point in the history
…#3991)

This is a reopening of apache#3989, since that PR was merged to master prematurely
and accidentally.
  • Loading branch information
gianm authored and jon-wei committed Mar 14, 2017
1 parent bad250f commit 3216134
Show file tree
Hide file tree
Showing 25 changed files with 550 additions and 307 deletions.
11 changes: 10 additions & 1 deletion docs/content/querying/sql.md
Original file line number Diff line number Diff line change
Expand Up @@ -129,6 +129,16 @@ Druid's SQL language supports a number of time operations, including:
By default, time operations use the UTC time zone. You can change the time zone for time operations by setting the
connection context parameter "sqlTimeZone" to the name of the time zone, like "America/Los_Angeles".

### Query-time lookups

Druid [query-time lookups](lookups.html) can be accessed through the `LOOKUP(expression, lookupName)` function. The
"lookupName" must refer to a lookup you have registered with Druid's lookup framework. For example, the following
query can be used to perform a groupBy on looked-up values:

```sql
SELECT LOOKUP(col, 'my_lookup') AS col_with_lookup FROM data_source GROUP BY LOOKUP(col, 'my_lookup')
```

### Subqueries

Druid's SQL layer supports many types of subqueries, including the ones listed below.
Expand Down Expand Up @@ -231,7 +241,6 @@ language. Some unsupported SQL features include:
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).
- [DataSketches](../development/extensions-core/datasketches-aggregators.html).

## Third-party SQL libraries
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.RowExtraction;
import io.druid.sql.calcite.planner.DruidOperatorTable;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.core.AggregateCall;
Expand Down Expand Up @@ -64,6 +65,7 @@ public SqlAggFunction calciteFunction()
public Aggregation toDruidAggregation(
final String name,
final RowSignature rowSignature,
final DruidOperatorTable operatorTable,
final PlannerContext plannerContext,
final List<Aggregation> existingAggregations,
final Project project,
Expand All @@ -72,6 +74,7 @@ public Aggregation toDruidAggregation(
)
{
final RowExtraction rex = Expressions.toRowExtraction(
operatorTable,
plannerContext,
rowSignature.getRowOrder(),
Expressions.fromFieldAccess(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import io.druid.segment.TestHelper;
import io.druid.segment.incremental.IncrementalIndexSchema;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.SqlExtractionOperator;
import io.druid.sql.calcite.filtration.Filtration;
import io.druid.sql.calcite.planner.Calcites;
import io.druid.sql.calcite.planner.DruidOperatorTable;
Expand Down Expand Up @@ -130,9 +131,8 @@ public void setUp() throws Exception
)
);
final DruidOperatorTable operatorTable = new DruidOperatorTable(
ImmutableSet.<SqlAggregator>of(
new QuantileSqlAggregator()
)
ImmutableSet.<SqlAggregator>of(new QuantileSqlAggregator()),
ImmutableSet.<SqlExtractionOperator>of()
);
plannerFactory = new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig);
}
Expand Down
5 changes: 5 additions & 0 deletions sql/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,11 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-processing</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.RowExtraction;
import io.druid.sql.calcite.planner.Calcites;
import io.druid.sql.calcite.planner.DruidOperatorTable;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.core.AggregateCall;
Expand Down Expand Up @@ -63,6 +64,7 @@ public SqlAggFunction calciteFunction()
public Aggregation toDruidAggregation(
final String name,
final RowSignature rowSignature,
final DruidOperatorTable operatorTable,
final PlannerContext plannerContext,
final List<Aggregation> existingAggregations,
final Project project,
Expand All @@ -76,6 +78,7 @@ public Aggregation toDruidAggregation(
Iterables.getOnlyElement(aggregateCall.getArgList())
);
final RowExtraction rex = Expressions.toRowExtraction(
operatorTable,
plannerContext,
rowSignature.getRowOrder(),
rexNode
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package io.druid.sql.calcite.aggregation;

import io.druid.query.filter.DimFilter;
import io.druid.sql.calcite.planner.DruidOperatorTable;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.core.AggregateCall;
Expand All @@ -46,6 +47,7 @@ public interface SqlAggregator
*
* @param name desired output name of the aggregation
* @param rowSignature signature of the rows being aggregated
* @param operatorTable Operator table that can be used to convert sub-expressions
* @param plannerContext SQL planner context
* @param existingAggregations existing aggregations for this query; useful for re-using aggregations. May be safely
* ignored if you do not want to re-use existing aggregations.
Expand All @@ -59,6 +61,7 @@ public interface SqlAggregator
Aggregation toDruidAggregation(
final String name,
final RowSignature rowSignature,
final DruidOperatorTable operatorTable,
final PlannerContext plannerContext,
final List<Aggregation> existingAggregations,
final Project project,
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -20,37 +20,38 @@
package io.druid.sql.calcite.expression;

import io.druid.query.extraction.StrlenExtractionFn;
import io.druid.sql.calcite.planner.DruidOperatorTable;
import io.druid.sql.calcite.planner.PlannerContext;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;

import java.util.List;

public class CharLengthExpressionConversion extends AbstractExpressionConversion
public class CharacterLengthExtractionOperator implements SqlExtractionOperator
{
private static final CharLengthExpressionConversion INSTANCE = new CharLengthExpressionConversion();

private CharLengthExpressionConversion()
{
super(SqlKind.OTHER_FUNCTION, "CHAR_LENGTH");
}

public static CharLengthExpressionConversion instance()
@Override
public SqlFunction calciteFunction()
{
return INSTANCE;
return SqlStdOperatorTable.CHAR_LENGTH;
}

@Override
public RowExtraction convert(
final ExpressionConverter converter,
final DruidOperatorTable operatorTable,
final PlannerContext plannerContext,
final List<String> rowOrder,
final RexNode expression
)
{
final RexCall call = (RexCall) expression;
final RowExtraction arg = converter.convert(plannerContext, rowOrder, call.getOperands().get(0));
final RowExtraction arg = Expressions.toRowExtraction(
operatorTable,
plannerContext,
rowOrder,
call.getOperands().get(0)
);
if (arg == null) {
return null;
}
Expand Down

This file was deleted.

Loading

0 comments on commit 3216134

Please sign in to comment.