mirror of https://github.com/apache/druid.git
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:
parent
4e1d0f49d8
commit
b20e3038b6
|
@ -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
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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.|
|
||||
|
|
|
@ -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.|
|
||||
|
|
|
@ -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
|
||||
|
|
4
pom.xml
4
pom.xml
|
@ -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>
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
|
|
@ -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
|
|
@ -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)
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
)
|
||||
);
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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))
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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()
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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;
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
|
@ -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()))
|
||||
)
|
||||
);
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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
|
||||
)
|
||||
);
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
)
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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})
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
|
|
Loading…
Reference in New Issue