SQL: Upgrade to Calcite 1.14.0, some refactoring of internals. (#4889)

* SQL: Upgrade to Calcite 1.14.0, some refactoring of internals.

This brings benefits:
- Ability to do GROUP BY and ORDER BY with ordinals.
- Ability to support IN filters beyond 19 elements (fixes #4203).

Some refactoring of druid-sql internals:
- Builtin aggregators and operators are implemented as SqlAggregators
  and SqlOperatorConversions rather being special cases. This simplifies
  the Expressions and GroupByRules code, which were becoming complex.
- SqlAggregator implementations are no longer responsible for filtering.

Added new functions:
- Expressions: strpos.
- SQL: TRUNCATE, TRUNC, LENGTH, CHAR_LENGTH, STRLEN, STRPOS, SUBSTR,
  and DATE_TRUNC.

* Add missing @Override annotation.

* Adjustments for forbidden APIs.

* Adjustments for forbidden APIs.

* Disable GROUP BY alias.

* Doc reword.
This commit is contained in:
Gian Merlino 2017-10-10 12:44:05 -07:00 committed by Fangjin Yang
parent 4e1d0f49d8
commit b20e3038b6
62 changed files with 2910 additions and 799 deletions

View File

@ -928,6 +928,28 @@ interface Function
}
}
class StrposFunc implements Function
{
@Override
public String name()
{
return "strpos";
}
@Override
public ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings)
{
if (args.size() != 2) {
throw new IAE("Function[%s] needs 2 arguments", name());
}
final String haystack = Strings.nullToEmpty(args.get(0).eval(bindings).asString());
final String needle = Strings.nullToEmpty(args.get(1).eval(bindings).asString());
return ExprEval.of(haystack.indexOf(needle));
}
}
class SubstringFunc implements Function
{
@Override

View File

@ -85,6 +85,14 @@ public class FunctionTest
assertExpr("strlen(nonexistent)", 0L);
}
@Test
public void testStrpos()
{
assertExpr("strpos(x, 'o')", 1L);
assertExpr("strpos(x, '')", 0L);
assertExpr("strpos(x, 'x')", -1L);
}
@Test
public void testLower()
{

View File

@ -43,6 +43,7 @@ Also, the following built-in functions are supported.
|replace|replace(expr, pattern, replacement) replaces pattern with replacement|
|substring|substring(expr, index, length) behaves like java.lang.String's substring|
|strlen|strlen(expr) returns length of a string in UTF-16 code units|
|strpos|strpos(haystack, needle) returns the position of the needle within the haystack, with indexes starting from 0. If the needle is not found then the function returns -1.|
|trim|trim(expr[, chars]) remove leading and trailing characters from `expr` if they are present in `chars`. `chars` defaults to ' ' (space) if not provided.|
|ltrim|ltrim(expr[, chars]) remove leading characters from `expr` if they are present in `chars`. `chars` defaults to ' ' (space) if not provided.|
|rtrim|rtrim(expr[, chars]) remove trailing characters from `expr` if they are present in `chars`. `chars` defaults to ' ' (space) if not provided.|

View File

@ -56,14 +56,17 @@ WHERE clause can also reference a subquery, like `WHERE col1 IN (SELECT foo FROM
as [semi-joins](#query-execution), described below.
The GROUP BY clause refers to columns in the FROM table. Using GROUP BY, DISTINCT, or any aggregation functions will
trigger an aggregation query using one of Druid's [three native aggregation query types](#query-execution).
trigger an aggregation query using one of Druid's [three native aggregation query types](#query-execution). GROUP BY
can refer to an expression or a select clause ordinal position (like `GROUP BY 2` to group by the second selected
column).
The HAVING clause refers to columns that are present after execution of GROUP BY. It can be used to filter on either
grouping expressions or aggregated values. It can only be used together with GROUP BY.
The ORDER BY clause refers to columns that are present after execution of GROUP BY. It can be used to order the results
based on either grouping expressions or aggregated values. The ORDER BY expression can be a column name, alias, or
ordinal position (like `ORDER BY 2` to order by the second column). ORDER BY can only be used together with GROUP BY.
based on either grouping expressions or aggregated values. ORDER BY can refer to an expression or a select clause
ordinal position (like `ORDER BY 2` to order by the second selected column). For non-aggregation queries, ORDER BY
can only order by the `__time` column. For aggregation queries, ORDER BY can order by any column.
The LIMIT clause can be used to limit the number of rows returned. It can be used with any query type. It is pushed down
to data nodes for queries that run with the native TopN query type, but not the native GroupBy query type. Future
@ -107,6 +110,8 @@ Numeric functions will return 64 bit integers or 64 bit floats, depending on the
|`LOG10(expr)`|Logarithm (base 10).|
|`POW(expr, power)`|expr to a power.|
|`SQRT(expr)`|Square root.|
|`TRUNCATE(expr[, digits])`|Truncate expr to a specific number of decimal digits. If digits is negative, then this truncates that many places to the left of the decimal point. Digits defaults to zero if not specified.|
|`TRUNC(expr[, digits])`|Synonym for `TRUNCATE`.|
|`x + y`|Addition.|
|`x - y`|Subtraction.|
|`x * y`|Multiplication.|
@ -120,12 +125,17 @@ String functions accept strings, and return a type appropriate to the function.
|Function|Notes|
|--------|-----|
|`x \|\| y`|Concat strings x and y.|
|`CHARACTER_LENGTH(expr)`|Length of expr in UTF-16 code units.|
|`LENGTH(expr)`|Length of expr in UTF-16 code units.|
|`CHAR_LENGTH(expr)`|Synonym for `LENGTH`.|
|`CHARACTER_LENGTH(expr)`|Synonym for `LENGTH`.|
|`STRLEN(expr)`|Synonym for `LENGTH`.|
|`LOOKUP(expr, lookupName)`|Look up expr in a registered [query-time lookup table](lookups.html).|
|`LOWER(expr)`|Returns expr in all lowercase.|
|`REGEXP_EXTRACT(expr, pattern, [index])`|Apply regular expression pattern and extract a capture group, or null if there is no match. If index is unspecified or zero, returns the substring that matched the pattern.|
|`REPLACE(expr, pattern, replacement)`|Replaces pattern with replacement in expr, and returns the result.|
|`STRPOS(haystack, needle)`|Returns the index of needle within haystack, starting from 1. If the needle is not found, returns 0.|
|`SUBSTRING(expr, index, [length])`|Returns a substring of expr starting at index, with a max length, both measured in UTF-16 code units.|
|`SUBSTR(expr, index, [length])`|Synonym for SUBSTRING.|
|`TRIM([BOTH | LEADING | TRAILING] [<chars> FROM] expr)`|Returns expr with characters removed from the leading, trailing, or both ends of "expr" if they are in "chars". If "chars" is not provided, it defaults to " " (a space). If the directional argument is not provided, it defaults to "BOTH".|
|`BTRIM(expr[, chars])`|Alternate form of `TRIM(BOTH <chars> FROM <expr>`).|
|`LTRIM(expr[, chars])`|Alternate form of `TRIM(LEADING <chars> FROM <expr>`).|
@ -146,6 +156,7 @@ over the connection time zone.
|--------|-----|
|`CURRENT_TIMESTAMP`|Current timestamp in the connection's time zone.|
|`CURRENT_DATE`|Current date in the connection's time zone.|
|`DATE_TRUNC(<unit>, <timestamp_expr>)`|Rounds down a timestamp, returning it as a new timestamp. Unit can be 'milliseconds', 'second', 'minute', 'hour', 'day', 'week', 'month', 'quarter', 'year', 'decade', 'century', or 'millenium'.|
|`TIME_FLOOR(<timestamp_expr>, <period>, [<origin>, [<timezone>]])`|Rounds down a timestamp, returning it as a new timestamp. Period can be any ISO8601 period, like P3M (quarters) or PT12H (half-days). The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00". This function is similar to `FLOOR` but is more flexible.|
|`TIME_SHIFT(<timestamp_expr>, <period>, <step>, [<timezone>])`|Shifts a timestamp by a period (step times), returning it as a new timestamp. Period can be any ISO8601 period. Step may be negative. The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|
|`TIME_EXTRACT(<timestamp_expr>, [<unit>, [<timezone>]])`|Extracts a time part from expr, returning it as a number. Unit can be EPOCH, SECOND, MINUTE, HOUR, DAY (day of month), DOW (day of week), DOY (day of year), WEEK (week of [week year](https://en.wikipedia.org/wiki/ISO_week_date)), MONTH (1 through 12), QUARTER (1 through 4), or YEAR. The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00". This function is similar to `EXTRACT` but is more flexible. Unit and time zone must be literals, and must be provided quoted, like `TIME_EXTRACT(__time, 'HOUR')` or `TIME_EXTRACT(__time, 'HOUR', 'America/Los_Angeles')`.|
@ -168,6 +179,8 @@ over the connection time zone.
|`x >= y`|Greater than or equal to.|
|`x < y`|Less than.|
|`x <= y`|Less than or equal to.|
|`x BETWEEN y AND z`|Equivalent to `x >= y AND x <= z`.|
|`x NOT BETWEEN y AND z`|Equivalent to `x < y OR x > z`.|
|`x LIKE pattern [ESCAPE esc]`|True if x matches a SQL LIKE pattern (with an optional escape).|
|`x NOT LIKE pattern [ESCAPE esc]`|True if x does not match a SQL LIKE pattern (with an optional escape).|
|`x IS NULL`|True if x is NULL or empty string.|

View File

@ -26,12 +26,10 @@ import io.druid.query.aggregation.histogram.ApproximateHistogram;
import io.druid.query.aggregation.histogram.ApproximateHistogramAggregatorFactory;
import io.druid.query.aggregation.histogram.ApproximateHistogramFoldingAggregatorFactory;
import io.druid.query.aggregation.histogram.QuantilePostAggregator;
import io.druid.query.filter.DimFilter;
import io.druid.segment.VirtualColumn;
import io.druid.segment.column.ValueType;
import io.druid.segment.virtual.ExpressionVirtualColumn;
import io.druid.sql.calcite.aggregation.Aggregation;
import io.druid.sql.calcite.aggregation.Aggregations;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
@ -39,6 +37,7 @@ import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
@ -65,13 +64,13 @@ public class QuantileSqlAggregator implements SqlAggregator
@Override
public Aggregation toDruidAggregation(
final String name,
final RowSignature rowSignature,
final PlannerContext plannerContext,
final List<Aggregation> existingAggregations,
final Project project,
final RowSignature rowSignature,
final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
final DimFilter filter
final Project project,
final List<Aggregation> existingAggregations
)
{
final DruidExpression input = Expressions.toDruidExpression(
@ -115,45 +114,42 @@ public class QuantileSqlAggregator implements SqlAggregator
// Look for existing matching aggregatorFactory.
for (final Aggregation existing : existingAggregations) {
for (AggregatorFactory factory : existing.getAggregatorFactories()) {
final boolean matches = Aggregations.aggregatorMatches(
factory,
filter,
ApproximateHistogramAggregatorFactory.class,
theFactory -> {
// Check input for equivalence.
final boolean inputMatches;
final VirtualColumn virtualInput = existing.getVirtualColumns()
.stream()
.filter(
virtualColumn ->
virtualColumn.getOutputName()
.equals(theFactory.getFieldName())
)
.findFirst()
.orElse(null);
if (factory instanceof ApproximateHistogramAggregatorFactory) {
final ApproximateHistogramAggregatorFactory theFactory = (ApproximateHistogramAggregatorFactory) factory;
if (virtualInput == null) {
inputMatches = input.isDirectColumnAccess()
&& input.getDirectColumn().equals(theFactory.getFieldName());
} else {
inputMatches = ((ExpressionVirtualColumn) virtualInput).getExpression()
.equals(input.getExpression());
}
// Check input for equivalence.
final boolean inputMatches;
final VirtualColumn virtualInput = existing.getVirtualColumns()
.stream()
.filter(
virtualColumn ->
virtualColumn.getOutputName()
.equals(theFactory.getFieldName())
)
.findFirst()
.orElse(null);
return inputMatches
&& theFactory.getResolution() == resolution
&& theFactory.getNumBuckets() == numBuckets
&& theFactory.getLowerLimit() == lowerLimit
&& theFactory.getUpperLimit() == upperLimit;
}
);
if (virtualInput == null) {
inputMatches = input.isDirectColumnAccess()
&& input.getDirectColumn().equals(theFactory.getFieldName());
} else {
inputMatches = ((ExpressionVirtualColumn) virtualInput).getExpression()
.equals(input.getExpression());
}
if (matches) {
// Found existing one. Use this.
return Aggregation.create(
ImmutableList.<AggregatorFactory>of(),
new QuantilePostAggregator(name, factory.getName(), probability)
);
final boolean matches = inputMatches
&& theFactory.getResolution() == resolution
&& theFactory.getNumBuckets() == numBuckets
&& theFactory.getLowerLimit() == lowerLimit
&& theFactory.getUpperLimit() == upperLimit;
if (matches) {
// Found existing one. Use this.
return Aggregation.create(
ImmutableList.of(),
new QuantilePostAggregator(name, factory.getName(), probability)
);
}
}
}
}
@ -202,7 +198,7 @@ public class QuantileSqlAggregator implements SqlAggregator
virtualColumns,
ImmutableList.of(aggregatorFactory),
new QuantilePostAggregator(name, histogramName, probability)
).filter(filter);
);
}
private static class QuantileSqlAggFunction extends SqlAggFunction

View File

@ -61,8 +61,8 @@
<maven.compiler.target>1.8</maven.compiler.target>
<apache.curator.version>4.0.0</apache.curator.version>
<apache.curator.test.version>2.12.0</apache.curator.test.version>
<avatica.version>1.9.0</avatica.version>
<calcite.version>1.12.0</calcite.version>
<avatica.version>1.10.0</avatica.version>
<calcite.version>1.14.0</calcite.version>
<guava.version>16.0.1</guava.version>
<guice.version>4.1.0</guice.version>
<jetty.version>9.3.19.v20170502</jetty.version>

View File

@ -66,8 +66,13 @@ public class TimestampParseExprMacro implements ExprMacroTable.ExprMacro
@Override
public ExprEval eval(final ObjectBinding bindings)
{
final String value = arg.eval(bindings).asString();
if (value == null) {
return ExprEval.of(null);
}
try {
return ExprEval.of(formatter.parse(arg.eval(bindings).asString()).getMillis());
return ExprEval.of(formatter.parse(value).getMillis());
}
catch (IllegalArgumentException e) {
// Catch exceptions potentially thrown by formatter.parseDateTime. Our docs say that unparseable timestamps

View File

@ -29,9 +29,13 @@ 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.AndDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.segment.VirtualColumn;
import io.druid.sql.calcite.filtration.Filtration;
import io.druid.sql.calcite.table.RowSignature;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Objects;
import java.util.Set;
@ -52,6 +56,10 @@ public class Aggregation
this.aggregatorFactories = Preconditions.checkNotNull(aggregatorFactories, "aggregatorFactories");
this.postAggregator = postAggregator;
if (aggregatorFactories.isEmpty()) {
Preconditions.checkArgument(postAggregator != null, "postAggregator must be present if there are no aggregators");
}
if (postAggregator == null) {
Preconditions.checkArgument(aggregatorFactories.size() == 1, "aggregatorFactories.size == 1");
} else {
@ -134,6 +142,7 @@ public class Aggregation
return aggregatorFactories;
}
@Nullable
public PostAggregator getPostAggregator()
{
return postAggregator;
@ -146,7 +155,7 @@ public class Aggregation
: Iterables.getOnlyElement(aggregatorFactories).getName();
}
public Aggregation filter(final DimFilter filter)
public Aggregation filter(final RowSignature sourceRowSignature, final DimFilter filter)
{
if (filter == null) {
return this;
@ -167,9 +176,25 @@ public class Aggregation
}
}
final DimFilter baseOptimizedFilter = Filtration.create(filter)
.optimizeFilterOnly(sourceRowSignature)
.getDimFilter();
final List<AggregatorFactory> newAggregators = Lists.newArrayList();
for (AggregatorFactory agg : aggregatorFactories) {
newAggregators.add(new FilteredAggregatorFactory(agg, filter));
if (agg instanceof FilteredAggregatorFactory) {
final FilteredAggregatorFactory filteredAgg = (FilteredAggregatorFactory) agg;
newAggregators.add(
new FilteredAggregatorFactory(
filteredAgg.getAggregator(),
Filtration.create(new AndDimFilter(ImmutableList.of(filteredAgg.getFilter(), baseOptimizedFilter)))
.optimizeFilterOnly(sourceRowSignature)
.getDimFilter()
)
);
} else {
newAggregators.add(new FilteredAggregatorFactory(agg, baseOptimizedFilter));
}
}
return new Aggregation(virtualColumns, newAggregators, postAggregator);

View File

@ -19,10 +19,19 @@
package io.druid.sql.calcite.aggregation;
import com.google.common.base.Predicate;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.FilteredAggregatorFactory;
import io.druid.query.filter.DimFilter;
import io.druid.segment.column.ValueType;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexNode;
import javax.annotation.Nullable;
import java.util.List;
import java.util.function.Function;
import java.util.stream.Collectors;
public class Aggregations
{
@ -31,30 +40,38 @@ public class Aggregations
// No instantiation.
}
/**
* Returns true if "factory" is an aggregator factory that either matches "predicate" (if filter is null) or is
* a filtered aggregator factory whose filter is equal to "filter" and underlying aggregator matches "predicate".
*
* @param factory factory to match
* @param filter filter, may be null
* @param clazz class of factory to match
* @param predicate predicate
*
* @return true if the aggregator matches filter + predicate
*/
public static <T extends AggregatorFactory> boolean aggregatorMatches(
final AggregatorFactory factory,
final DimFilter filter,
final Class<T> clazz,
final Predicate<T> predicate
@Nullable
public static List<DruidExpression> getArgumentsForSimpleAggregator(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final AggregateCall call,
final Project project
)
{
if (filter != null) {
return factory instanceof FilteredAggregatorFactory &&
((FilteredAggregatorFactory) factory).getFilter().equals(filter)
&& aggregatorMatches(((FilteredAggregatorFactory) factory).getAggregator(), null, clazz, predicate);
return call.getArgList().stream()
.map(i -> Expressions.fromFieldAccess(rowSignature, project, i))
.map(rexNode -> toDruidExpressionForSimpleAggregator(plannerContext, rowSignature, rexNode))
.collect(Collectors.toList());
}
private static DruidExpression toDruidExpressionForSimpleAggregator(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexNode rexNode
)
{
final DruidExpression druidExpression = Expressions.toDruidExpression(plannerContext, rowSignature, rexNode);
if (druidExpression == null) {
return null;
}
if (druidExpression.isSimpleExtraction() &&
(!druidExpression.isDirectColumnAccess()
|| rowSignature.getColumnType(druidExpression.getDirectColumn()) == ValueType.STRING)) {
// Aggregators are unable to implicitly cast strings to numbers. So remove the simple extraction in this case.
return druidExpression.map(simpleExtraction -> null, Function.identity());
} else {
return clazz.isAssignableFrom(factory.getClass()) && predicate.apply(clazz.cast(factory));
return druidExpression;
}
}
}

View File

@ -19,11 +19,11 @@
package io.druid.sql.calcite.aggregation;
import io.druid.query.filter.DimFilter;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.sql.SqlAggFunction;
import javax.annotation.Nullable;
@ -42,27 +42,28 @@ public interface SqlAggregator
SqlAggFunction calciteFunction();
/**
* Returns Druid Aggregation corresponding to a SQL {@link AggregateCall}.
* Returns a Druid Aggregation corresponding to a SQL {@link AggregateCall}. This method should ignore filters;
* they will be applied to your aggregator in a later step.
*
* @param name desired output name of the aggregation
* @param rowSignature signature of the rows being aggregated
* @param plannerContext SQL planner context
* @param rowSignature signature of the rows being aggregated
* @param rexBuilder a rexBuilder, in case you need one
* @param name desired output name of the aggregation
* @param aggregateCall aggregate call object
* @param project project that should be applied before aggregation; may be null
* @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.
* @param project SQL projection to apply before the aggregate call, may be null
* @param aggregateCall SQL aggregate call
* @param filter filter that should be applied to the aggregation, may be null
*
* @return aggregation, or null if the call cannot be translated
*/
@Nullable
Aggregation toDruidAggregation(
final String name,
final RowSignature rowSignature,
final PlannerContext plannerContext,
final List<Aggregation> existingAggregations,
final Project project,
final RowSignature rowSignature,
final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
final DimFilter filter
final Project project,
final List<Aggregation> existingAggregations
);
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package io.druid.sql.calcite.aggregation;
package io.druid.sql.calcite.aggregation.builtin;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
@ -28,10 +28,11 @@ import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.filter.DimFilter;
import io.druid.segment.VirtualColumn;
import io.druid.segment.column.ValueType;
import io.druid.segment.virtual.ExpressionVirtualColumn;
import io.druid.sql.calcite.aggregation.Aggregation;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.planner.Calcites;
@ -39,6 +40,7 @@ import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
@ -48,6 +50,7 @@ import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeName;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.List;
@ -62,36 +65,36 @@ public class ApproxCountDistinctSqlAggregator implements SqlAggregator
return FUNCTION_INSTANCE;
}
@Nullable
@Override
public Aggregation toDruidAggregation(
final String name,
final RowSignature rowSignature,
final PlannerContext plannerContext,
final List<Aggregation> existingAggregations,
final Project project,
final RowSignature rowSignature,
final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
final DimFilter filter
final Project project,
final List<Aggregation> existingAggregations
)
{
// Don't use Aggregations.getArgumentsForSimpleAggregator, since it won't let us use direct column access
// for string columns.
final RexNode rexNode = Expressions.fromFieldAccess(
rowSignature,
project,
Iterables.getOnlyElement(aggregateCall.getArgList())
);
final DruidExpression input = Expressions.toDruidExpression(
plannerContext,
rowSignature,
rexNode
);
if (input == null) {
final DruidExpression arg = Expressions.toDruidExpression(plannerContext, rowSignature, rexNode);
if (arg == null) {
return null;
}
final List<VirtualColumn> virtualColumns = new ArrayList<>();
final AggregatorFactory aggregatorFactory;
if (input.isDirectColumnAccess() && rowSignature.getColumnType(input.getDirectColumn()) == ValueType.COMPLEX) {
aggregatorFactory = new HyperUniquesAggregatorFactory(name, input.getDirectColumn(), false, true);
if (arg.isDirectColumnAccess() && rowSignature.getColumnType(arg.getDirectColumn()) == ValueType.COMPLEX) {
aggregatorFactory = new HyperUniquesAggregatorFactory(name, arg.getDirectColumn(), false, true);
} else {
final SqlTypeName sqlTypeName = rexNode.getType().getSqlTypeName();
final ValueType inputType = Calcites.getValueTypeForSqlTypeName(sqlTypeName);
@ -101,10 +104,10 @@ public class ApproxCountDistinctSqlAggregator implements SqlAggregator
final DimensionSpec dimensionSpec;
if (input.isSimpleExtraction()) {
dimensionSpec = input.getSimpleExtraction().toDimensionSpec(null, ValueType.STRING);
if (arg.isSimpleExtraction()) {
dimensionSpec = arg.getSimpleExtraction().toDimensionSpec(null, inputType);
} else {
final ExpressionVirtualColumn virtualColumn = input.toVirtualColumn(
final ExpressionVirtualColumn virtualColumn = arg.toVirtualColumn(
StringUtils.format("%s:v", name),
inputType,
plannerContext.getExprMacroTable()
@ -116,7 +119,7 @@ public class ApproxCountDistinctSqlAggregator implements SqlAggregator
aggregatorFactory = new CardinalityAggregatorFactory(name, null, ImmutableList.of(dimensionSpec), false, true);
}
return Aggregation.create(virtualColumns, aggregatorFactory).filter(filter);
return Aggregation.create(virtualColumns, aggregatorFactory);
}
private static class ApproxCountDistinctSqlAggFunction extends SqlAggFunction

View File

@ -0,0 +1,129 @@
/*
* 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.builtin;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import io.druid.java.util.common.StringUtils;
import io.druid.math.expr.ExprMacroTable;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.post.ArithmeticPostAggregator;
import io.druid.query.aggregation.post.FieldAccessPostAggregator;
import io.druid.segment.column.ValueType;
import io.druid.sql.calcite.aggregation.Aggregation;
import io.druid.sql.calcite.aggregation.Aggregations;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.type.SqlTypeName;
import javax.annotation.Nullable;
import java.util.List;
public class AvgSqlAggregator implements SqlAggregator
{
@Override
public SqlAggFunction calciteFunction()
{
return SqlStdOperatorTable.AVG;
}
@Nullable
@Override
public Aggregation toDruidAggregation(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
final Project project,
final List<Aggregation> existingAggregations
)
{
if (aggregateCall.isDistinct()) {
return null;
}
final List<DruidExpression> arguments = Aggregations.getArgumentsForSimpleAggregator(
plannerContext,
rowSignature,
aggregateCall,
project
);
if (arguments == null) {
return null;
}
final DruidExpression arg = Iterables.getOnlyElement(arguments);
final ValueType sumType;
// Use 64-bit sum regardless of the type of the AVG aggregator.
if (SqlTypeName.INT_TYPES.contains(aggregateCall.getType().getSqlTypeName())) {
sumType = ValueType.LONG;
} else {
sumType = ValueType.DOUBLE;
}
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
final String fieldName;
final String expression;
if (arg.isDirectColumnAccess()) {
fieldName = arg.getDirectColumn();
expression = null;
} else {
fieldName = null;
expression = arg.getExpression();
}
final String sumName = StringUtils.format("%s:sum", name);
final String countName = StringUtils.format("%s:count", name);
final AggregatorFactory sum = SumSqlAggregator.createSumAggregatorFactory(
sumType,
sumName,
fieldName,
expression,
macroTable
);
final AggregatorFactory count = new CountAggregatorFactory(countName);
return Aggregation.create(
ImmutableList.of(sum, count),
new ArithmeticPostAggregator(
name,
"quotient",
ImmutableList.of(
new FieldAccessPostAggregator(null, sumName),
new FieldAccessPostAggregator(null, countName)
)
)
);
}
}

View File

@ -0,0 +1,123 @@
/*
* 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.builtin;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import io.druid.java.util.common.ISE;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.filter.DimFilter;
import io.druid.sql.calcite.aggregation.Aggregation;
import io.druid.sql.calcite.aggregation.Aggregations;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import javax.annotation.Nullable;
import java.util.List;
public class CountSqlAggregator implements SqlAggregator
{
private static final ApproxCountDistinctSqlAggregator APPROX_COUNT_DISTINCT = new ApproxCountDistinctSqlAggregator();
@Override
public SqlAggFunction calciteFunction()
{
return SqlStdOperatorTable.COUNT;
}
@Nullable
@Override
public Aggregation toDruidAggregation(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
final Project project,
final List<Aggregation> existingAggregations
)
{
final List<DruidExpression> args = Aggregations.getArgumentsForSimpleAggregator(
plannerContext,
rowSignature,
aggregateCall,
project
);
if (args == null) {
return null;
}
if (args.isEmpty()) {
// COUNT(*)
return Aggregation.create(new CountAggregatorFactory(name));
} else if (aggregateCall.isDistinct()) {
// COUNT(DISTINCT x)
if (plannerContext.getPlannerConfig().isUseApproximateCountDistinct()) {
return APPROX_COUNT_DISTINCT.toDruidAggregation(
plannerContext,
rowSignature,
rexBuilder,
name,
aggregateCall,
project,
existingAggregations
);
} else {
return null;
}
} else {
// Not COUNT(*), not distinct
// COUNT(x) should count all non-null values of x.
final RexNode rexNode = Expressions.fromFieldAccess(
rowSignature,
project,
Iterables.getOnlyElement(aggregateCall.getArgList())
);
if (rexNode.getType().isNullable()) {
final DimFilter nonNullFilter = Expressions.toFilter(
plannerContext,
rowSignature,
rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, ImmutableList.of(rexNode))
);
if (nonNullFilter == null) {
// Don't expect this to happen.
throw new ISE("Could not create not-null filter for rexNode[%s]", rexNode);
}
return Aggregation.create(new CountAggregatorFactory(name)).filter(rowSignature, nonNullFilter);
} else {
return Aggregation.create(new CountAggregatorFactory(name));
}
}
}
}

View File

@ -0,0 +1,118 @@
/*
* 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.builtin;
import com.google.common.collect.Iterables;
import io.druid.java.util.common.ISE;
import io.druid.math.expr.ExprMacroTable;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
import io.druid.query.aggregation.FloatMaxAggregatorFactory;
import io.druid.query.aggregation.LongMaxAggregatorFactory;
import io.druid.segment.column.ValueType;
import io.druid.sql.calcite.aggregation.Aggregation;
import io.druid.sql.calcite.aggregation.Aggregations;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.planner.Calcites;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import javax.annotation.Nullable;
import java.util.List;
public class MaxSqlAggregator implements SqlAggregator
{
@Override
public SqlAggFunction calciteFunction()
{
return SqlStdOperatorTable.MAX;
}
@Nullable
@Override
public Aggregation toDruidAggregation(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
final Project project,
final List<Aggregation> existingAggregations
)
{
if (aggregateCall.isDistinct()) {
return null;
}
final List<DruidExpression> arguments = Aggregations.getArgumentsForSimpleAggregator(
plannerContext,
rowSignature,
aggregateCall,
project
);
if (arguments == null) {
return null;
}
final DruidExpression arg = Iterables.getOnlyElement(arguments);
final ValueType valueType = Calcites.getValueTypeForSqlTypeName(aggregateCall.getType().getSqlTypeName());
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
final String fieldName;
final String expression;
if (arg.isDirectColumnAccess()) {
fieldName = arg.getDirectColumn();
expression = null;
} else {
fieldName = null;
expression = arg.getExpression();
}
return Aggregation.create(createMaxAggregatorFactory(valueType, name, fieldName, expression, macroTable));
}
private static AggregatorFactory createMaxAggregatorFactory(
final ValueType aggregationType,
final String name,
final String fieldName,
final String expression,
final ExprMacroTable macroTable
)
{
switch (aggregationType) {
case LONG:
return new LongMaxAggregatorFactory(name, fieldName, expression, macroTable);
case FLOAT:
return new FloatMaxAggregatorFactory(name, fieldName, expression, macroTable);
case DOUBLE:
return new DoubleMaxAggregatorFactory(name, fieldName, expression, macroTable);
default:
throw new ISE("Cannot create aggregator factory for type[%s]", aggregationType);
}
}
}

View File

@ -0,0 +1,118 @@
/*
* 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.builtin;
import com.google.common.collect.Iterables;
import io.druid.java.util.common.ISE;
import io.druid.math.expr.ExprMacroTable;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleMinAggregatorFactory;
import io.druid.query.aggregation.FloatMinAggregatorFactory;
import io.druid.query.aggregation.LongMinAggregatorFactory;
import io.druid.segment.column.ValueType;
import io.druid.sql.calcite.aggregation.Aggregation;
import io.druid.sql.calcite.aggregation.Aggregations;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.planner.Calcites;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import javax.annotation.Nullable;
import java.util.List;
public class MinSqlAggregator implements SqlAggregator
{
@Override
public SqlAggFunction calciteFunction()
{
return SqlStdOperatorTable.MIN;
}
@Nullable
@Override
public Aggregation toDruidAggregation(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
final Project project,
final List<Aggregation> existingAggregations
)
{
if (aggregateCall.isDistinct()) {
return null;
}
final List<DruidExpression> arguments = Aggregations.getArgumentsForSimpleAggregator(
plannerContext,
rowSignature,
aggregateCall,
project
);
if (arguments == null) {
return null;
}
final DruidExpression arg = Iterables.getOnlyElement(arguments);
final ValueType valueType = Calcites.getValueTypeForSqlTypeName(aggregateCall.getType().getSqlTypeName());
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
final String fieldName;
final String expression;
if (arg.isDirectColumnAccess()) {
fieldName = arg.getDirectColumn();
expression = null;
} else {
fieldName = null;
expression = arg.getExpression();
}
return Aggregation.create(createMinAggregatorFactory(valueType, name, fieldName, expression, macroTable));
}
private static AggregatorFactory createMinAggregatorFactory(
final ValueType aggregationType,
final String name,
final String fieldName,
final String expression,
final ExprMacroTable macroTable
)
{
switch (aggregationType) {
case LONG:
return new LongMinAggregatorFactory(name, fieldName, expression, macroTable);
case FLOAT:
return new FloatMinAggregatorFactory(name, fieldName, expression, macroTable);
case DOUBLE:
return new DoubleMinAggregatorFactory(name, fieldName, expression, macroTable);
default:
throw new ISE("Cannot create aggregator factory for type[%s]", aggregationType);
}
}
}

View File

@ -0,0 +1,118 @@
/*
* 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.builtin;
import com.google.common.collect.Iterables;
import io.druid.java.util.common.ISE;
import io.druid.math.expr.ExprMacroTable;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.query.aggregation.FloatSumAggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.segment.column.ValueType;
import io.druid.sql.calcite.aggregation.Aggregation;
import io.druid.sql.calcite.aggregation.Aggregations;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.planner.Calcites;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import javax.annotation.Nullable;
import java.util.List;
public class SumSqlAggregator implements SqlAggregator
{
@Override
public SqlAggFunction calciteFunction()
{
return SqlStdOperatorTable.SUM;
}
@Nullable
@Override
public Aggregation toDruidAggregation(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
final Project project,
final List<Aggregation> existingAggregations
)
{
if (aggregateCall.isDistinct()) {
return null;
}
final List<DruidExpression> arguments = Aggregations.getArgumentsForSimpleAggregator(
plannerContext,
rowSignature,
aggregateCall,
project
);
if (arguments == null) {
return null;
}
final DruidExpression arg = Iterables.getOnlyElement(arguments);
final ValueType valueType = Calcites.getValueTypeForSqlTypeName(aggregateCall.getType().getSqlTypeName());
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
final String fieldName;
final String expression;
if (arg.isDirectColumnAccess()) {
fieldName = arg.getDirectColumn();
expression = null;
} else {
fieldName = null;
expression = arg.getExpression();
}
return Aggregation.create(createSumAggregatorFactory(valueType, name, fieldName, expression, macroTable));
}
static AggregatorFactory createSumAggregatorFactory(
final ValueType aggregationType,
final String name,
final String fieldName,
final String expression,
final ExprMacroTable macroTable
)
{
switch (aggregationType) {
case LONG:
return new LongSumAggregatorFactory(name, fieldName, expression, macroTable);
case FLOAT:
return new FloatSumAggregatorFactory(name, fieldName, expression, macroTable);
case DOUBLE:
return new DoubleSumAggregatorFactory(name, fieldName, expression, macroTable);
default:
throw new ISE("Cannot create aggregator factory for type[%s]", aggregationType);
}
}
}

View File

@ -0,0 +1,32 @@
/*
* 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.builtin;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
public class SumZeroSqlAggregator extends SumSqlAggregator
{
@Override
public SqlAggFunction calciteFunction()
{
return SqlStdOperatorTable.SUM0;
}
}

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.java.util.common.IAE;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlOperator;
public class AliasedOperatorConversion implements SqlOperatorConversion
{
private final SqlOperatorConversion baseConversion;
private final String name;
private final SqlOperator operator;
public AliasedOperatorConversion(final SqlOperatorConversion baseConversion, final String name)
{
if (!SqlKind.FUNCTION.contains(baseConversion.calciteOperator().getKind())) {
throw new IAE("Base operator must be a function but was[%s]", baseConversion.calciteOperator().getKind());
}
final SqlFunction baseFunction = (SqlFunction) baseConversion.calciteOperator();
this.baseConversion = baseConversion;
this.name = name;
this.operator = new SqlFunction(
name,
baseFunction.getKind(),
baseFunction.getReturnTypeInference(),
baseFunction.getOperandTypeInference(),
baseFunction.getOperandTypeChecker(),
baseFunction.getFunctionType()
);
}
@Override
public SqlOperator calciteOperator()
{
return operator;
}
@Override
public DruidExpression toDruidExpression(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexNode rexNode
)
{
return baseConversion.toDruidExpression(plannerContext, rowSignature, rexNode);
}
}

View File

@ -0,0 +1,73 @@
/*
* 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.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlOperator;
public class BinaryOperatorConversion implements SqlOperatorConversion
{
private final SqlOperator operator;
private final String druidOperator;
public BinaryOperatorConversion(final SqlOperator operator, final String druidOperator)
{
this.operator = operator;
this.druidOperator = druidOperator;
}
@Override
public SqlOperator calciteOperator()
{
return operator;
}
@Override
public DruidExpression toDruidExpression(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexNode rexNode
)
{
return OperatorConversions.convertCall(
plannerContext,
rowSignature,
rexNode,
operands -> {
if (operands.size() != 2) {
throw new ISE("WTF?! Got binary operator[%s] with %s args?", operator.getName(), operands.size());
}
return DruidExpression.fromExpression(
StringUtils.format(
"(%s %s %s)",
operands.get(0).getExpression(),
druidOperator,
operands.get(1).getExpression()
)
);
}
);
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlOperator;
public class DirectOperatorConversion implements SqlOperatorConversion
{
private final SqlOperator operator;
private final String druidFunctionName;
public DirectOperatorConversion(final SqlOperator operator, final String druidFunctionName)
{
this.operator = operator;
this.druidFunctionName = druidFunctionName;
}
@Override
public SqlOperator calciteOperator()
{
return operator;
}
@Override
public DruidExpression toDruidExpression(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexNode rexNode
)
{
return OperatorConversions.convertCall(
plannerContext,
rowSignature,
rexNode,
operands -> DruidExpression.fromExpression(DruidExpression.functionCall(druidFunctionName, operands))
);
}
}

View File

@ -20,15 +20,11 @@
package io.druid.sql.calcite.expression;
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 io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.java.util.common.granularity.PeriodGranularity;
import io.druid.math.expr.ExprType;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.TimeFormatExtractionFn;
@ -57,71 +53,19 @@ import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.joda.time.Interval;
import org.joda.time.Period;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
/**
* A collection of functions for translating from Calcite expressions into Druid objects.
*/
public class Expressions
{
private static final Map<SqlOperator, String> DIRECT_CONVERSIONS = ImmutableMap.<SqlOperator, String>builder()
.put(SqlStdOperatorTable.ABS, "abs")
.put(SqlStdOperatorTable.CASE, "case_searched")
.put(SqlStdOperatorTable.CHAR_LENGTH, "strlen")
.put(SqlStdOperatorTable.CHARACTER_LENGTH, "strlen")
.put(SqlStdOperatorTable.CONCAT, "concat")
.put(SqlStdOperatorTable.EXP, "exp")
.put(SqlStdOperatorTable.DIVIDE_INTEGER, "div")
.put(SqlStdOperatorTable.LIKE, "like")
.put(SqlStdOperatorTable.LN, "log")
.put(SqlStdOperatorTable.LOWER, "lower")
.put(SqlStdOperatorTable.LOG10, "log10")
.put(SqlStdOperatorTable.POWER, "pow")
.put(SqlStdOperatorTable.REPLACE, "replace")
.put(SqlStdOperatorTable.SQRT, "sqrt")
.put(SqlStdOperatorTable.UPPER, "upper")
.build();
private static final Map<SqlOperator, String> UNARY_PREFIX_OPERATOR_MAP = ImmutableMap.<SqlOperator, String>builder()
.put(SqlStdOperatorTable.NOT, "!")
.put(SqlStdOperatorTable.UNARY_MINUS, "-")
.build();
private static final Map<SqlOperator, String> UNARY_SUFFIX_OPERATOR_MAP = ImmutableMap.<SqlOperator, String>builder()
.put(SqlStdOperatorTable.IS_NULL, "== ''")
.put(SqlStdOperatorTable.IS_NOT_NULL, "!= ''")
.put(SqlStdOperatorTable.IS_FALSE, "<= 0") // Matches Evals.asBoolean
.put(SqlStdOperatorTable.IS_NOT_TRUE, "<= 0") // Matches Evals.asBoolean
.put(SqlStdOperatorTable.IS_TRUE, "> 0") // Matches Evals.asBoolean
.put(SqlStdOperatorTable.IS_NOT_FALSE, "> 0") // Matches Evals.asBoolean
.build();
private static final Map<SqlOperator, String> BINARY_OPERATOR_MAP = ImmutableMap.<SqlOperator, String>builder()
.put(SqlStdOperatorTable.MULTIPLY, "*")
.put(SqlStdOperatorTable.MOD, "%")
.put(SqlStdOperatorTable.DIVIDE, "/")
.put(SqlStdOperatorTable.PLUS, "+")
.put(SqlStdOperatorTable.MINUS, "-")
.put(SqlStdOperatorTable.EQUALS, "==")
.put(SqlStdOperatorTable.NOT_EQUALS, "!=")
.put(SqlStdOperatorTable.GREATER_THAN, ">")
.put(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, ">=")
.put(SqlStdOperatorTable.LESS_THAN, "<")
.put(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, "<=")
.put(SqlStdOperatorTable.AND, "&&")
.put(SqlStdOperatorTable.OR, "||")
.build();
private Expressions()
{
// No instantiation.
@ -207,131 +151,16 @@ public class Expressions
}
return DruidExpression.fromColumn(columnName);
} else if (kind == SqlKind.CAST || kind == SqlKind.REINTERPRET) {
// Translate casts.
final RexNode operand = ((RexCall) rexNode).getOperands().get(0);
final DruidExpression operandExpression = toDruidExpression(
plannerContext,
rowSignature,
operand
);
if (operandExpression == null) {
return null;
}
final SqlTypeName fromType = operand.getType().getSqlTypeName();
final SqlTypeName toType = rexNode.getType().getSqlTypeName();
if (SqlTypeName.CHAR_TYPES.contains(fromType) && SqlTypeName.DATETIME_TYPES.contains(toType)) {
// Cast strings to datetimes by parsing them from SQL format.
final DruidExpression timestampExpression = DruidExpression.fromFunctionCall(
"timestamp_parse",
ImmutableList.of(
operandExpression,
DruidExpression.fromExpression(DruidExpression.stringLiteral(dateTimeFormatString(toType)))
)
);
if (toType == SqlTypeName.DATE) {
return TimeFloorOperatorConversion.applyTimestampFloor(
timestampExpression,
new PeriodGranularity(Period.days(1), null, plannerContext.getTimeZone())
);
} else {
return timestampExpression;
}
} else if (SqlTypeName.DATETIME_TYPES.contains(fromType) && SqlTypeName.CHAR_TYPES.contains(toType)) {
// Cast datetimes to strings by formatting them in SQL format.
return DruidExpression.fromFunctionCall(
"timestamp_format",
ImmutableList.of(
operandExpression,
DruidExpression.fromExpression(DruidExpression.stringLiteral(dateTimeFormatString(fromType)))
)
);
} else {
// Handle other casts.
final ExprType fromExprType = exprTypeForValueType(Calcites.getValueTypeForSqlTypeName(fromType));
final ExprType toExprType = exprTypeForValueType(Calcites.getValueTypeForSqlTypeName(toType));
if (fromExprType == null || toExprType == null) {
// We have no runtime type for these SQL types.
return null;
}
final DruidExpression typeCastExpression;
if (fromExprType != toExprType) {
// Ignore casts for simple extractions (use Function.identity) since it is ok in many cases.
typeCastExpression = operandExpression.map(
Function.identity(),
expression -> StringUtils.format("CAST(%s, '%s')", expression, toExprType.toString())
);
} else {
typeCastExpression = operandExpression;
}
if (toType == SqlTypeName.DATE) {
// Floor to day when casting to DATE.
return TimeFloorOperatorConversion.applyTimestampFloor(
typeCastExpression,
new PeriodGranularity(Period.days(1), null, plannerContext.getTimeZone())
);
} else {
return typeCastExpression;
}
}
} else if (rexNode instanceof RexCall) {
final SqlOperator operator = ((RexCall) rexNode).getOperator();
final SqlOperatorConversion conversion = plannerContext.getOperatorTable()
.lookupOperatorConversion(operator);
if (conversion != null) {
return conversion.toDruidExpression(plannerContext, rowSignature, rexNode);
}
final List<DruidExpression> operands = Expressions.toDruidExpressions(
plannerContext,
rowSignature,
((RexCall) rexNode).getOperands()
);
if (operands == null) {
if (conversion == null) {
return null;
} else if (UNARY_PREFIX_OPERATOR_MAP.containsKey(operator)) {
return DruidExpression.fromExpression(
StringUtils.format(
"(%s %s)",
UNARY_PREFIX_OPERATOR_MAP.get(operator),
Iterables.getOnlyElement(operands).getExpression()
)
);
} else if (UNARY_SUFFIX_OPERATOR_MAP.containsKey(operator)) {
return DruidExpression.fromExpression(
StringUtils.format(
"(%s %s)",
Iterables.getOnlyElement(operands).getExpression(),
UNARY_SUFFIX_OPERATOR_MAP.get(operator)
)
);
} else if (BINARY_OPERATOR_MAP.containsKey(operator)) {
if (operands.size() != 2) {
throw new ISE("WTF?! Got binary operator[%s] with %s args?", kind, operands.size());
}
return DruidExpression.fromExpression(
StringUtils.format(
"(%s %s %s)",
operands.get(0).getExpression(),
BINARY_OPERATOR_MAP.get(operator),
operands.get(1).getExpression()
)
);
} else if (DIRECT_CONVERSIONS.containsKey(operator)) {
final String functionName = DIRECT_CONVERSIONS.get(operator);
return DruidExpression.fromExpression(DruidExpression.functionCall(functionName, operands));
} else {
return null;
return conversion.toDruidExpression(plannerContext, rowSignature, rexNode);
}
} else if (kind == SqlKind.LITERAL) {
// Translate literal.
@ -651,15 +480,4 @@ public class Expressions
? null
: new ExpressionDimFilter(druidExpression.getExpression(), plannerContext.getExprMacroTable());
}
private static String dateTimeFormatString(final SqlTypeName sqlTypeName)
{
if (sqlTypeName == SqlTypeName.DATE) {
return "yyyy-MM-dd";
} else if (sqlTypeName == SqlTypeName.TIMESTAMP) {
return "yyyy-MM-dd HH:mm:ss";
} else {
throw new ISE("Unsupported DateTime type[%s]", sqlTypeName);
}
}
}

View File

@ -147,6 +147,12 @@ public class OperatorConversions
return this;
}
public OperatorBuilder returnTypeInference(final SqlReturnTypeInference returnTypeInference)
{
this.returnTypeInference = returnTypeInference;
return this;
}
public OperatorBuilder functionCategory(final SqlFunctionCategory functionCategory)
{
this.functionCategory = functionCategory;

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.expression;
import com.google.common.collect.Iterables;
import io.druid.java.util.common.StringUtils;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlOperator;
public class UnaryPrefixOperatorConversion implements SqlOperatorConversion
{
private final SqlOperator operator;
private final String druidOperator;
public UnaryPrefixOperatorConversion(final SqlOperator operator, final String druidOperator)
{
this.operator = operator;
this.druidOperator = druidOperator;
}
@Override
public SqlOperator calciteOperator()
{
return operator;
}
@Override
public DruidExpression toDruidExpression(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexNode rexNode
)
{
return OperatorConversions.convertCall(
plannerContext,
rowSignature,
rexNode,
operands -> DruidExpression.fromExpression(
StringUtils.format(
"(%s %s)",
druidOperator,
Iterables.getOnlyElement(operands).getExpression()
)
)
);
}
}

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.expression;
import com.google.common.collect.Iterables;
import io.druid.java.util.common.StringUtils;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlOperator;
public class UnarySuffixOperatorConversion implements SqlOperatorConversion
{
private final SqlOperator operator;
private final String druidOperator;
public UnarySuffixOperatorConversion(final SqlOperator operator, final String druidOperator)
{
this.operator = operator;
this.druidOperator = druidOperator;
}
@Override
public SqlOperator calciteOperator()
{
return operator;
}
@Override
public DruidExpression toDruidExpression(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexNode rexNode
)
{
return OperatorConversions.convertCall(
plannerContext,
rowSignature,
rexNode,
operands -> DruidExpression.fromExpression(
StringUtils.format(
"(%s %s)",
Iterables.getOnlyElement(operands).getExpression(),
druidOperator
)
)
);
}
}

View File

@ -17,8 +17,11 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;

View File

@ -0,0 +1,191 @@
/*
* 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.builtin;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.PeriodGranularity;
import io.druid.math.expr.ExprType;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.type.SqlTypeName;
import org.joda.time.Period;
import java.util.Map;
import java.util.function.Function;
public class CastOperatorConversion implements SqlOperatorConversion
{
private static final Map<SqlTypeName, ExprType> EXPRESSION_TYPES;
static {
final ImmutableMap.Builder<SqlTypeName, ExprType> builder = ImmutableMap.builder();
for (SqlTypeName type : SqlTypeName.FRACTIONAL_TYPES) {
builder.put(type, ExprType.DOUBLE);
}
for (SqlTypeName type : SqlTypeName.INT_TYPES) {
builder.put(type, ExprType.LONG);
}
for (SqlTypeName type : SqlTypeName.STRING_TYPES) {
builder.put(type, ExprType.STRING);
}
// Booleans are treated as longs in Druid expressions, using two-value logic (positive = true, nonpositive = false).
builder.put(SqlTypeName.BOOLEAN, ExprType.LONG);
// Timestamps are treated as longs (millis since the epoch) in Druid expressions.
builder.put(SqlTypeName.TIMESTAMP, ExprType.LONG);
builder.put(SqlTypeName.DATE, ExprType.LONG);
EXPRESSION_TYPES = builder.build();
}
@Override
public SqlOperator calciteOperator()
{
return SqlStdOperatorTable.CAST;
}
@Override
public DruidExpression toDruidExpression(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexNode rexNode
)
{
final RexNode operand = ((RexCall) rexNode).getOperands().get(0);
final DruidExpression operandExpression = Expressions.toDruidExpression(
plannerContext,
rowSignature,
operand
);
if (operandExpression == null) {
return null;
}
final SqlTypeName fromType = operand.getType().getSqlTypeName();
final SqlTypeName toType = rexNode.getType().getSqlTypeName();
if (SqlTypeName.CHAR_TYPES.contains(fromType) && SqlTypeName.DATETIME_TYPES.contains(toType)) {
return castCharToDateTime(plannerContext, operandExpression, toType);
} else if (SqlTypeName.DATETIME_TYPES.contains(fromType) && SqlTypeName.CHAR_TYPES.contains(toType)) {
return castDateTimeToChar(plannerContext, operandExpression, fromType);
} else {
// Handle other casts.
final ExprType fromExprType = EXPRESSION_TYPES.get(fromType);
final ExprType toExprType = EXPRESSION_TYPES.get(toType);
if (fromExprType == null || toExprType == null) {
// We have no runtime type for these SQL types.
return null;
}
final DruidExpression typeCastExpression;
if (fromExprType != toExprType) {
// Ignore casts for simple extractions (use Function.identity) since it is ok in many cases.
typeCastExpression = operandExpression.map(
Function.identity(),
expression -> StringUtils.format("CAST(%s, '%s')", expression, toExprType.toString())
);
} else {
typeCastExpression = operandExpression;
}
if (toType == SqlTypeName.DATE) {
// Floor to day when casting to DATE.
return TimeFloorOperatorConversion.applyTimestampFloor(
typeCastExpression,
new PeriodGranularity(Period.days(1), null, plannerContext.getTimeZone())
);
} else {
return typeCastExpression;
}
}
}
private static DruidExpression castCharToDateTime(
final PlannerContext plannerContext,
final DruidExpression operand,
final SqlTypeName toType
)
{
// Cast strings to datetimes by parsin them from SQL format.
final DruidExpression timestampExpression = DruidExpression.fromFunctionCall(
"timestamp_parse",
ImmutableList.of(
operand,
DruidExpression.fromExpression(DruidExpression.stringLiteral(dateTimeFormatString(toType))),
DruidExpression.fromExpression(DruidExpression.stringLiteral(plannerContext.getTimeZone().getID()))
)
);
if (toType == SqlTypeName.DATE) {
return TimeFloorOperatorConversion.applyTimestampFloor(
timestampExpression,
new PeriodGranularity(Period.days(1), null, plannerContext.getTimeZone())
);
} else if (toType == SqlTypeName.TIMESTAMP) {
return timestampExpression;
} else {
throw new ISE("Unsupported DateTime type[%s]", toType);
}
}
private static DruidExpression castDateTimeToChar(
final PlannerContext plannerContext,
final DruidExpression operand,
final SqlTypeName fromType
)
{
return DruidExpression.fromFunctionCall(
"timestamp_format",
ImmutableList.of(
operand,
DruidExpression.fromExpression(DruidExpression.stringLiteral(dateTimeFormatString(fromType))),
DruidExpression.fromExpression(DruidExpression.stringLiteral(plannerContext.getTimeZone().getID()))
)
);
}
private static String dateTimeFormatString(final SqlTypeName sqlTypeName)
{
if (sqlTypeName == SqlTypeName.DATE) {
return "yyyy-MM-dd";
} else if (sqlTypeName == SqlTypeName.TIMESTAMP) {
return "yyyy-MM-dd HH:mm:ss";
} else {
throw new ISE("Unsupported DateTime type[%s]", sqlTypeName);
}
}
}

View File

@ -17,11 +17,15 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.PeriodGranularity;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.expression.TimeUnits;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.avatica.util.TimeUnitRange;

View File

@ -0,0 +1,121 @@
/*
* 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.builtin;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.StringUtils;
import io.druid.math.expr.Expr;
import io.druid.math.expr.Parser;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.joda.time.Period;
import java.util.Map;
/**
* DATE_TRUNC function similar to PostgreSQL.
*/
public class DateTruncOperatorConversion implements SqlOperatorConversion
{
private static final Map<String, Period> TRUNC_PERIOD_MAP =
ImmutableMap.<String, Period>builder()
.put("microseconds", Period.millis(1)) // We don't support microsecond precision, so millis is fine.
.put("milliseconds", Period.millis(1))
.put("second", Period.seconds(1))
.put("minute", Period.minutes(1))
.put("hour", Period.hours(1))
.put("day", Period.days(1))
.put("week", Period.weeks(1))
.put("month", Period.months(1))
.put("quarter", Period.months(3))
.put("year", Period.years(1))
.put("decade", Period.years(10))
.put("century", Period.years(100))
.put("millennium", Period.years(1000))
.build();
private static final SqlFunction SQL_FUNCTION = OperatorConversions
.operatorBuilder("DATE_TRUNC")
.operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.TIMESTAMP)
.requiredOperands(2)
.returnType(SqlTypeName.TIMESTAMP)
.functionCategory(SqlFunctionCategory.TIMEDATE)
.build();
@Override
public SqlOperator calciteOperator()
{
return SQL_FUNCTION;
}
@Override
public DruidExpression toDruidExpression(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexNode rexNode
)
{
return OperatorConversions.convertCall(
plannerContext,
rowSignature,
rexNode,
inputExpressions -> {
final DruidExpression arg = inputExpressions.get(1);
final Expr truncTypeExpr = Parser.parse(
inputExpressions.get(0).getExpression(),
plannerContext.getExprMacroTable()
);
if (!truncTypeExpr.isLiteral()) {
throw new IAE("Operator[%s] truncType must be a literal", calciteOperator().getName());
}
final String truncType = (String) truncTypeExpr.getLiteralValue();
final Period truncPeriod = TRUNC_PERIOD_MAP.get(StringUtils.toLowerCase(truncType));
if (truncPeriod == null) {
throw new IAE("Operator[%s] cannot truncate to[%s]", calciteOperator().getName(), truncType);
}
return DruidExpression.fromFunctionCall(
"timestamp_floor",
ImmutableList.of(
arg,
DruidExpression.fromExpression(DruidExpression.stringLiteral(truncPeriod.toString())),
DruidExpression.fromExpression(DruidExpression.stringLiteral(null)),
DruidExpression.fromExpression(DruidExpression.stringLiteral(plannerContext.getTimeZone().getID()))
)
);
}
);
}
}

View File

@ -17,10 +17,13 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import com.google.common.collect.ImmutableMap;
import io.druid.query.expression.TimestampExtractExprMacro;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.avatica.util.TimeUnitRange;

View File

@ -17,10 +17,14 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.PeriodGranularity;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.expression.TimeUnits;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.avatica.util.TimeUnitRange;

View File

@ -17,8 +17,11 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;

View File

@ -17,13 +17,16 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import com.google.inject.Inject;
import io.druid.java.util.common.StringUtils;
import io.druid.math.expr.Expr;
import io.druid.query.lookup.LookupReferencesManager;
import io.druid.query.lookup.RegisteredLookupExtractionFn;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;

View File

@ -17,9 +17,13 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import com.google.common.collect.Iterables;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexCall;

View File

@ -17,8 +17,11 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;

View File

@ -17,11 +17,14 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import io.druid.java.util.common.StringUtils;
import io.druid.math.expr.Expr;
import io.druid.query.extraction.RegexDimExtractionFn;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;

View File

@ -0,0 +1,32 @@
/*
* 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.builtin;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
public class ReinterpretOperatorConversion extends CastOperatorConversion
{
@Override
public SqlOperator calciteOperator()
{
return SqlStdOperatorTable.REINTERPRET;
}
}

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.expression.builtin;
import io.druid.java.util.common.StringUtils;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
public class StrposOperatorConversion implements SqlOperatorConversion
{
private static final SqlFunction SQL_FUNCTION = OperatorConversions
.operatorBuilder("STRPOS")
.operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER)
.functionCategory(SqlFunctionCategory.STRING)
.returnType(SqlTypeName.INTEGER)
.build();
@Override
public SqlOperator calciteOperator()
{
return SQL_FUNCTION;
}
@Override
public DruidExpression toDruidExpression(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexNode rexNode
)
{
return OperatorConversions.convertCall(
plannerContext,
rowSignature,
rexNode,
druidExpressions -> DruidExpression.of(
null,
StringUtils.format(
"(%s + 1)",
DruidExpression.functionCall("strpos", druidExpressions)
)
)
);
}
}

View File

@ -17,24 +17,39 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import io.druid.java.util.common.StringUtils;
import io.druid.query.extraction.SubstringDimExtractionFn;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
public class SubstringOperatorConversion implements SqlOperatorConversion
{
private static final SqlFunction SQL_FUNCTION = OperatorConversions
.operatorBuilder("SUBSTRING")
.operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER, SqlTypeFamily.INTEGER)
.functionCategory(SqlFunctionCategory.STRING)
.returnTypeInference(ReturnTypes.ARG0)
.requiredOperands(2)
.build();
@Override
public SqlOperator calciteOperator()
{
return SqlStdOperatorTable.SUBSTRING;
return SQL_FUNCTION;
}
@Override

View File

@ -17,12 +17,15 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import com.google.common.base.Preconditions;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexCall;

View File

@ -17,11 +17,15 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import com.google.common.collect.ImmutableMap;
import io.druid.java.util.common.StringUtils;
import io.druid.query.expression.TimestampExtractExprMacro;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexCall;

View File

@ -17,11 +17,16 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.granularity.PeriodGranularity;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.ExtractionFns;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.Calcites;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;

View File

@ -17,13 +17,19 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.TimeFormatExtractionFn;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.ExtractionFns;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SimpleExtraction;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexCall;

View File

@ -17,8 +17,11 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;

View File

@ -17,8 +17,11 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;

View File

@ -17,9 +17,13 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import com.google.common.collect.Iterables;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexCall;

View File

@ -17,9 +17,12 @@
* under the License.
*/
package io.druid.sql.calcite.expression;
package io.druid.sql.calcite.expression.builtin;
import com.google.common.collect.ImmutableList;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexCall;

View File

@ -0,0 +1,91 @@
/*
* 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.builtin;
import io.druid.java.util.common.StringUtils;
import io.druid.math.expr.Expr;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
public class TruncateOperatorConversion implements SqlOperatorConversion
{
private static final SqlFunction SQL_FUNCTION = OperatorConversions
.operatorBuilder("TRUNCATE")
.operandTypes(SqlTypeFamily.NUMERIC, SqlTypeFamily.INTEGER)
.requiredOperands(1)
.returnTypeInference(ReturnTypes.ARG0)
.functionCategory(SqlFunctionCategory.NUMERIC)
.build();
@Override
public SqlFunction calciteOperator()
{
return SQL_FUNCTION;
}
@Override
public DruidExpression toDruidExpression(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexNode rexNode
)
{
return OperatorConversions.convertCall(
plannerContext,
rowSignature,
rexNode,
inputExpressions -> {
final DruidExpression arg = inputExpressions.get(0);
final Expr digitsExpr = inputExpressions.size() > 1
? inputExpressions.get(1).parse(plannerContext.getExprMacroTable())
: null;
final String factorString;
if (digitsExpr == null) {
factorString = "1";
} else if (digitsExpr.isLiteral()) {
final int digits = ((Number) digitsExpr.getLiteralValue()).intValue();
final double factor = Math.pow(10, digits);
factorString = DruidExpression.numberLiteral(factor);
} else {
factorString = StringUtils.format("pow(10,%s)", inputExpressions.get(1));
}
return DruidExpression.fromExpression(
StringUtils.format(
"(cast(cast(%s * %s,'long'),'double') / %s)",
arg.getExpression(),
factorString,
factorString
)
);
}
);
}
}

View File

@ -38,22 +38,38 @@ import org.apache.calcite.schema.SchemaPlus;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.util.ConversionUtil;
import org.apache.calcite.util.DateString;
import org.apache.calcite.util.TimeString;
import org.apache.calcite.util.TimestampString;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.Days;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.DateTimeFormatterBuilder;
import org.joda.time.format.ISODateTimeFormat;
import java.nio.charset.Charset;
import java.util.Calendar;
import java.util.Locale;
import java.util.NavigableSet;
import java.util.TimeZone;
/**
* Utility functions for Calcite.
*/
public class Calcites
{
private static final TimeZone GMT_TIME_ZONE = TimeZone.getTimeZone("GMT");
private static final DateTimes.UtcFormatter CALCITE_DATE_PARSER = DateTimes.wrapFormatter(ISODateTimeFormat.dateParser());
private static final DateTimes.UtcFormatter CALCITE_TIMESTAMP_PARSER = DateTimes.wrapFormatter(
new DateTimeFormatterBuilder()
.append(ISODateTimeFormat.dateParser())
.appendLiteral(' ')
.append(ISODateTimeFormat.timeParser())
.toFormatter()
);
private static final DateTimeFormatter CALCITE_TIME_PRINTER = DateTimeFormat.forPattern("HH:mm:ss.S");
private static final DateTimeFormatter CALCITE_DATE_PRINTER = DateTimeFormat.forPattern("yyyy-MM-dd");
private static final DateTimeFormatter CALCITE_TIMESTAMP_PRINTER = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.S");
private static final Charset DEFAULT_CHARSET = Charset.forName(ConversionUtil.NATIVE_UTF16_CHARSET_NAME);
private Calcites()
@ -174,19 +190,44 @@ public class Calcites
}
/**
* Calcite expects TIMESTAMP and DATE literals to be represented by Calendars that would have the expected
* local time fields if printed as UTC.
* Calcite expects TIMESTAMP literals to be represented by TimestampStrings in the local time zone.
*
* @param dateTime joda timestamp
* @param timeZone session time zone
*
* @return Calcite style Calendar, appropriate for literals
*/
public static Calendar jodaToCalciteCalendarLiteral(final DateTime dateTime, final DateTimeZone timeZone)
public static TimestampString jodaToCalciteTimestampString(final DateTime dateTime, final DateTimeZone timeZone)
{
final Calendar calendar = Calendar.getInstance(GMT_TIME_ZONE, Locale.ENGLISH);
calendar.setTimeInMillis(Calcites.jodaToCalciteTimestamp(dateTime, timeZone));
return calendar;
// The replaceAll is because Calcite doesn't like trailing zeroes in its fractional seconds part.
return new TimestampString(CALCITE_TIMESTAMP_PRINTER.print(dateTime.withZone(timeZone)).replaceAll("\\.?0+$", ""));
}
/**
* Calcite expects TIME literals to be represented by TimeStrings in the local time zone.
*
* @param dateTime joda timestamp
* @param timeZone session time zone
*
* @return Calcite style Calendar, appropriate for literals
*/
public static TimeString jodaToCalciteTimeString(final DateTime dateTime, final DateTimeZone timeZone)
{
// The replaceAll is because Calcite doesn't like trailing zeroes in its fractional seconds part.
return new TimeString(CALCITE_TIME_PRINTER.print(dateTime.withZone(timeZone)).replaceAll("\\.?0+$", ""));
}
/**
* Calcite expects DATE literals to be represented by DateStrings in the local time zone.
*
* @param dateTime joda timestamp
* @param timeZone session time zone
*
* @return Calcite style Calendar, appropriate for literals
*/
public static DateString jodaToCalciteDateString(final DateTime dateTime, final DateTimeZone timeZone)
{
return new DateString(CALCITE_DATE_PRINTER.print(dateTime.withZone(timeZone)));
}
/**
@ -202,11 +243,18 @@ public class Calcites
{
final SqlTypeName typeName = literal.getType().getSqlTypeName();
if (literal.getKind() != SqlKind.LITERAL || (typeName != SqlTypeName.TIMESTAMP && typeName != SqlTypeName.DATE)) {
throw new IAE("Expected TIMESTAMP or DATE literal but got[%s:%s]", literal.getKind(), typeName);
throw new IAE("Expected literal but got[%s]", literal.getKind());
}
final Calendar calendar = (Calendar) RexLiteral.value(literal);
return calciteTimestampToJoda(calendar.getTimeInMillis(), timeZone);
if (typeName == SqlTypeName.TIMESTAMP) {
final TimestampString timestampString = (TimestampString) RexLiteral.value(literal);
return CALCITE_TIMESTAMP_PARSER.parse(timestampString.toString()).withZoneRetainFields(timeZone);
} else if (typeName == SqlTypeName.DATE) {
final DateString dateString = (DateString) RexLiteral.value(literal);
return CALCITE_DATE_PARSER.parse(dateString.toString()).withZoneRetainFields(timeZone);
} else {
throw new IAE("Expected TIMESTAMP or DATE but got[%s]", typeName);
}
}
/**

View File

@ -58,4 +58,24 @@ public class DruidConformance extends SqlAbstractConformance
// For ORDER BY columnAlias (where columnAlias is a "column AS columnAlias")
return true;
}
@Override
public boolean isGroupByAlias()
{
// Disable GROUP BY columnAlias (where columnAlias is a "column AS columnAlias") since it causes ambiguity in the
// Calcite validator for queries like SELECT TRIM(x) AS x, COUNT(*) FROM druid.foo GROUP BY TRIM(x) ORDER BY TRIM(x)
return false;
}
@Override
public boolean isGroupByOrdinal()
{
return true;
}
@Override
public boolean isHavingAlias()
{
return true;
}
}

View File

@ -19,7 +19,7 @@
package io.druid.sql.calcite.planner;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.ISE;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rex.RexNode;
@ -32,34 +32,48 @@ import org.apache.calcite.sql2rel.SqlRexConvertlet;
import org.apache.calcite.sql2rel.SqlRexConvertletTable;
import org.apache.calcite.sql2rel.StandardConvertletTable;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class DruidConvertletTable implements SqlRexConvertletTable
{
private static final SqlRexConvertlet BYPASS_CONVERTLET = new SqlRexConvertlet()
{
@Override
public RexNode convertCall(SqlRexContext cx, SqlCall call)
{
return StandardConvertletTable.INSTANCE.convertCall(cx, call);
}
};
// Apply a convertlet that doesn't do anything other than a "dumb" call translation.
private static final SqlRexConvertlet BYPASS_CONVERTLET = StandardConvertletTable.INSTANCE::convertCall;
// Apply the default convertlet found in StandardConvertletTable, which may do "smart" things.
private static final SqlRexConvertlet STANDARD_CONVERTLET =
(cx, call) -> StandardConvertletTable.INSTANCE.get(call).convertCall(cx, call);
private static final List<SqlOperator> CURRENT_TIME_CONVERTLET_OPERATORS =
ImmutableList.<SqlOperator>builder()
.add(SqlStdOperatorTable.CURRENT_TIMESTAMP)
.add(SqlStdOperatorTable.CURRENT_TIME)
.add(SqlStdOperatorTable.CURRENT_DATE)
.add(SqlStdOperatorTable.LOCALTIMESTAMP)
.add(SqlStdOperatorTable.LOCALTIME)
.build();
// Operators we don't have standard conversions for, but which can be converted into ones that do by
// Calcite's StandardConvertletTable.
private static final List<SqlOperator> STANDARD_CONVERTLET_OPERATORS =
ImmutableList.<SqlOperator>builder()
.add(SqlStdOperatorTable.ROW)
.add(SqlStdOperatorTable.NOT_IN)
.add(SqlStdOperatorTable.NOT_LIKE)
.add(SqlStdOperatorTable.BETWEEN)
.add(SqlStdOperatorTable.NOT_BETWEEN)
.add(SqlStdOperatorTable.SYMMETRIC_BETWEEN)
.add(SqlStdOperatorTable.SYMMETRIC_NOT_BETWEEN)
.add(SqlStdOperatorTable.ITEM)
.build();
private final PlannerContext plannerContext;
private final Map<SqlOperator, SqlRexConvertlet> table;
public DruidConvertletTable(final PlannerContext plannerContext)
{
this.plannerContext = plannerContext;
final SqlRexConvertlet currentTimestampAndFriendsConvertlet = new CurrentTimestampAndFriendsConvertlet();
this.table = ImmutableMap.<SqlOperator, SqlRexConvertlet>builder()
.put(SqlStdOperatorTable.CURRENT_TIMESTAMP, currentTimestampAndFriendsConvertlet)
.put(SqlStdOperatorTable.CURRENT_TIME, currentTimestampAndFriendsConvertlet)
.put(SqlStdOperatorTable.CURRENT_DATE, currentTimestampAndFriendsConvertlet)
.put(SqlStdOperatorTable.LOCALTIMESTAMP, currentTimestampAndFriendsConvertlet)
.put(SqlStdOperatorTable.LOCALTIME, currentTimestampAndFriendsConvertlet)
.build();
this.table = createConvertletMap(plannerContext);
}
@Override
@ -75,25 +89,61 @@ public class DruidConvertletTable implements SqlRexConvertletTable
}
}
private class CurrentTimestampAndFriendsConvertlet implements SqlRexConvertlet
public static List<SqlOperator> knownOperators()
{
final ArrayList<SqlOperator> retVal = new ArrayList<>(
CURRENT_TIME_CONVERTLET_OPERATORS.size() + STANDARD_CONVERTLET_OPERATORS.size()
);
retVal.addAll(CURRENT_TIME_CONVERTLET_OPERATORS);
retVal.addAll(STANDARD_CONVERTLET_OPERATORS);
return retVal;
}
private static Map<SqlOperator, SqlRexConvertlet> createConvertletMap(final PlannerContext plannerContext)
{
final SqlRexConvertlet currentTimestampAndFriends = new CurrentTimestampAndFriendsConvertlet(plannerContext);
final Map<SqlOperator, SqlRexConvertlet> table = new HashMap<>();
for (SqlOperator operator : CURRENT_TIME_CONVERTLET_OPERATORS) {
table.put(operator, currentTimestampAndFriends);
}
for (SqlOperator operator : STANDARD_CONVERTLET_OPERATORS) {
table.put(operator, STANDARD_CONVERTLET);
}
return table;
}
private static class CurrentTimestampAndFriendsConvertlet implements SqlRexConvertlet
{
private final PlannerContext plannerContext;
public CurrentTimestampAndFriendsConvertlet(final PlannerContext plannerContext)
{
this.plannerContext = plannerContext;
}
@Override
public RexNode convertCall(final SqlRexContext cx, final SqlCall call)
{
final SqlOperator operator = call.getOperator();
if (operator == SqlStdOperatorTable.CURRENT_TIMESTAMP || operator == SqlStdOperatorTable.LOCALTIMESTAMP) {
if (operator.equals(SqlStdOperatorTable.CURRENT_TIMESTAMP)
|| operator.equals(SqlStdOperatorTable.LOCALTIMESTAMP)) {
return cx.getRexBuilder().makeTimestampLiteral(
Calcites.jodaToCalciteCalendarLiteral(plannerContext.getLocalNow(), plannerContext.getTimeZone()),
Calcites.jodaToCalciteTimestampString(plannerContext.getLocalNow(), plannerContext.getTimeZone()),
RelDataType.PRECISION_NOT_SPECIFIED
);
} else if (operator == SqlStdOperatorTable.CURRENT_TIME || operator == SqlStdOperatorTable.LOCALTIME) {
} else if (operator.equals(SqlStdOperatorTable.CURRENT_TIME) || operator.equals(SqlStdOperatorTable.LOCALTIME)) {
return cx.getRexBuilder().makeTimeLiteral(
Calcites.jodaToCalciteCalendarLiteral(plannerContext.getLocalNow(), plannerContext.getTimeZone()),
Calcites.jodaToCalciteTimeString(plannerContext.getLocalNow(), plannerContext.getTimeZone()),
RelDataType.PRECISION_NOT_SPECIFIED
);
} else if (operator == SqlStdOperatorTable.CURRENT_DATE) {
} else if (operator.equals(SqlStdOperatorTable.CURRENT_DATE)) {
return cx.getRexBuilder().makeDateLiteral(
Calcites.jodaToCalciteCalendarLiteral(
Calcites.jodaToCalciteDateString(
plannerContext.getLocalNow().hourOfDay().roundFloorCopy(),
plannerContext.getTimeZone()
)

View File

@ -19,12 +19,48 @@
package io.druid.sql.calcite.planner;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Maps;
import com.google.inject.Inject;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.aggregation.builtin.ApproxCountDistinctSqlAggregator;
import io.druid.sql.calcite.aggregation.builtin.AvgSqlAggregator;
import io.druid.sql.calcite.aggregation.builtin.CountSqlAggregator;
import io.druid.sql.calcite.aggregation.builtin.MaxSqlAggregator;
import io.druid.sql.calcite.aggregation.builtin.MinSqlAggregator;
import io.druid.sql.calcite.aggregation.builtin.SumSqlAggregator;
import io.druid.sql.calcite.aggregation.builtin.SumZeroSqlAggregator;
import io.druid.sql.calcite.expression.AliasedOperatorConversion;
import io.druid.sql.calcite.expression.BinaryOperatorConversion;
import io.druid.sql.calcite.expression.DirectOperatorConversion;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.expression.UnaryPrefixOperatorConversion;
import io.druid.sql.calcite.expression.UnarySuffixOperatorConversion;
import io.druid.sql.calcite.expression.builtin.BTrimOperatorConversion;
import io.druid.sql.calcite.expression.builtin.CastOperatorConversion;
import io.druid.sql.calcite.expression.builtin.CeilOperatorConversion;
import io.druid.sql.calcite.expression.builtin.DateTruncOperatorConversion;
import io.druid.sql.calcite.expression.builtin.ExtractOperatorConversion;
import io.druid.sql.calcite.expression.builtin.FloorOperatorConversion;
import io.druid.sql.calcite.expression.builtin.LTrimOperatorConversion;
import io.druid.sql.calcite.expression.builtin.MillisToTimestampOperatorConversion;
import io.druid.sql.calcite.expression.builtin.RTrimOperatorConversion;
import io.druid.sql.calcite.expression.builtin.RegexpExtractOperatorConversion;
import io.druid.sql.calcite.expression.builtin.ReinterpretOperatorConversion;
import io.druid.sql.calcite.expression.builtin.StrposOperatorConversion;
import io.druid.sql.calcite.expression.builtin.SubstringOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TimeArithmeticOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TimeExtractOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TimeFloorOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TimeFormatOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TimeParseOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TimeShiftOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TimestampToMillisOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TrimOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TruncateOperatorConversion;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlIdentifier;
@ -36,42 +72,152 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;
public class DruidOperatorTable implements SqlOperatorTable
{
private static final SqlStdOperatorTable STANDARD_TABLE = SqlStdOperatorTable.instance();
private static final List<SqlAggregator> STANDARD_AGGREGATORS =
ImmutableList.<SqlAggregator>builder()
.add(new ApproxCountDistinctSqlAggregator())
.add(new AvgSqlAggregator())
.add(new CountSqlAggregator())
.add(new MinSqlAggregator())
.add(new MaxSqlAggregator())
.add(new SumSqlAggregator())
.add(new SumZeroSqlAggregator())
.build();
private final Map<String, SqlAggregator> aggregators;
private final Map<String, SqlOperatorConversion> extractionOperators;
// STRLEN has so many aliases.
private static final SqlOperatorConversion CHARACTER_LENGTH_CONVERSION = new DirectOperatorConversion(
SqlStdOperatorTable.CHARACTER_LENGTH,
"strlen"
);
private static final List<SqlOperatorConversion> STANDARD_OPERATOR_CONVERSIONS =
ImmutableList.<SqlOperatorConversion>builder()
.add(new DirectOperatorConversion(SqlStdOperatorTable.ABS, "abs"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.CASE, "case_searched"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.CHAR_LENGTH, "strlen"))
.add(CHARACTER_LENGTH_CONVERSION)
.add(new AliasedOperatorConversion(CHARACTER_LENGTH_CONVERSION, "LENGTH"))
.add(new AliasedOperatorConversion(CHARACTER_LENGTH_CONVERSION, "STRLEN"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.CONCAT, "concat"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.EXP, "exp"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.DIVIDE_INTEGER, "div"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.LIKE, "like"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.LN, "log"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.LOWER, "lower"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.LOG10, "log10"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.POWER, "pow"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.REPLACE, "replace"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.SQRT, "sqrt"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.UPPER, "upper"))
.add(new UnaryPrefixOperatorConversion(SqlStdOperatorTable.NOT, "!"))
.add(new UnaryPrefixOperatorConversion(SqlStdOperatorTable.UNARY_MINUS, "-"))
.add(new UnarySuffixOperatorConversion(SqlStdOperatorTable.IS_NULL, "== ''"))
.add(new UnarySuffixOperatorConversion(SqlStdOperatorTable.IS_NOT_NULL, "!= ''"))
.add(new UnarySuffixOperatorConversion(SqlStdOperatorTable.IS_FALSE, "<= 0")) // Matches Evals.asBoolean
.add(new UnarySuffixOperatorConversion(SqlStdOperatorTable.IS_NOT_TRUE, "<= 0")) // Matches Evals.asBoolean
.add(new UnarySuffixOperatorConversion(SqlStdOperatorTable.IS_TRUE, "> 0")) // Matches Evals.asBoolean
.add(new UnarySuffixOperatorConversion(SqlStdOperatorTable.IS_NOT_FALSE, "> 0")) // Matches Evals.asBoolean
.add(new BinaryOperatorConversion(SqlStdOperatorTable.MULTIPLY, "*"))
.add(new BinaryOperatorConversion(SqlStdOperatorTable.MOD, "%"))
.add(new BinaryOperatorConversion(SqlStdOperatorTable.DIVIDE, "/"))
.add(new BinaryOperatorConversion(SqlStdOperatorTable.PLUS, "+"))
.add(new BinaryOperatorConversion(SqlStdOperatorTable.MINUS, "-"))
.add(new BinaryOperatorConversion(SqlStdOperatorTable.EQUALS, "=="))
.add(new BinaryOperatorConversion(SqlStdOperatorTable.NOT_EQUALS, "!="))
.add(new BinaryOperatorConversion(SqlStdOperatorTable.GREATER_THAN, ">"))
.add(new BinaryOperatorConversion(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, ">="))
.add(new BinaryOperatorConversion(SqlStdOperatorTable.LESS_THAN, "<"))
.add(new BinaryOperatorConversion(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, "<="))
.add(new BinaryOperatorConversion(SqlStdOperatorTable.AND, "&&"))
.add(new BinaryOperatorConversion(SqlStdOperatorTable.OR, "||"))
.add(new CastOperatorConversion())
.add(new CeilOperatorConversion())
.add(new DateTruncOperatorConversion())
.add(new ExtractOperatorConversion())
.add(new FloorOperatorConversion())
.add(new MillisToTimestampOperatorConversion())
.add(new ReinterpretOperatorConversion())
.add(new RegexpExtractOperatorConversion())
.add(new StrposOperatorConversion())
.add(new SubstringOperatorConversion())
.add(new AliasedOperatorConversion(new SubstringOperatorConversion(), "SUBSTR"))
.add(new TimeArithmeticOperatorConversion.TimeMinusIntervalOperatorConversion())
.add(new TimeArithmeticOperatorConversion.TimePlusIntervalOperatorConversion())
.add(new TimeExtractOperatorConversion())
.add(new TimeFloorOperatorConversion())
.add(new TimeFormatOperatorConversion())
.add(new TimeParseOperatorConversion())
.add(new TimeShiftOperatorConversion())
.add(new TimestampToMillisOperatorConversion())
.add(new TruncateOperatorConversion())
.add(new TrimOperatorConversion())
.add(new BTrimOperatorConversion())
.add(new LTrimOperatorConversion())
.add(new RTrimOperatorConversion())
.add(new AliasedOperatorConversion(new TruncateOperatorConversion(), "TRUNC"))
.build();
// Operators that have no conversion, but are handled in the convertlet table, so they still need to exist.
private static final Map<OperatorKey, SqlOperator> CONVERTLET_OPERATORS =
DruidConvertletTable.knownOperators()
.stream()
.collect(Collectors.toMap(OperatorKey::of, Function.identity()));
private final Map<OperatorKey, SqlAggregator> aggregators;
private final Map<OperatorKey, SqlOperatorConversion> operatorConversions;
@Inject
public DruidOperatorTable(
final Set<SqlAggregator> aggregators,
final Set<SqlOperatorConversion> extractionOperators
final Set<SqlOperatorConversion> operatorConversions
)
{
this.aggregators = Maps.newHashMap();
this.extractionOperators = Maps.newHashMap();
this.operatorConversions = Maps.newHashMap();
for (SqlAggregator aggregator : aggregators) {
final String lcname = StringUtils.toLowerCase(aggregator.calciteFunction().getName());
if (this.aggregators.put(lcname, aggregator) != null) {
throw new ISE("Cannot have two operators with name[%s]", lcname);
final OperatorKey operatorKey = OperatorKey.of(aggregator.calciteFunction());
if (this.aggregators.put(operatorKey, aggregator) != null) {
throw new ISE("Cannot have two operators with key[%s]", operatorKey);
}
}
for (SqlOperatorConversion extractionFunction : extractionOperators) {
final String lcname = StringUtils.toLowerCase(extractionFunction.calciteOperator().getName());
if (this.aggregators.containsKey(lcname) || this.extractionOperators.put(lcname, extractionFunction) != null) {
throw new ISE("Cannot have two operators with name[%s]", lcname);
for (SqlAggregator aggregator : STANDARD_AGGREGATORS) {
final OperatorKey operatorKey = OperatorKey.of(aggregator.calciteFunction());
// Don't complain if the name already exists; we allow standard operators to be overridden.
this.aggregators.putIfAbsent(operatorKey, aggregator);
}
for (SqlOperatorConversion operatorConversion : operatorConversions) {
final OperatorKey operatorKey = OperatorKey.of(operatorConversion.calciteOperator());
if (this.aggregators.containsKey(operatorKey)
|| this.operatorConversions.put(operatorKey, operatorConversion) != null) {
throw new ISE("Cannot have two operators with key[%s]", operatorKey);
}
}
for (SqlOperatorConversion operatorConversion : STANDARD_OPERATOR_CONVERSIONS) {
final OperatorKey operatorKey = OperatorKey.of(operatorConversion.calciteOperator());
// Don't complain if the name already exists; we allow standard operators to be overridden.
if (this.aggregators.containsKey(operatorKey)) {
continue;
}
this.operatorConversions.putIfAbsent(operatorKey, operatorConversion);
}
}
public SqlAggregator lookupAggregator(final SqlAggFunction aggFunction)
{
final SqlAggregator sqlAggregator = aggregators.get(StringUtils.toLowerCase(aggFunction.getName()));
final SqlAggregator sqlAggregator = aggregators.get(OperatorKey.of(aggFunction));
if (sqlAggregator != null && sqlAggregator.calciteFunction().equals(aggFunction)) {
return sqlAggregator;
} else {
@ -81,7 +227,7 @@ public class DruidOperatorTable implements SqlOperatorTable
public SqlOperatorConversion lookupOperatorConversion(final SqlOperator operator)
{
final SqlOperatorConversion operatorConversion = extractionOperators.get(StringUtils.toLowerCase(operator.getName()));
final SqlOperatorConversion operatorConversion = operatorConversions.get(OperatorKey.of(operator));
if (operatorConversion != null && operatorConversion.calciteOperator().equals(operator)) {
return operatorConversion;
} else {
@ -97,19 +243,26 @@ public class DruidOperatorTable implements SqlOperatorTable
final List<SqlOperator> operatorList
)
{
if (opName.names.size() == 1 && syntax == SqlSyntax.FUNCTION) {
final SqlAggregator aggregator = aggregators.get(StringUtils.toLowerCase(opName.getSimple()));
if (aggregator != null) {
operatorList.add(aggregator.calciteFunction());
}
final SqlOperatorConversion extractionFunction = extractionOperators.get(StringUtils.toLowerCase(opName.getSimple()));
if (extractionFunction != null) {
operatorList.add(extractionFunction.calciteOperator());
}
if (opName.names.size() != 1) {
return;
}
STANDARD_TABLE.lookupOperatorOverloads(opName, category, syntax, operatorList);
final OperatorKey operatorKey = OperatorKey.of(opName.getSimple(), syntax);
final SqlAggregator aggregator = aggregators.get(operatorKey);
if (aggregator != null) {
operatorList.add(aggregator.calciteFunction());
}
final SqlOperatorConversion operatorConversion = operatorConversions.get(operatorKey);
if (operatorConversion != null) {
operatorList.add(operatorConversion.calciteOperator());
}
final SqlOperator convertletOperator = CONVERTLET_OPERATORS.get(operatorKey);
if (convertletOperator != null) {
operatorList.add(convertletOperator);
}
}
@Override
@ -119,10 +272,81 @@ public class DruidOperatorTable implements SqlOperatorTable
for (SqlAggregator aggregator : aggregators.values()) {
retVal.add(aggregator.calciteFunction());
}
for (SqlOperatorConversion extractionFunction : extractionOperators.values()) {
retVal.add(extractionFunction.calciteOperator());
for (SqlOperatorConversion operatorConversion : operatorConversions.values()) {
retVal.add(operatorConversion.calciteOperator());
}
retVal.addAll(STANDARD_TABLE.getOperatorList());
retVal.addAll(DruidConvertletTable.knownOperators());
return retVal;
}
private static SqlSyntax normalizeSyntax(final SqlSyntax syntax)
{
// Treat anything other than prefix/suffix/binary syntax as function syntax.
if (syntax == SqlSyntax.PREFIX || syntax == SqlSyntax.BINARY || syntax == SqlSyntax.POSTFIX) {
return syntax;
} else {
return SqlSyntax.FUNCTION;
}
}
private static class OperatorKey
{
private final String name;
private final SqlSyntax syntax;
public OperatorKey(final String name, final SqlSyntax syntax)
{
this.name = StringUtils.toLowerCase(Preconditions.checkNotNull(name, "name"));
this.syntax = normalizeSyntax(Preconditions.checkNotNull(syntax, "syntax"));
}
public static OperatorKey of(final String name, final SqlSyntax syntax)
{
return new OperatorKey(name, syntax);
}
public static OperatorKey of(final SqlOperator operator)
{
return new OperatorKey(operator.getName(), operator.getSyntax());
}
public String getName()
{
return name;
}
public SqlSyntax getSyntax()
{
return syntax;
}
@Override
public boolean equals(final Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final OperatorKey that = (OperatorKey) o;
return Objects.equals(name, that.name) &&
syntax == that.syntax;
}
@Override
public int hashCode()
{
return Objects.hash(name, syntax);
}
@Override
public String toString()
{
return "OperatorKey{" +
"name='" + name + '\'' +
", syntax=" + syntax +
'}';
}
}
}

View File

@ -19,7 +19,6 @@
package io.druid.sql.calcite.planner;
import com.google.common.collect.ImmutableMap;
import io.druid.java.util.common.DateTimes;
import io.druid.math.expr.Expr;
import io.druid.math.expr.ExprEval;
@ -28,6 +27,7 @@ import io.druid.math.expr.Parser;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexExecutor;
import org.apache.calcite.rex.RexNode;
@ -70,25 +70,48 @@ public class DruidRexExecutor implements RexExecutor
} else {
final SqlTypeName sqlTypeName = constExp.getType().getSqlTypeName();
final Expr expr = Parser.parse(druidExpression.getExpression(), plannerContext.getExprMacroTable());
final ExprEval exprResult = expr.eval(Parser.withMap(ImmutableMap.of()));
final Object literalValue;
final ExprEval exprResult = expr.eval(
name -> {
// Sanity check. Bindings should not be used for a constant expression.
throw new UnsupportedOperationException();
}
);
final RexNode literal;
if (sqlTypeName == SqlTypeName.BOOLEAN) {
literalValue = exprResult.asBoolean();
} else if (sqlTypeName == SqlTypeName.DATE || sqlTypeName == SqlTypeName.TIMESTAMP) {
literalValue = Calcites.jodaToCalciteCalendarLiteral(
DateTimes.utc(exprResult.asLong()),
plannerContext.getTimeZone()
literal = rexBuilder.makeLiteral(exprResult.asBoolean(), constExp.getType(), true);
} else if (sqlTypeName == SqlTypeName.DATE) {
literal = rexBuilder.makeDateLiteral(
Calcites.jodaToCalciteDateString(
DateTimes.utc(exprResult.asLong()),
plannerContext.getTimeZone()
)
);
} else if (sqlTypeName == SqlTypeName.TIMESTAMP) {
literal = rexBuilder.makeTimestampLiteral(
Calcites.jodaToCalciteTimestampString(
DateTimes.utc(exprResult.asLong()),
plannerContext.getTimeZone()
),
RelDataType.PRECISION_NOT_SPECIFIED
);
} else if (SqlTypeName.NUMERIC_TYPES.contains(sqlTypeName)) {
literalValue = exprResult.type() == ExprType.LONG
? new BigDecimal(exprResult.asLong())
: new BigDecimal(exprResult.asDouble());
final BigDecimal bigDecimal;
if (exprResult.type() == ExprType.LONG) {
bigDecimal = BigDecimal.valueOf(exprResult.asLong());
} else {
bigDecimal = BigDecimal.valueOf(exprResult.asDouble());
}
literal = rexBuilder.makeLiteral(bigDecimal, constExp.getType(), true);
} else {
literalValue = exprResult.value();
literal = rexBuilder.makeLiteral(exprResult.value(), constExp.getType(), true);
}
reducedValues.add(rexBuilder.makeLiteral(literalValue, constExp.getType(), true));
reducedValues.add(literal);
}
}
}

View File

@ -99,6 +99,37 @@ public class DruidTypeSystem implements RelDataTypeSystem
}
}
@Override
public RelDataType deriveAvgAggType(
final RelDataTypeFactory typeFactory,
final RelDataType argumentType
)
{
return RelDataTypeSystem.DEFAULT.deriveAvgAggType(typeFactory, argumentType);
}
@Override
public RelDataType deriveCovarType(
final RelDataTypeFactory typeFactory,
final RelDataType arg0Type,
final RelDataType arg1Type
)
{
return RelDataTypeSystem.DEFAULT.deriveCovarType(typeFactory, arg0Type, arg1Type);
}
@Override
public RelDataType deriveFractionalRankType(final RelDataTypeFactory typeFactory)
{
return RelDataTypeSystem.DEFAULT.deriveFractionalRankType(typeFactory);
}
@Override
public RelDataType deriveRankType(final RelDataTypeFactory typeFactory)
{
return RelDataTypeSystem.DEFAULT.deriveRankType(typeFactory);
}
@Override
public boolean isSchemaCaseSensitive()
{

View File

@ -31,16 +31,21 @@ import io.druid.sql.calcite.rel.QueryMaker;
import io.druid.sql.calcite.schema.DruidSchema;
import org.apache.calcite.avatica.util.Casing;
import org.apache.calcite.avatica.util.Quoting;
import org.apache.calcite.config.CalciteConnectionConfig;
import org.apache.calcite.config.CalciteConnectionConfigImpl;
import org.apache.calcite.plan.Context;
import org.apache.calcite.plan.Contexts;
import org.apache.calcite.plan.ConventionTraitDef;
import org.apache.calcite.rel.RelCollationTraitDef;
import org.apache.calcite.rel.type.RelDataTypeSystem;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.calcite.sql.parser.SqlParser;
import org.apache.calcite.sql.validate.SqlConformance;
import org.apache.calcite.sql2rel.SqlToRelConverter;
import org.apache.calcite.tools.FrameworkConfig;
import org.apache.calcite.tools.Frameworks;
import java.util.Map;
import java.util.Properties;
public class PlannerFactory
{
@ -99,7 +104,13 @@ public class PlannerFactory
queryContext
);
final QueryMaker queryMaker = new QueryMaker(queryLifecycleFactory, plannerContext, jsonMapper);
final SqlToRelConverter.Config sqlToRelConverterConfig = SqlToRelConverter
.configBuilder()
.withExpand(false)
.withDecorrelationEnabled(false)
.withTrimUnusedFields(false)
.withInSubQueryThreshold(Integer.MAX_VALUE)
.build();
final FrameworkConfig frameworkConfig = Frameworks
.newConfigBuilder()
.parserConfig(PARSER_CONFIG)
@ -109,9 +120,32 @@ public class PlannerFactory
.programs(Rules.programs(plannerContext, queryMaker))
.executor(new DruidRexExecutor(plannerContext))
.context(Contexts.EMPTY_CONTEXT)
.typeSystem(RelDataTypeSystem.DEFAULT)
.defaultSchema(rootSchema.getSubSchema(DruidSchema.NAME))
.typeSystem(DruidTypeSystem.INSTANCE)
.defaultSchema(rootSchema.getSubSchema(DruidSchema.NAME))
.sqlToRelConverterConfig(sqlToRelConverterConfig)
.context(new Context()
{
@Override
@SuppressWarnings("unchecked")
public <C> C unwrap(final Class<C> aClass)
{
if (aClass.equals(CalciteConnectionConfig.class)) {
// This seems to be the best way to provide our own SqlConformance instance. Otherwise, Calcite's
// validator will not respect it.
final Properties props = new Properties();
return (C) new CalciteConnectionConfigImpl(props)
{
@Override
public SqlConformance conformance()
{
return DruidConformance.instance();
}
};
} else {
return null;
}
}
})
.build();
return new DruidPlanner(

View File

@ -29,8 +29,15 @@ import io.druid.sql.calcite.rule.DruidSemiJoinRule;
import io.druid.sql.calcite.rule.DruidTableScanRule;
import io.druid.sql.calcite.rule.SortCollapseRule;
import org.apache.calcite.interpreter.Bindables;
import org.apache.calcite.plan.RelOptLattice;
import org.apache.calcite.plan.RelOptMaterialization;
import org.apache.calcite.plan.RelOptPlanner;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.plan.volcano.AbstractConverter;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.core.RelFactories;
import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
import org.apache.calcite.rel.rules.AggregateJoinTransposeRule;
import org.apache.calcite.rel.rules.AggregateProjectMergeRule;
@ -59,13 +66,17 @@ 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.SubQueryRemoveRule;
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 org.apache.calcite.sql2rel.RelDecorrelator;
import org.apache.calcite.sql2rel.RelFieldTrimmer;
import org.apache.calcite.tools.Program;
import org.apache.calcite.tools.Programs;
import org.apache.calcite.tools.RelBuilder;
import java.util.List;
@ -153,6 +164,13 @@ public class Rules
FilterMergeRule.INSTANCE
);
private static final List<RelOptRule> SUB_QUERY_REMOVE_RULES =
ImmutableList.of(
SubQueryRemoveRule.PROJECT,
SubQueryRemoveRule.FILTER,
SubQueryRemoveRule.JOIN
);
private Rules()
{
// No instantiation.
@ -160,9 +178,14 @@ public class Rules
public static List<Program> programs(final PlannerContext plannerContext, final QueryMaker queryMaker)
{
final Program hepProgram =
Programs.sequence(
Programs.subQuery(DefaultRelMetadataProvider.INSTANCE),
new DecorrelateAndTrimFieldsProgram()
);
return ImmutableList.of(
Programs.ofRules(druidConventionRuleSet(plannerContext, queryMaker)),
Programs.ofRules(bindableConventionRuleSet(plannerContext, queryMaker))
Programs.sequence(hepProgram, Programs.ofRules(druidConventionRuleSet(plannerContext, queryMaker))),
Programs.sequence(hepProgram, Programs.ofRules(bindableConventionRuleSet(plannerContext, queryMaker)))
);
}
@ -202,6 +225,7 @@ public class Rules
rules.addAll(CONSTANT_REDUCTION_RULES);
rules.addAll(VOLCANO_ABSTRACT_RULES);
rules.addAll(RELOPTUTIL_ABSTRACT_RULES);
rules.addAll(SUB_QUERY_REMOVE_RULES);
if (!plannerConfig.isUseApproximateCountDistinct()) {
// We'll need this to expand COUNT DISTINCTs.
@ -226,4 +250,23 @@ public class Rules
return rules.build();
}
// Based on Calcite's Programs.DecorrelateProgram and Programs.TrimFieldsProgram, which are private and only
// accessible through Programs.standard (which we don't want, since it also adds Enumerable rules).
private static class DecorrelateAndTrimFieldsProgram implements Program
{
@Override
public RelNode run(
RelOptPlanner planner,
RelNode rel,
RelTraitSet requiredOutputTraits,
List<RelOptMaterialization> materializations,
List<RelOptLattice> lattices
)
{
final RelNode decorrelatedRel = RelDecorrelator.decorrelateQuery(rel);
final RelBuilder relBuilder = RelFactories.LOGICAL_BUILDER.create(decorrelatedRel.getCluster(), null);
return new RelFieldTrimmer(null, relBuilder).trim(decorrelatedRel);
}
}
}

View File

@ -19,31 +19,11 @@
package io.druid.sql.calcite.rule;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.math.expr.ExprMacroTable;
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.FloatMaxAggregatorFactory;
import io.druid.query.aggregation.FloatMinAggregatorFactory;
import io.druid.query.aggregation.FloatSumAggregatorFactory;
import io.druid.query.aggregation.LongMaxAggregatorFactory;
import io.druid.query.aggregation.LongMinAggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.query.aggregation.post.ArithmeticPostAggregator;
import io.druid.query.aggregation.post.FieldAccessPostAggregator;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.aggregation.FilteredAggregatorFactory;
import io.druid.query.filter.DimFilter;
import io.druid.segment.column.ValueType;
import io.druid.sql.calcite.aggregation.Aggregation;
import io.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.filtration.Filtration;
import io.druid.sql.calcite.planner.PlannerContext;
@ -53,16 +33,15 @@ import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.type.SqlTypeName;
import java.util.ArrayList;
import java.util.List;
import java.util.function.Function;
import java.util.Set;
import java.util.stream.Collectors;
public class GroupByRules
{
private static final ApproxCountDistinctSqlAggregator APPROX_COUNT_DISTINCT = new ApproxCountDistinctSqlAggregator();
private GroupByRules()
{
// No instantiation.
@ -95,258 +74,94 @@ public class GroupByRules
}
final RexNode expression = project.getChildExps().get(call.filterArg);
filter = Expressions.toFilter(plannerContext, sourceRowSignature, expression);
if (filter == null) {
final DimFilter nonOptimizedFilter = Expressions.toFilter(plannerContext, sourceRowSignature, expression);
if (nonOptimizedFilter == null) {
return null;
} else {
filter = Filtration.create(nonOptimizedFilter).optimizeFilterOnly(sourceRowSignature).getDimFilter();
}
} else {
filter = null;
}
if (kind == SqlKind.COUNT && call.getArgList().isEmpty()) {
// COUNT(*)
return Aggregation.create(new CountAggregatorFactory(name)).filter(makeFilter(filter, sourceRowSignature));
} else if (call.isDistinct()) {
// AGG(DISTINCT x)
if (kind == SqlKind.COUNT && plannerContext.getPlannerConfig().isUseApproximateCountDistinct()) {
// Approximate COUNT(DISTINCT x)
return APPROX_COUNT_DISTINCT.toDruidAggregation(
name,
sourceRowSignature,
plannerContext,
existingAggregations,
project,
call,
makeFilter(filter, sourceRowSignature)
);
} else {
// Exact COUNT(DISTINCT x), or some non-COUNT aggregator.
return null;
}
} else if (kind == SqlKind.COUNT
|| kind == SqlKind.SUM
|| kind == SqlKind.SUM0
|| kind == SqlKind.MIN
|| kind == SqlKind.MAX
|| kind == SqlKind.AVG) {
// Built-in agg, not distinct, not COUNT(*)
final RexNode rexNode = Expressions.fromFieldAccess(
sourceRowSignature,
project,
Iterables.getOnlyElement(call.getArgList())
);
final SqlAggregator sqlAggregator = plannerContext.getOperatorTable()
.lookupAggregator(call.getAggregation());
final DruidExpression input = toDruidExpressionForAggregator(plannerContext, sourceRowSignature, rexNode);
if (input == null) {
return null;
}
if (kind == SqlKind.COUNT) {
// COUNT(x) should count all non-null values of x.
if (rexNode.getType().isNullable()) {
final DimFilter nonNullFilter = Expressions.toFilter(
plannerContext,
sourceRowSignature,
rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, ImmutableList.of(rexNode))
);
if (nonNullFilter == null) {
// Don't expect this to happen.
throw new ISE("Could not create not-null filter for rexNode[%s]", rexNode);
}
return Aggregation.create(new CountAggregatorFactory(name)).filter(
makeFilter(
filter == null ? nonNullFilter : new AndDimFilter(ImmutableList.of(filter, nonNullFilter)),
sourceRowSignature
)
);
} else {
return Aggregation.create(new CountAggregatorFactory(name)).filter(makeFilter(filter, sourceRowSignature));
}
} else {
// Built-in aggregator that is not COUNT.
final Aggregation retVal;
final ValueType aggregationType;
if (SqlTypeName.INT_TYPES.contains(outputType)
|| SqlTypeName.TIMESTAMP == outputType
|| SqlTypeName.DATE == outputType) {
aggregationType = ValueType.LONG;
} else if (SqlTypeName.FLOAT == outputType) {
aggregationType = ValueType.FLOAT;
} else if (SqlTypeName.FRACTIONAL_TYPES.contains(outputType)) {
aggregationType = ValueType.DOUBLE;
} else {
throw new ISE(
"Cannot determine aggregation type for SQL operator[%s] type[%s]",
call.getAggregation().getName(),
outputType
);
}
final String fieldName;
final String expression;
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
if (input.isDirectColumnAccess()) {
fieldName = input.getDirectColumn();
expression = null;
} else {
fieldName = null;
expression = input.getExpression();
}
if (kind == SqlKind.SUM || kind == SqlKind.SUM0) {
retVal = Aggregation.create(
createSumAggregatorFactory(aggregationType, name, fieldName, expression, macroTable)
);
} else if (kind == SqlKind.MIN) {
retVal = Aggregation.create(
createMinAggregatorFactory(aggregationType, name, fieldName, expression, macroTable)
);
} else if (kind == SqlKind.MAX) {
retVal = Aggregation.create(
createMaxAggregatorFactory(aggregationType, name, fieldName, expression, macroTable)
);
} else if (kind == SqlKind.AVG) {
final String sumName = StringUtils.format("%s:sum", name);
final String countName = StringUtils.format("%s:count", name);
final AggregatorFactory sum = createSumAggregatorFactory(
aggregationType,
sumName,
fieldName,
expression,
macroTable
);
final AggregatorFactory count = new CountAggregatorFactory(countName);
retVal = Aggregation.create(
ImmutableList.of(sum, count),
new ArithmeticPostAggregator(
name,
"quotient",
ImmutableList.of(
new FieldAccessPostAggregator(null, sumName),
new FieldAccessPostAggregator(null, countName)
)
)
);
} else {
// Not reached.
throw new ISE("WTF?! Kind[%s] got into the built-in aggregator path somehow?!", kind);
}
return retVal.filter(makeFilter(filter, sourceRowSignature));
}
} else {
// Not a built-in aggregator, check operator table.
final SqlAggregator sqlAggregator = plannerContext.getOperatorTable()
.lookupAggregator(call.getAggregation());
if (sqlAggregator != null) {
return sqlAggregator.toDruidAggregation(
name,
sourceRowSignature,
plannerContext,
existingAggregations,
project,
call,
makeFilter(filter, sourceRowSignature)
);
} else {
return null;
}
}
}
private static DruidExpression toDruidExpressionForAggregator(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexNode rexNode
)
{
final DruidExpression druidExpression = Expressions.toDruidExpression(plannerContext, rowSignature, rexNode);
if (druidExpression == null) {
if (sqlAggregator == null) {
return null;
}
if (druidExpression.isSimpleExtraction() &&
(!druidExpression.isDirectColumnAccess()
|| rowSignature.getColumnType(druidExpression.getDirectColumn()) == ValueType.STRING)) {
// Aggregators are unable to implicitly cast strings to numbers. So remove the simple extraction in this case.
return druidExpression.map(simpleExtraction -> null, Function.identity());
// Compute existingAggregations for SqlAggregator impls that want it.
final List<Aggregation> existingAggregationsWithSameFilter = new ArrayList<>();
for (Aggregation existingAggregation : existingAggregations) {
if (filter == null) {
final boolean doesMatch = existingAggregation.getAggregatorFactories().stream().allMatch(
factory -> !(factory instanceof FilteredAggregatorFactory)
);
if (doesMatch) {
existingAggregationsWithSameFilter.add(existingAggregation);
}
} else {
final boolean doesMatch = existingAggregation.getAggregatorFactories().stream().allMatch(
factory -> factory instanceof FilteredAggregatorFactory &&
((FilteredAggregatorFactory) factory).getFilter().equals(filter)
);
if (doesMatch) {
existingAggregationsWithSameFilter.add(
Aggregation.create(
existingAggregation.getVirtualColumns(),
existingAggregation.getAggregatorFactories().stream()
.map(factory -> ((FilteredAggregatorFactory) factory).getAggregator())
.collect(Collectors.toList()),
existingAggregation.getPostAggregator()
)
);
}
}
}
final Aggregation retVal = sqlAggregator.toDruidAggregation(
plannerContext,
sourceRowSignature,
rexBuilder,
name,
call,
project,
existingAggregationsWithSameFilter
);
if (retVal == null) {
return null;
} else {
return druidExpression;
// Check if this refers to the existingAggregationsWithSameFilter. If so, no need to apply the filter.
if (isUsingExistingAggregation(retVal, existingAggregationsWithSameFilter)) {
return retVal;
} else {
return retVal.filter(sourceRowSignature, filter);
}
}
}
private static DimFilter makeFilter(final DimFilter filter, final RowSignature sourceRowSignature)
{
return filter == null
? null
: Filtration.create(filter)
.optimizeFilterOnly(sourceRowSignature)
.getDimFilter();
}
private static AggregatorFactory createSumAggregatorFactory(
final ValueType aggregationType,
final String name,
final String fieldName,
final String expression,
final ExprMacroTable macroTable
/**
* Checks if "aggregation" is exclusively based on existing aggregations from "existingAggregations'.
*/
private static boolean isUsingExistingAggregation(
final Aggregation aggregation,
final List<Aggregation> existingAggregations
)
{
switch (aggregationType) {
case LONG:
return new LongSumAggregatorFactory(name, fieldName, expression, macroTable);
case FLOAT:
return new FloatSumAggregatorFactory(name, fieldName, expression, macroTable);
case DOUBLE:
return new DoubleSumAggregatorFactory(name, fieldName, expression, macroTable);
default:
throw new ISE("Cannot create aggregator factory for type[%s]", aggregationType);
if (!aggregation.getAggregatorFactories().isEmpty()) {
return false;
}
}
private static AggregatorFactory createMinAggregatorFactory(
final ValueType aggregationType,
final String name,
final String fieldName,
final String expression,
final ExprMacroTable macroTable
)
{
switch (aggregationType) {
case LONG:
return new LongMinAggregatorFactory(name, fieldName, expression, macroTable);
case FLOAT:
return new FloatMinAggregatorFactory(name, fieldName, expression, macroTable);
case DOUBLE:
return new DoubleMinAggregatorFactory(name, fieldName, expression, macroTable);
default:
throw new ISE("Cannot create aggregator factory for type[%s]", aggregationType);
}
}
final Set<String> existingAggregationNames = existingAggregations
.stream()
.flatMap(xs -> xs.getAggregatorFactories().stream())
.map(AggregatorFactory::getName)
.collect(Collectors.toSet());
private static AggregatorFactory createMaxAggregatorFactory(
final ValueType aggregationType,
final String name,
final String fieldName,
final String expression,
final ExprMacroTable macroTable
)
{
switch (aggregationType) {
case LONG:
return new LongMaxAggregatorFactory(name, fieldName, expression, macroTable);
case FLOAT:
return new FloatMaxAggregatorFactory(name, fieldName, expression, macroTable);
case DOUBLE:
return new DoubleMaxAggregatorFactory(name, fieldName, expression, macroTable);
default:
throw new ISE("Cannot create aggregator factory for type[%s]", aggregationType);
}
return aggregation.getPostAggregator().getDependentFields().stream().allMatch(existingAggregationNames::contains);
}
}

View File

@ -44,6 +44,7 @@ import org.apache.calcite.schema.Statistics;
import org.apache.calcite.schema.Table;
import org.apache.calcite.schema.TableMacro;
import org.apache.calcite.schema.impl.AbstractSchema;
import org.apache.calcite.schema.impl.AbstractTable;
import org.apache.calcite.sql.type.SqlTypeName;
import javax.annotation.Nullable;
@ -117,7 +118,7 @@ public class InformationSchema extends AbstractSchema
return tableMap;
}
class SchemataTable implements ScannableTable
class SchemataTable extends AbstractTable implements ScannableTable
{
@Override
public Enumerable<Object[]> scan(final DataContext root)
@ -166,7 +167,7 @@ public class InformationSchema extends AbstractSchema
}
}
class TablesTable implements ScannableTable
class TablesTable extends AbstractTable implements ScannableTable
{
@Override
public Enumerable<Object[]> scan(final DataContext root)
@ -245,7 +246,7 @@ public class InformationSchema extends AbstractSchema
}
}
class ColumnsTable implements ScannableTable
class ColumnsTable extends AbstractTable implements ScannableTable
{
@Override
public Enumerable<Object[]> scan(final DataContext root)

View File

@ -21,6 +21,7 @@ package io.druid.sql.calcite.table;
import com.google.common.base.Preconditions;
import io.druid.query.DataSource;
import org.apache.calcite.config.CalciteConnectionConfig;
import org.apache.calcite.plan.RelOptTable;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.logical.LogicalTableScan;
@ -30,6 +31,8 @@ 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.SqlCall;
import org.apache.calcite.sql.SqlNode;
public class DruidTable implements TranslatableTable
{
@ -73,6 +76,23 @@ public class DruidTable implements TranslatableTable
return rowSignature.getRelDataType(typeFactory);
}
@Override
public boolean isRolledUp(final String column)
{
return false;
}
@Override
public boolean rolledUpColumnValidInsideAgg(
final String column,
final SqlCall call,
final SqlNode parent,
final CalciteConnectionConfig config
)
{
return true;
}
@Override
public RelNode toRel(final RelOptTable.ToRelContext context, final RelOptTable table)
{

View File

@ -20,10 +20,10 @@
package io.druid.sql.guice;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import com.google.inject.Inject;
import com.google.inject.Module;
import com.google.inject.multibindings.Multibinder;
import io.druid.guice.Jerseys;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.LazySingleton;
@ -33,27 +33,8 @@ import io.druid.server.metrics.MetricsModule;
import io.druid.sql.avatica.AvaticaMonitor;
import io.druid.sql.avatica.AvaticaServerConfig;
import io.druid.sql.avatica.DruidAvaticaHandler;
import io.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.BTrimOperatorConversion;
import io.druid.sql.calcite.expression.CeilOperatorConversion;
import io.druid.sql.calcite.expression.ExtractOperatorConversion;
import io.druid.sql.calcite.expression.FloorOperatorConversion;
import io.druid.sql.calcite.expression.LTrimOperatorConversion;
import io.druid.sql.calcite.expression.LookupOperatorConversion;
import io.druid.sql.calcite.expression.MillisToTimestampOperatorConversion;
import io.druid.sql.calcite.expression.RTrimOperatorConversion;
import io.druid.sql.calcite.expression.RegexpExtractOperatorConversion;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.expression.SubstringOperatorConversion;
import io.druid.sql.calcite.expression.TimeArithmeticOperatorConversion;
import io.druid.sql.calcite.expression.TimeExtractOperatorConversion;
import io.druid.sql.calcite.expression.TimeFloorOperatorConversion;
import io.druid.sql.calcite.expression.TimeFormatOperatorConversion;
import io.druid.sql.calcite.expression.TimeParseOperatorConversion;
import io.druid.sql.calcite.expression.TimeShiftOperatorConversion;
import io.druid.sql.calcite.expression.TimestampToMillisOperatorConversion;
import io.druid.sql.calcite.expression.TrimOperatorConversion;
import io.druid.sql.calcite.expression.builtin.LookupOperatorConversion;
import io.druid.sql.calcite.planner.Calcites;
import io.druid.sql.calcite.planner.PlannerConfig;
import io.druid.sql.calcite.schema.DruidSchema;
@ -61,37 +42,10 @@ import io.druid.sql.calcite.view.NoopViewManager;
import io.druid.sql.calcite.view.ViewManager;
import io.druid.sql.http.SqlResource;
import java.util.List;
import java.util.Properties;
public class SqlModule implements Module
{
public static final List<Class<? extends SqlAggregator>> DEFAULT_AGGREGATOR_CLASSES = ImmutableList.<Class<? extends SqlAggregator>>of(
ApproxCountDistinctSqlAggregator.class
);
public static final List<Class<? extends SqlOperatorConversion>> DEFAULT_OPERATOR_CONVERSION_CLASSES = ImmutableList.<Class<? extends SqlOperatorConversion>>builder()
.add(CeilOperatorConversion.class)
.add(ExtractOperatorConversion.class)
.add(FloorOperatorConversion.class)
.add(LookupOperatorConversion.class)
.add(MillisToTimestampOperatorConversion.class)
.add(RegexpExtractOperatorConversion.class)
.add(SubstringOperatorConversion.class)
.add(TimeArithmeticOperatorConversion.TimeMinusIntervalOperatorConversion.class)
.add(TimeArithmeticOperatorConversion.TimePlusIntervalOperatorConversion.class)
.add(TimeExtractOperatorConversion.class)
.add(TimeFloorOperatorConversion.class)
.add(TimeFormatOperatorConversion.class)
.add(TimeParseOperatorConversion.class)
.add(TimeShiftOperatorConversion.class)
.add(TimestampToMillisOperatorConversion.class)
.add(TrimOperatorConversion.class)
.add(BTrimOperatorConversion.class)
.add(LTrimOperatorConversion.class)
.add(RTrimOperatorConversion.class)
.build();
private static final String PROPERTY_SQL_ENABLE = "druid.sql.enable";
private static final String PROPERTY_SQL_ENABLE_JSON_OVER_HTTP = "druid.sql.http.enable";
private static final String PROPERTY_SQL_ENABLE_AVATICA = "druid.sql.avatica.enable";
@ -114,13 +68,11 @@ public class SqlModule implements Module
LifecycleModule.register(binder, DruidSchema.class);
binder.bind(ViewManager.class).to(NoopViewManager.class).in(LazySingleton.class);
for (Class<? extends SqlAggregator> clazz : DEFAULT_AGGREGATOR_CLASSES) {
SqlBindings.addAggregator(binder, clazz);
}
// Add empty SqlAggregator binder.
Multibinder.newSetBinder(binder, SqlAggregator.class);
for (Class<? extends SqlOperatorConversion> clazz : DEFAULT_OPERATOR_CONVERSION_CLASSES) {
SqlBindings.addOperatorConversion(binder, clazz);
}
// LookupOperatorConversion isn't in DruidOperatorTable since it needs a LookupReferencesManager injected.
SqlBindings.addOperatorConversion(binder, LookupOperatorConversion.class);
if (isJsonOverHttpEnabled()) {
Jerseys.addResource(binder, SqlResource.class);

View File

@ -19,6 +19,7 @@
package io.druid.sql.calcite;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
@ -116,6 +117,7 @@ import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
@ -690,19 +692,19 @@ public class CalciteQueryTest
+ " druid.foo x INNER JOIN druid.foo y ON x.dim1 = y.dim2\n"
+ "WHERE\n"
+ " x.dim1 <> ''",
ImmutableList.<Query>of(
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity()))
.columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1")
.columns("dim1")
.filters(NOT(SELECTOR("dim1", "", null)))
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build(),
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity()))
.filters(NOT(SELECTOR("dim1", "", null)))
.columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1")
.columns("dim1", "dim2")
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
@ -717,10 +719,9 @@ public class CalciteQueryTest
public void testExplainSelfJoinWithFallback() throws Exception
{
final String explanation =
"BindableProject(dim1=[$9], dim10=[$2], dim2=[$3])\n"
+ " BindableJoin(condition=[=($9, $3)], joinType=[inner])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n";
"BindableJoin(condition=[=($0, $2)], joinType=[inner])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}], signature=[{dim1:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}], signature=[{dim1:STRING, dim2:STRING}])\n";
testQuery(
PLANNER_CONFIG_FALLBACK,
@ -758,6 +759,119 @@ public class CalciteQueryTest
);
}
@Test
public void testGroupByOrdinal() throws Exception
{
testQuery(
"SELECT cnt, COUNT(*) FROM druid.foo GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(DIMS(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
.setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{1L, 6L}
)
);
}
@Test
@Ignore // Disabled since GROUP BY alias can confuse the validator; see DruidConformance::isGroupByAlias
public void testGroupByAndOrderByAlias() throws Exception
{
testQuery(
"SELECT cnt AS theCnt, COUNT(*) FROM druid.foo GROUP BY theCnt ORDER BY theCnt ASC",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(DIMS(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
.setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
new OrderByColumnSpec(
"d0",
OrderByColumnSpec.Direction.ASCENDING,
StringComparators.NUMERIC
)
),
Integer.MAX_VALUE
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{1L, 6L}
)
);
}
@Test
public void testGroupByExpressionAliasedAsOriginalColumnName() throws Exception
{
testQuery(
"SELECT\n"
+ "FLOOR(__time TO MONTH) AS __time,\n"
+ "COUNT(*)\n"
+ "FROM druid.foo\n"
+ "GROUP BY FLOOR(__time TO MONTH)",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity()))
.granularity(Granularities.MONTH)
.aggregators(AGGS(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{T("2000-01-01"), 3L},
new Object[]{T("2001-01-01"), 3L}
)
);
}
@Test
public void testGroupByAndOrderByOrdinalOfAlias() throws Exception
{
testQuery(
"SELECT cnt as theCnt, COUNT(*) FROM druid.foo GROUP BY 1 ORDER BY 1 ASC",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(DIMS(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
.setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
new OrderByColumnSpec(
"d0",
OrderByColumnSpec.Direction.ASCENDING,
StringComparators.NUMERIC
)
),
Integer.MAX_VALUE
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{1L, 6L}
)
);
}
@Test
public void testGroupByFloat() throws Exception
{
@ -2003,8 +2117,8 @@ public class CalciteQueryTest
new FilteredAggregatorFactory(
new CountAggregatorFactory("a3"),
AND(
NOT(SELECTOR("dim1", "1", null)),
NOT(SELECTOR("dim2", "", null))
NOT(SELECTOR("dim2", "", null)),
NOT(SELECTOR("dim1", "1", null))
)
),
new FilteredAggregatorFactory(
@ -2044,6 +2158,30 @@ public class CalciteQueryTest
);
}
@Test
@Ignore // https://issues.apache.org/jira/browse/CALCITE-1910
public void testFilteredAggregationWithNotIn() throws Exception
{
testQuery(
"SELECT\n"
+ "COUNT(*) filter(WHERE dim1 NOT IN ('1')),\n"
+ "COUNT(dim2) filter(WHERE dim1 NOT IN ('1'))\n"
+ "FROM druid.foo",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(AGGS())
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{1L, 5L}
)
);
}
@Test
public void testExpressionAggregations() throws Exception
{
@ -2254,6 +2392,45 @@ public class CalciteQueryTest
);
}
@Test
public void testInFilterWith23Elements() throws Exception
{
// Regression test for https://github.com/druid-io/druid/issues/4203.
final List<String> elements = new ArrayList<>();
elements.add("abc");
elements.add("def");
elements.add("ghi");
for (int i = 0; i < 20; i++) {
elements.add("dummy" + i);
}
final String elementsString = Joiner.on(",").join(elements.stream().map(s -> "'" + s + "'").iterator());
testQuery(
"SELECT dim1, COUNT(*) FROM druid.foo WHERE dim1 IN (" + elementsString + ") GROUP BY dim1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
.setDimFilter(new InDimFilter("dim1", elements, null))
.setAggregatorSpecs(
AGGS(
new CountAggregatorFactory("a0")
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{"abc", 1L},
new Object[]{"def", 1L}
)
);
}
@Test
public void testCountStarWithDegenerateFilter() throws Exception
{
@ -2280,17 +2457,8 @@ public class CalciteQueryTest
{
testQuery(
"SELECT COUNT(*) FROM druid.foo WHERE dim2 = 'a' and not (dim1 > 'a' OR dim1 < 'b')",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS())
.granularity(Granularities.ALL)
.filters(null)
.aggregators(AGGS(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of()
ImmutableList.of(),
ImmutableList.of(new Object[]{0L})
);
}
@ -3422,6 +3590,68 @@ public class CalciteQueryTest
);
}
@Test
@Ignore // Doesn't work
public void testTopNFilterJoinWithProjection() throws Exception
{
// Filters on top N values of some dimension by using an inner join. Also projects the outer dimension.
testQuery(
"SELECT SUBSTRING(t1.dim1, 1, 10), SUM(t1.cnt)\n"
+ "FROM druid.foo t1\n"
+ " INNER JOIN (\n"
+ " SELECT\n"
+ " SUM(cnt) AS sum_cnt,\n"
+ " dim2\n"
+ " FROM druid.foo\n"
+ " GROUP BY dim2\n"
+ " ORDER BY 1 DESC\n"
+ " LIMIT 2\n"
+ ") t2 ON (t1.dim2 = t2.dim2)\n"
+ "GROUP BY SUBSTRING(t1.dim1, 1, 10)",
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim2", "d0"))
.aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
.metric(new NumericTopNMetricSpec("a0"))
.threshold(2)
.context(QUERY_CONTEXT_DEFAULT)
.build(),
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimFilter(IN("dim2", ImmutableList.of("", "a"), null))
.setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
.setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
new OrderByColumnSpec(
"d0",
OrderByColumnSpec.Direction.ASCENDING,
StringComparators.LEXICOGRAPHIC
)
),
Integer.MAX_VALUE
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{"", 1L},
new Object[]{"1", 1L},
new Object[]{"10.1", 1L},
new Object[]{"2", 1L},
new Object[]{"abc", 1L}
)
);
}
@Test
public void testRemovableLeftJoin() throws Exception
{
@ -5371,51 +5601,32 @@ public class CalciteQueryTest
}
@Test
@Ignore // https://issues.apache.org/jira/browse/CALCITE-1799
public void testUsingSubqueryAsPartOfOrFilter() throws Exception
{
// This query should ideally be plannable without fallback, but it's not. The "OR" means it isn't really
// a semiJoin and so the filter condition doesn't get converted.
final String explanation =
"BindableSort(sort0=[$1], dir0=[ASC])\n"
+ " BindableAggregate(group=[{0, 1}], EXPR$2=[COUNT()])\n"
+ " BindableFilter(condition=[OR(=($0, 'xxx'), CAST(AND(IS NOT NULL($4), <>($2, 0))):BOOLEAN)])\n"
+ " BindableJoin(condition=[=($1, $3)], joinType=[left])\n"
+ " BindableJoin(condition=[true], joinType=[inner])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}], signature=[{dim1:STRING, dim2:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":{\"type\":\"like\",\"dimension\":\"dim1\",\"pattern\":\"%bc\",\"escape\":null,\"extractionFn\":null},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"}}], signature=[{a0:LONG}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"d1:v\",\"expression\":\"1\",\"outputType\":\"LONG\"}],\"filter\":{\"type\":\"like\",\"dimension\":\"dim1\",\"pattern\":\"%bc\",\"escape\":null,\"extractionFn\":null},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"},{\"type\":\"default\",\"dimension\":\"d1:v\",\"outputName\":\"d1\",\"outputType\":\"LONG\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}], signature=[{d0:STRING, d1:LONG}])\n";
final String theQuery = "SELECT dim1, dim2, COUNT(*) FROM druid.foo\n"
+ "WHERE dim1 = 'xxx' OR dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 LIKE '%bc')\n"
+ "group by dim1, dim2 ORDER BY dim2";
assertQueryIsUnplannable(theQuery);
testQuery(
"SELECT dim1, dim2, COUNT(*) FROM druid.foo\n"
+ "WHERE dim1 = 'xxx' OR dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 LIKE '%bc')\n"
+ "group by dim1, dim2 ORDER BY dim2",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimFilter(new LikeDimFilter("dim1", "%bc", null, null))
.setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build(),
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimFilter(
OR(
SELECTOR("dim2", "abc", null),
SELECTOR("dim1", "xxx", null)
)
)
.setDimensions(
DIMS(
new DefaultDimensionSpec("dim2", "d1"),
new DefaultDimensionSpec("dim1", "d0")
)
)
.setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(new OrderByColumnSpec("d1", OrderByColumnSpec.Direction.ASCENDING)),
Integer.MAX_VALUE
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{"def", "abc", 1L}
)
PLANNER_CONFIG_FALLBACK,
"EXPLAIN PLAN FOR " + theQuery,
ImmutableList.of(),
ImmutableList.of(new Object[]{explanation})
);
}

View File

@ -30,6 +30,15 @@ import io.druid.query.extraction.RegexDimExtractionFn;
import io.druid.query.extraction.TimeFormatExtractionFn;
import io.druid.segment.column.ValueType;
import io.druid.server.security.AuthTestUtils;
import io.druid.sql.calcite.expression.builtin.DateTruncOperatorConversion;
import io.druid.sql.calcite.expression.builtin.RegexpExtractOperatorConversion;
import io.druid.sql.calcite.expression.builtin.StrposOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TimeExtractOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TimeFloorOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TimeFormatOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TimeParseOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TimeShiftOperatorConversion;
import io.druid.sql.calcite.expression.builtin.TruncateOperatorConversion;
import io.druid.sql.calcite.planner.Calcites;
import io.druid.sql.calcite.planner.PlannerConfig;
import io.druid.sql.calcite.planner.PlannerContext;
@ -42,6 +51,7 @@ import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.SqlIntervalQualifier;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.fun.SqlTrimFunction;
@ -74,6 +84,7 @@ public class ExpressionsTest
.add("b", ValueType.LONG)
.add("x", ValueType.FLOAT)
.add("y", ValueType.LONG)
.add("z", ValueType.FLOAT)
.add("s", ValueType.STRING)
.add("spacey", ValueType.STRING)
.add("tstr", ValueType.STRING)
@ -82,9 +93,10 @@ public class ExpressionsTest
private final Map<String, Object> bindings = ImmutableMap.<String, Object>builder()
.put("t", DateTimes.of("2000-02-03T04:05:06").getMillis())
.put("a", 10)
.put("b", 20)
.put("x", 2.5)
.put("b", 25)
.put("x", 2.25)
.put("y", 3.0)
.put("z", -2.25)
.put("s", "foo")
.put("spacey", " hey there ")
.put("tstr", "2000-02-03 04:05:06")
@ -155,6 +167,174 @@ public class ExpressionsTest
);
}
@Test
public void testStrpos()
{
testExpression(
rexBuilder.makeCall(
new StrposOperatorConversion().calciteOperator(),
inputRef("s"),
rexBuilder.makeLiteral("oo")
),
DruidExpression.fromExpression("(strpos(\"s\",'oo') + 1)"),
2L
);
testExpression(
rexBuilder.makeCall(
new StrposOperatorConversion().calciteOperator(),
inputRef("s"),
rexBuilder.makeLiteral("ax")
),
DruidExpression.fromExpression("(strpos(\"s\",'ax') + 1)"),
0L
);
testExpression(
rexBuilder.makeCall(
new StrposOperatorConversion().calciteOperator(),
rexBuilder.makeNullLiteral(typeFactory.createSqlType(SqlTypeName.VARCHAR)),
rexBuilder.makeLiteral("ax")
),
DruidExpression.fromExpression("(strpos('','ax') + 1)"),
0L
);
}
@Test
public void testFloor()
{
testExpression(
rexBuilder.makeCall(SqlStdOperatorTable.FLOOR, inputRef("a")),
DruidExpression.fromExpression("floor(\"a\")"),
10.0
);
testExpression(
rexBuilder.makeCall(SqlStdOperatorTable.FLOOR, inputRef("x")),
DruidExpression.fromExpression("floor(\"x\")"),
2.0
);
testExpression(
rexBuilder.makeCall(SqlStdOperatorTable.FLOOR, inputRef("y")),
DruidExpression.fromExpression("floor(\"y\")"),
3.0
);
testExpression(
rexBuilder.makeCall(SqlStdOperatorTable.FLOOR, inputRef("z")),
DruidExpression.fromExpression("floor(\"z\")"),
-3.0
);
}
@Test
public void testCeil()
{
testExpression(
rexBuilder.makeCall(SqlStdOperatorTable.CEIL, inputRef("a")),
DruidExpression.fromExpression("ceil(\"a\")"),
10.0
);
testExpression(
rexBuilder.makeCall(SqlStdOperatorTable.CEIL, inputRef("x")),
DruidExpression.fromExpression("ceil(\"x\")"),
3.0
);
testExpression(
rexBuilder.makeCall(SqlStdOperatorTable.CEIL, inputRef("y")),
DruidExpression.fromExpression("ceil(\"y\")"),
3.0
);
testExpression(
rexBuilder.makeCall(SqlStdOperatorTable.CEIL, inputRef("z")),
DruidExpression.fromExpression("ceil(\"z\")"),
-2.0
);
}
@Test
public void testTruncate()
{
final SqlFunction truncateFunction = new TruncateOperatorConversion().calciteOperator();
testExpression(
rexBuilder.makeCall(truncateFunction, inputRef("a")),
DruidExpression.fromExpression("(cast(cast(\"a\" * 1,'long'),'double') / 1)"),
10.0
);
testExpression(
rexBuilder.makeCall(truncateFunction, inputRef("x")),
DruidExpression.fromExpression("(cast(cast(\"x\" * 1,'long'),'double') / 1)"),
2.0
);
testExpression(
rexBuilder.makeCall(truncateFunction, inputRef("y")),
DruidExpression.fromExpression("(cast(cast(\"y\" * 1,'long'),'double') / 1)"),
3.0
);
testExpression(
rexBuilder.makeCall(truncateFunction, inputRef("z")),
DruidExpression.fromExpression("(cast(cast(\"z\" * 1,'long'),'double') / 1)"),
-2.0
);
testExpression(
rexBuilder.makeCall(truncateFunction, inputRef("x"), integerLiteral(1)),
DruidExpression.fromExpression("(cast(cast(\"x\" * 10.0,'long'),'double') / 10.0)"),
2.2
);
testExpression(
rexBuilder.makeCall(truncateFunction, inputRef("z"), integerLiteral(1)),
DruidExpression.fromExpression("(cast(cast(\"z\" * 10.0,'long'),'double') / 10.0)"),
-2.2
);
testExpression(
rexBuilder.makeCall(truncateFunction, inputRef("b"), integerLiteral(-1)),
DruidExpression.fromExpression("(cast(cast(\"b\" * 0.1,'long'),'double') / 0.1)"),
20.0
);
testExpression(
rexBuilder.makeCall(truncateFunction, inputRef("z"), integerLiteral(-1)),
DruidExpression.fromExpression("(cast(cast(\"z\" * 0.1,'long'),'double') / 0.1)"),
0.0
);
}
@Test
public void testDateTrunc()
{
testExpression(
rexBuilder.makeCall(
new DateTruncOperatorConversion().calciteOperator(),
rexBuilder.makeLiteral("hour"),
timestampLiteral(DateTimes.of("2000-02-03T04:05:06Z"))
),
DruidExpression.fromExpression("timestamp_floor(949550706000,'PT1H','','UTC')"),
DateTimes.of("2000-02-03T04:00:00").getMillis()
);
testExpression(
rexBuilder.makeCall(
new DateTruncOperatorConversion().calciteOperator(),
rexBuilder.makeLiteral("DAY"),
timestampLiteral(DateTimes.of("2000-02-03T04:05:06Z"))
),
DruidExpression.fromExpression("timestamp_floor(949550706000,'P1D','','UTC')"),
DateTimes.of("2000-02-03T00:00:00").getMillis()
);
}
@Test
public void testTrim()
{
@ -534,7 +714,7 @@ public class ExpressionsTest
),
DruidExpression.of(
null,
"timestamp_parse(\"tstr\",'yyyy-MM-dd HH:mm:ss')"
"timestamp_parse(\"tstr\",'yyyy-MM-dd HH:mm:ss','UTC')"
),
DateTimes.of("2000-02-03T04:05:06Z").getMillis()
);
@ -552,7 +732,7 @@ public class ExpressionsTest
)
),
DruidExpression.fromExpression(
"timestamp_format(\"t\",'yyyy-MM-dd HH:mm:ss')"
"timestamp_format(\"t\",'yyyy-MM-dd HH:mm:ss','UTC')"
),
"2000-02-03 04:05:06"
);
@ -594,7 +774,7 @@ public class ExpressionsTest
inputRef("dstr")
),
DruidExpression.fromExpression(
"timestamp_floor(timestamp_parse(\"dstr\",'yyyy-MM-dd'),'P1D','','UTC')"
"timestamp_floor(timestamp_parse(\"dstr\",'yyyy-MM-dd','UTC'),'P1D','','UTC')"
),
DateTimes.of("2000-02-03").getMillis()
);
@ -612,7 +792,7 @@ public class ExpressionsTest
)
),
DruidExpression.fromExpression(
"timestamp_format(timestamp_floor(\"t\",'P1D','','UTC'),'yyyy-MM-dd')"
"timestamp_format(timestamp_floor(\"t\",'P1D','','UTC'),'yyyy-MM-dd','UTC')"
),
"2000-02-03"
);
@ -641,7 +821,7 @@ public class ExpressionsTest
private RexNode timestampLiteral(final DateTime timestamp)
{
return rexBuilder.makeTimestampLiteral(Calcites.jodaToCalciteCalendarLiteral(timestamp, DateTimeZone.UTC), 0);
return rexBuilder.makeTimestampLiteral(Calcites.jodaToCalciteTimestampString(timestamp, DateTimeZone.UTC), 0);
}
private RexNode integerLiteral(final int integer)

View File

@ -25,6 +25,7 @@ import com.google.common.base.Suppliers;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.inject.Binder;
import com.google.inject.Guice;
import com.google.inject.Injector;
@ -94,14 +95,13 @@ import io.druid.server.initialization.ServerConfig;
import io.druid.server.log.NoopRequestLogger;
import io.druid.server.security.AuthConfig;
import io.druid.server.security.AuthTestUtils;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.expression.builtin.LookupOperatorConversion;
import io.druid.sql.calcite.planner.DruidOperatorTable;
import io.druid.sql.calcite.planner.PlannerConfig;
import io.druid.sql.calcite.schema.DruidSchema;
import io.druid.sql.calcite.view.NoopViewManager;
import io.druid.sql.calcite.view.ViewManager;
import io.druid.sql.guice.SqlModule;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.LinearShardSpec;
import org.joda.time.DateTime;
@ -384,18 +384,9 @@ public class CalciteTests
public static DruidOperatorTable createOperatorTable()
{
try {
final Set<SqlAggregator> aggregators = new HashSet<>();
final Set<SqlOperatorConversion> extractionOperators = new HashSet<>();
for (Class<? extends SqlAggregator> clazz : SqlModule.DEFAULT_AGGREGATOR_CLASSES) {
aggregators.add(INJECTOR.getInstance(clazz));
}
for (Class<? extends SqlOperatorConversion> clazz : SqlModule.DEFAULT_OPERATOR_CONVERSION_CLASSES) {
extractionOperators.add(INJECTOR.getInstance(clazz));
}
return new DruidOperatorTable(aggregators, extractionOperators);
extractionOperators.add(INJECTOR.getInstance(LookupOperatorConversion.class));
return new DruidOperatorTable(ImmutableSet.of(), extractionOperators);
}
catch (Exception e) {
throw Throwables.propagate(e);