Cache parsed expressions and binding analysis in more places. (#14124)

* Cache parsed expressions and binding analysis in more places.

Main changes:

1) Cache parsed and analyzed expressions within PlannerContext for a
   single SQL query.

2) Cache parsed expressions together with input binding analysis using
   a new class AnalyzeExpr.

This speeds up SQL planning, because SQL planning involves parsing
analyzing the same expression strings over and over again.

* Fixes.

* Fix style.

* Fix test.

* Simplify: get rid of AnalyzedExpr, focus on caching.

* Rename parse -> parseExpression.
This commit is contained in:
Gian Merlino 2023-06-27 13:40:35 -07:00 committed by GitHub
parent 4bd6bd0d4f
commit c78d885b80
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
40 changed files with 363 additions and 292 deletions

View File

@ -67,7 +67,6 @@ public class ExpressionPostAggregator implements PostAggregator
@Nullable
private final String ordering;
private final ExprMacroTable macroTable;
private final Map<String, Function<Object, Object>> finalizers;
private final Expr.InputBindingInspector partialTypeInformation;
@ -76,7 +75,7 @@ public class ExpressionPostAggregator implements PostAggregator
private final Supplier<byte[]> cacheKey;
/**
* Constructor for serialization.
* Constructor for deserialization.
*/
@JsonCreator
public ExpressionPostAggregator(
@ -90,16 +89,35 @@ public class ExpressionPostAggregator implements PostAggregator
name,
expression,
ordering,
macroTable,
Parser.lazyParse(expression, macroTable)
);
}
/**
* Constructor for a pre-parsed expression.
*/
public ExpressionPostAggregator(
final String name,
final String expression,
@Nullable final String ordering,
final Expr parsed
)
{
this(
name,
expression,
ordering,
() -> parsed
);
}
/**
* Constructor for a supplier of a pre-parsed expression.
*/
private ExpressionPostAggregator(
final String name,
final String expression,
@Nullable final String ordering,
final ExprMacroTable macroTable,
final Supplier<Expr> parsed
)
{
@ -107,7 +125,6 @@ public class ExpressionPostAggregator implements PostAggregator
name,
expression,
ordering,
macroTable,
ImmutableMap.of(),
InputBindings.nilBindings(),
parsed,
@ -119,7 +136,6 @@ public class ExpressionPostAggregator implements PostAggregator
final String name,
final String expression,
@Nullable final String ordering,
final ExprMacroTable macroTable,
final Map<String, Function<Object, Object>> finalizers,
final Expr.InputBindingInspector partialTypeInformation,
final Supplier<Expr> parsed,
@ -133,7 +149,6 @@ public class ExpressionPostAggregator implements PostAggregator
this.ordering = ordering;
// comparator should be specialized to output type ... someday
this.comparator = ordering == null ? DEFAULT_COMPARATOR : Ordering.valueOf(ordering);
this.macroTable = macroTable;
this.finalizers = finalizers;
this.partialTypeInformation = partialTypeInformation;
@ -147,7 +162,6 @@ public class ExpressionPostAggregator implements PostAggregator
});
}
@Override
public Set<String> getDependentFields()
{
@ -208,7 +222,6 @@ public class ExpressionPostAggregator implements PostAggregator
name,
expression,
ordering,
macroTable,
finalizers,
InputBindings.inspectorFromTypeMap(types),
parsed,
@ -252,7 +265,7 @@ public class ExpressionPostAggregator implements PostAggregator
* Ensures the following order: numeric > NaN > Infinite.
*
* The name may be referenced via Ordering.valueOf(String) in the constructor {@link
* ExpressionPostAggregator#ExpressionPostAggregator(String, String, String, ExprMacroTable, Map, Expr.InputBindingInspector, Supplier, Supplier)}.
* ExpressionPostAggregator#ExpressionPostAggregator(String, String, String, Map, Expr.InputBindingInspector, Supplier, Supplier)}.
*/
@SuppressWarnings("unused")
numericFirst {

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.collect.RangeSet;
@ -45,6 +44,9 @@ public class ExpressionDimFilter extends AbstractOptimizableDimFilter implements
@Nullable
private final FilterTuning filterTuning;
/**
* Constructor for deserialization.
*/
@JsonCreator
public ExpressionDimFilter(
@JsonProperty("expression") final String expression,
@ -52,20 +54,40 @@ public class ExpressionDimFilter extends AbstractOptimizableDimFilter implements
@JacksonInject ExprMacroTable macroTable
)
{
this.expression = expression;
this.filterTuning = filterTuning;
this.parsed = Parser.lazyParse(expression, macroTable);
this.cacheKey = Suppliers.memoize(() -> {
return new CacheKeyBuilder(DimFilterUtils.EXPRESSION_CACHE_ID)
.appendCacheable(parsed.get())
.build();
});
this(expression, Parser.lazyParse(expression, macroTable), filterTuning);
}
@VisibleForTesting
/**
* Constructor used in various tests that don't need to provide {@link FilterTuning}.
*/
public ExpressionDimFilter(final String expression, ExprMacroTable macroTable)
{
this(expression, null, macroTable);
this(expression, Parser.lazyParse(expression, macroTable), null);
}
/**
* Constructor for already-parsed-and-analyzed expressions.
*/
public ExpressionDimFilter(final String expression, final Expr parsed, @Nullable final FilterTuning filterTuning)
{
this(expression, () -> parsed, filterTuning);
}
private ExpressionDimFilter(
String expression,
Supplier<Expr> parsed,
@Nullable FilterTuning filterTuning
)
{
this.expression = expression;
this.parsed = parsed;
this.filterTuning = filterTuning;
this.cacheKey = Suppliers.memoize(
() ->
new CacheKeyBuilder(DimFilterUtils.EXPRESSION_CACHE_ID)
.appendCacheable(parsed.get())
.build()
);
}
@JsonProperty

View File

@ -99,7 +99,23 @@ public class JoinConditionAnalysis
final ExprMacroTable macroTable
)
{
final Expr conditionExpr = Parser.parse(condition, macroTable);
return forExpression(condition, Parser.parse(condition, macroTable), rightPrefix);
}
/**
* Analyze a join condition from a pre-parsed expression.
*
* @param condition the condition expression
* @param conditionExpr the parsed condition expression. Must match "condition".
* @param rightPrefix prefix for the right-hand side of the join; will be used to determine which identifiers in
* the condition come from the right-hand side and which come from the left-hand side
*/
public static JoinConditionAnalysis forExpression(
final String condition,
final Expr conditionExpr,
final String rightPrefix
)
{
final List<Equality> equiConditions = new ArrayList<>();
final List<Expr> nonEquiConditions = new ArrayList<>();

View File

@ -63,6 +63,9 @@ public class ExpressionVirtualColumn implements VirtualColumn
private final Supplier<Expr> parsedExpression;
private final Supplier<byte[]> cacheKey;
/**
* Constructor for deserialization.
*/
@JsonCreator
public ExpressionVirtualColumn(
@JsonProperty("name") String name,
@ -71,28 +74,57 @@ public class ExpressionVirtualColumn implements VirtualColumn
@JacksonInject ExprMacroTable macroTable
)
{
this.name = Preconditions.checkNotNull(name, "name");
this.expression = Preconditions.checkNotNull(expression, "expression");
this.outputType = outputType;
this.parsedExpression = Parser.lazyParse(expression, macroTable);
this.cacheKey = makeCacheKeySupplier();
this(name, expression, outputType, Parser.lazyParse(expression, macroTable));
}
/**
* Constructor for creating an ExpressionVirtualColumn from a pre-parsed expression.
* Constructor for creating an ExpressionVirtualColumn from a pre-parsed-and-analyzed expression, where the original
* expression string is known.
*/
public ExpressionVirtualColumn(
String name,
Expr parsedExpression,
@Nullable ColumnType outputType
final String name,
final String expression,
final Expr parsedExpression,
@Nullable final ColumnType outputType
)
{
this(name, expression, outputType, () -> parsedExpression);
}
/**
* Constructor for creating an ExpressionVirtualColumn from a pre-parsed expression, where the original
* expression string is not known.
*
* This constructor leads to an instance where {@link #getExpression()} is the toString representation of the
* parsed expression, which is not necessarily a valid expression. Do not try to reparse it as an expression, as
* this will not work.
*
* If you know the original expression, use
* {@link ExpressionVirtualColumn#ExpressionVirtualColumn(String, String, Expr, ColumnType)} instead.
*/
public ExpressionVirtualColumn(
final String name,
final Expr parsedExpression,
@Nullable final ColumnType outputType
)
{
this(name, parsedExpression.toString(), outputType, () -> parsedExpression);
}
/**
* Private constructor used by the public ones.
*/
private ExpressionVirtualColumn(
final String name,
final String expression,
@Nullable final ColumnType outputType,
final Supplier<Expr> parsedExpression
)
{
this.name = Preconditions.checkNotNull(name, "name");
// Unfortunately this string representation can't be reparsed into the same expression, might be useful
// if the expression system supported that
this.expression = parsedExpression.toString();
this.expression = Preconditions.checkNotNull(expression, "expression");
this.outputType = outputType;
this.parsedExpression = Suppliers.ofInstance(parsedExpression);
this.parsedExpression = parsedExpression;
this.cacheKey = makeCacheKeySupplier();
}

View File

@ -63,7 +63,7 @@ public class ExpressionPostAggregatorTest extends InitializedNullHandlingTest
{
EqualsVerifier.forClass(ExpressionPostAggregator.class)
.usingGetClass()
.withIgnoredFields("macroTable", "finalizers", "parsed", "dependentFields", "cacheKey", "partialTypeInformation")
.withIgnoredFields("finalizers", "parsed", "dependentFields", "cacheKey", "partialTypeInformation")
.verify();
}

View File

@ -88,6 +88,10 @@
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.github.ben-manes.caffeine</groupId>
<artifactId>caffeine</artifactId>
</dependency>
<dependency>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>

View File

@ -93,7 +93,7 @@ public class ArrayConcatSqlAggregator implements SqlAggregator
maxSizeBytes = ((Number) RexLiteral.value(maxBytes)).intValue();
}
final DruidExpression arg = Expressions.toDruidExpression(plannerContext, rowSignature, arguments.get(0));
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable();
final String fieldName;
final ColumnType druidType = Calcites.getValueTypeForRelDataTypeFull(aggregateCall.getType());

View File

@ -99,7 +99,7 @@ public class ArraySqlAggregator implements SqlAggregator
// can't translate argument
return null;
}
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable();
final String fieldName;
final String initialvalue;

View File

@ -94,7 +94,7 @@ public class AvgSqlAggregator implements SqlAggregator
final DruidExpression arg = Iterables.getOnlyElement(arguments);
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable();
final ColumnType sumType;
// Use 64-bit sum regardless of the type of the AVG aggregator.
if (SqlTypeName.INT_TYPES.contains(aggregateCall.getType().getSqlTypeName())) {

View File

@ -142,7 +142,7 @@ public class BitwiseSqlAggregator implements SqlAggregator
}
final DruidExpression arg = arguments.get(0);
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable();
final String fieldName;
if (arg.isDirectColumnAccess()) {

View File

@ -1,153 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.sql.calcite.aggregation.builtin;
import com.google.common.base.Preconditions;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.Aggregations;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.List;
/**
* Abstraction for simple multi-column post aggregators like greatest, least
*/
public abstract class MultiColumnSqlAggregator implements SqlAggregator
{
/**
* Useful Abstraction for passing field information to subclasses from shared parent methods
*/
protected static class FieldInfo
{
final String fieldName;
final String expression;
private FieldInfo(String fieldName, String expression)
{
this.fieldName = fieldName;
this.expression = expression;
}
public static FieldInfo fromFieldName(String fieldName)
{
return new FieldInfo(fieldName, null);
}
public static FieldInfo fromExpression(String expression)
{
return new FieldInfo(null, expression);
}
}
@Nullable
@Override
public Aggregation toDruidAggregation(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final VirtualColumnRegistry virtualColumnRegistry,
final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
final Project project,
final List<Aggregation> existingAggregations,
final boolean finalizeAggregations
)
{
if (aggregateCall.isDistinct()) {
return null;
}
final List<DruidExpression> arguments = Aggregations.getArgumentsForSimpleAggregator(
rexBuilder,
plannerContext,
rowSignature,
aggregateCall,
project
);
if (arguments == null) {
return null;
}
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
final List<FieldInfo> fieldInfoList = new ArrayList<>();
// Convert arguments to concise field information
for (DruidExpression argument : arguments) {
if (argument.isDirectColumnAccess()) {
fieldInfoList.add(FieldInfo.fromFieldName(argument.getDirectColumn()));
} else {
fieldInfoList.add(FieldInfo.fromExpression(argument.getExpression()));
}
}
Preconditions.checkArgument(!fieldInfoList.isEmpty(), "FieldInfoList should not be empty");
return getAggregation(name, aggregateCall, macroTable, fieldInfoList);
}
private Aggregation getAggregation(
String name,
AggregateCall aggregateCall,
ExprMacroTable macroTable,
List<FieldInfo> fieldInfoList
)
{
final ColumnType valueType = Calcites.getColumnTypeForRelDataType(aggregateCall.getType());
List<AggregatorFactory> aggregatorFactories = new ArrayList<>();
List<PostAggregator> postAggregators = new ArrayList<>();
// Delegate aggregator factory construction to subclasses for provided fields.
// Create corresponding field access post aggregators.
int id = 0;
for (FieldInfo fieldInfo : fieldInfoList) {
String prefixedName = Calcites.makePrefixedName(name, String.valueOf(id++));
postAggregators.add(new FieldAccessPostAggregator(null, prefixedName));
aggregatorFactories.add(createAggregatorFactory(valueType, prefixedName, fieldInfo, macroTable));
}
// Delegate final post aggregator construction to subclasses by passing the above aggregators.
final PostAggregator finalPostAggregator = createFinalPostAggregator(valueType, name, postAggregators);
return Aggregation.create(aggregatorFactories, finalPostAggregator);
}
abstract AggregatorFactory createAggregatorFactory(
ColumnType valueType,
String prefixedName,
FieldInfo fieldInfo,
ExprMacroTable macroTable);
abstract PostAggregator createFinalPostAggregator(
ColumnType valueType,
String name,
List<PostAggregator> postAggregators);
}

View File

@ -84,7 +84,7 @@ public abstract class SimpleSqlAggregator implements SqlAggregator
}
final DruidExpression arg = Iterables.getOnlyElement(arguments);
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable();
final String fieldName;

View File

@ -128,7 +128,7 @@ public class StringSqlAggregator implements SqlAggregator
maxSizeBytes = ((Number) RexLiteral.value(maxBytes)).intValue();
}
final DruidExpression arg = arguments.get(0);
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable();
final String initialvalue = "[]";
final ColumnType elementType = ColumnType.STRING;

View File

@ -26,11 +26,10 @@ import com.google.common.io.BaseEncoding;
import com.google.common.primitives.Chars;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.sql.calcite.planner.ExpressionParser;
import javax.annotation.Nullable;
import java.util.ArrayList;
@ -48,7 +47,7 @@ import java.util.function.Function;
*
* When added to {@link org.apache.druid.sql.calcite.rel.VirtualColumnRegistry} whenever used by projections, filters,
* aggregators, or other query components, these will be converted into native virtual columns using
* {@link #toVirtualColumn(String, ColumnType, ExprMacroTable)}
* {@link #toVirtualColumn(String, ColumnType, ExpressionParser)}
*
* Approximate expression structure is retained in the {@link #arguments}, which when fed into the
* {@link ExpressionGenerator} that all {@link DruidExpression} must be created with will produce the final String
@ -375,30 +374,22 @@ public class DruidExpression
return arguments;
}
/**
* Compile the {@link DruidExpression} into a string and parse it into a native Druid {@link Expr}
*/
public Expr parse(final ExprMacroTable macroTable)
{
return Parser.parse(expression.get(), macroTable);
}
public VirtualColumn toVirtualColumn(
final String name,
final ColumnType outputType,
final ExprMacroTable macroTable
final ExpressionParser parser
)
{
return virtualColumnCreator.create(name, outputType, expression.get(), macroTable);
return virtualColumnCreator.create(name, outputType, expression.get(), parser);
}
public VirtualColumn toExpressionVirtualColumn(
final String name,
final ColumnType outputType,
final ExprMacroTable macroTable
final ExpressionParser parser
)
{
return DEFAULT_VIRTUAL_COLUMN_BUILDER.create(name, outputType, expression.get(), macroTable);
return DEFAULT_VIRTUAL_COLUMN_BUILDER.create(name, outputType, expression.get(), parser);
}
public NodeType getType()
@ -413,7 +404,7 @@ public class DruidExpression
* supplied by other means.
*
* This value is not currently used other than for tracking the types of the {@link DruidExpression} tree. The
* value passed to {@link #toVirtualColumn(String, ColumnType, ExprMacroTable)} will instead be whatever type "hint"
* value passed to {@link #toVirtualColumn(String, ColumnType, ExpressionParser)} will instead be whatever type "hint"
* was specified when the expression was added to the {@link org.apache.druid.sql.calcite.rel.VirtualColumnRegistry}.
*/
@Nullable
@ -597,15 +588,15 @@ public class DruidExpression
@FunctionalInterface
public interface VirtualColumnCreator
{
VirtualColumn create(String name, ColumnType outputType, String expression, ExprMacroTable macroTable);
VirtualColumn create(String name, ColumnType outputType, String expression, ExpressionParser parser);
}
public static class ExpressionVirtualColumnCreator implements VirtualColumnCreator
{
@Override
public VirtualColumn create(String name, ColumnType outputType, String expression, ExprMacroTable macroTable)
public VirtualColumn create(String name, ColumnType outputType, String expression, ExpressionParser parser)
{
return new ExpressionVirtualColumn(name, expression, outputType, macroTable);
return new ExpressionVirtualColumn(name, expression, parser.parse(expression), outputType);
}
}
}

View File

@ -37,8 +37,6 @@ import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.expression.TimestampFloorExprMacro;
import org.apache.druid.query.extraction.ExtractionFn;
@ -58,6 +56,7 @@ import org.apache.druid.sql.calcite.filtration.BoundRefKey;
import org.apache.druid.sql.calcite.filtration.Bounds;
import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.ExpressionParser;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import org.apache.druid.sql.calcite.table.RowSignatures;
@ -598,7 +597,8 @@ public class Expressions
}
// Special handling for filters on FLOOR(__time TO granularity).
final Granularity queryGranularity = toQueryGranularity(lhsExpression, plannerContext.getExprMacroTable());
final Granularity queryGranularity =
toQueryGranularity(lhsExpression, plannerContext.getExpressionParser());
if (queryGranularity != null) {
// lhs is FLOOR(__time TO granularity); rhs must be a timestamp
final long rhsMillis = Calcites.calciteDateTimeLiteralToJoda(rhs, plannerContext.getTimeZone()).getMillis();
@ -713,9 +713,16 @@ public class Expressions
)
{
final DruidExpression druidExpression = toDruidExpression(plannerContext, rowSignature, rexNode);
return druidExpression != null
? new ExpressionDimFilter(druidExpression.getExpression(), plannerContext.getExprMacroTable())
: null;
if (druidExpression != null) {
return new ExpressionDimFilter(
druidExpression.getExpression(),
plannerContext.parseExpression(druidExpression.getExpression()),
null
);
}
return null;
}
/**
@ -725,9 +732,9 @@ public class Expressions
* @return granularity or null if not possible
*/
@Nullable
public static Granularity toQueryGranularity(final DruidExpression expression, final ExprMacroTable macroTable)
public static Granularity toQueryGranularity(final DruidExpression expression, final ExpressionParser parser)
{
final TimestampFloorExprMacro.TimestampFloorExpr expr = asTimestampFloorExpr(expression, macroTable);
final TimestampFloorExprMacro.TimestampFloorExpr expr = asTimestampFloorExpr(expression, parser);
if (expr == null) {
return null;
@ -746,10 +753,10 @@ public class Expressions
@Nullable
public static TimestampFloorExprMacro.TimestampFloorExpr asTimestampFloorExpr(
final DruidExpression expression,
final ExprMacroTable macroTable
final ExpressionParser parser
)
{
final Expr expr = Parser.parse(expression.getExpression(), macroTable);
final Expr expr = parser.parse(expression.getExpression());
if (expr instanceof TimestampFloorExprMacro.TimestampFloorExpr) {
return (TimestampFloorExprMacro.TimestampFloorExpr) expr;

View File

@ -303,7 +303,7 @@ public class OperatorConversions
postAggregatorVisitor.getOutputNamePrefix() + postAggregatorVisitor.getAndIncrementCounter(),
druidExpression.getExpression(),
null,
plannerContext.getExprMacroTable()
plannerContext.parseExpression(druidExpression.getExpression())
);
}
}

View File

@ -29,7 +29,6 @@ import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.InputBindings;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.query.filter.AndDimFilter;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.segment.column.RowSignature;
@ -96,7 +95,7 @@ public class ArrayContainsOperatorConversion extends BaseExpressionDimFilterOper
final DruidExpression rightExpr = druidExpressions.get(1);
if (leftExpr.isSimpleExtraction() && !(leftExpr.isDirectColumnAccess() && leftExpr.getDruidType() != null && leftExpr.getDruidType().isArray())) {
Expr expr = Parser.parse(rightExpr.getExpression(), plannerContext.getExprMacroTable());
Expr expr = plannerContext.parseExpression(rightExpr.getExpression());
// To convert this expression filter into an And of Selector filters, we need to extract all array elements.
// For now, we can optimize only when rightExpr is a literal because there is no way to extract the array elements
// by traversing the Expr. Note that all implementations of Expr are defined as package-private classes in a

View File

@ -29,7 +29,6 @@ import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.InputBindings;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.filter.InDimFilter;
import org.apache.druid.segment.column.RowSignature;
@ -112,7 +111,7 @@ public class ArrayOverlapOperatorConversion extends BaseExpressionDimFilterOpera
return toExpressionFilter(plannerContext, getDruidFunctionName(), druidExpressions);
}
Expr expr = Parser.parse(complexExpr.getExpression(), plannerContext.getExprMacroTable());
Expr expr = plannerContext.parseExpression(complexExpr.getExpression());
if (expr.isLiteral()) {
// Evaluate the expression to take out the array elements.
// We can safely pass null if the expression is literal.

View File

@ -50,7 +50,8 @@ public abstract class BaseExpressionDimFilterOperatorConversion extends DirectOp
return new ExpressionDimFilter(
filterExpr,
plannerContext.getExprMacroTable()
plannerContext.parseExpression(filterExpr),
null
);
}

View File

@ -142,9 +142,17 @@ public class CaseOperatorConversion implements SqlOperatorConversion
// if either then or else filters produced a native filter (that wasn't just another expression filter)
// make sure we have filters for both sides by filling in the gaps with expression filter
if (thenFilter != null && !(thenFilter instanceof ExpressionDimFilter) && elseFilter == null) {
elseFilter = new ExpressionDimFilter(elseExpression.getExpression(), plannerContext.getExprMacroTable());
elseFilter = new ExpressionDimFilter(
elseExpression.getExpression(),
plannerContext.parseExpression(elseExpression.getExpression()),
null
);
} else if (thenFilter == null && elseFilter != null && !(elseFilter instanceof ExpressionDimFilter)) {
thenFilter = new ExpressionDimFilter(thenExpression.getExpression(), plannerContext.getExprMacroTable());
thenFilter = new ExpressionDimFilter(
thenExpression.getExpression(),
plannerContext.parseExpression(thenExpression.getExpression()),
null
);
}
if (thenFilter != null && elseFilter != null) {

View File

@ -118,7 +118,7 @@ public class CastOperatorConversion implements SqlOperatorConversion
return TimeFloorOperatorConversion.applyTimestampFloor(
typeCastExpression,
new PeriodGranularity(Period.days(1), null, plannerContext.getTimeZone()),
plannerContext.getExprMacroTable()
plannerContext
);
} else {
return typeCastExpression;
@ -148,7 +148,7 @@ public class CastOperatorConversion implements SqlOperatorConversion
return TimeFloorOperatorConversion.applyTimestampFloor(
timestampExpression,
new PeriodGranularity(Period.days(1), null, plannerContext.getTimeZone()),
plannerContext.getExprMacroTable()
plannerContext
);
} else if (toType == SqlTypeName.TIMESTAMP) {
return timestampExpression;

View File

@ -30,7 +30,6 @@ import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.OperatorConversions;
@ -91,10 +90,7 @@ public class DateTruncOperatorConversion implements SqlOperatorConversion
rexNode,
inputExpressions -> {
final DruidExpression arg = inputExpressions.get(1);
final Expr truncTypeExpr = Parser.parse(
inputExpressions.get(0).getExpression(),
plannerContext.getExprMacroTable()
);
final Expr truncTypeExpr = plannerContext.parseExpression(inputExpressions.get(0).getExpression());
if (!truncTypeExpr.isLiteral()) {
throw new IAE("Operator[%s] truncType must be a literal", calciteOperator().getName());

View File

@ -32,7 +32,6 @@ import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.InputBindings;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.virtual.ListFilteredVirtualColumn;
@ -333,7 +332,7 @@ public class MultiValueStringOperatorConversions
return null;
}
Expr expr = Parser.parse(druidExpressions.get(1).getExpression(), plannerContext.getExprMacroTable());
Expr expr = plannerContext.parseExpression(druidExpressions.get(1).getExpression());
// the right expression must be a literal array for this to work, since we need the values of the column
if (!expr.isLiteral()) {
return null;

View File

@ -46,7 +46,6 @@ import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.InputBindings;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.nested.NestedPathFinder;
@ -191,7 +190,7 @@ public class NestedDataOperatorConversions
return null;
}
final Expr pathExpr = Parser.parse(druidExpressions.get(1).getExpression(), plannerContext.getExprMacroTable());
final Expr pathExpr = plannerContext.parseExpression(druidExpressions.get(1).getExpression());
if (!pathExpr.isLiteral()) {
return null;
}
@ -371,7 +370,7 @@ public class NestedDataOperatorConversions
return null;
}
final Expr pathExpr = Parser.parse(druidExpressions.get(1).getExpression(), plannerContext.getExprMacroTable());
final Expr pathExpr = plannerContext.parseExpression(druidExpressions.get(1).getExpression());
if (!pathExpr.isLiteral()) {
return null;
}
@ -493,7 +492,7 @@ public class NestedDataOperatorConversions
return null;
}
final Expr pathExpr = Parser.parse(druidExpressions.get(1).getExpression(), plannerContext.getExprMacroTable());
final Expr pathExpr = plannerContext.parseExpression(druidExpressions.get(1).getExpression());
if (!pathExpr.isLiteral()) {
return null;
}
@ -661,7 +660,7 @@ public class NestedDataOperatorConversions
return null;
}
final Expr pathExpr = Parser.parse(druidExpressions.get(1).getExpression(), plannerContext.getExprMacroTable());
final Expr pathExpr = plannerContext.parseExpression(druidExpressions.get(1).getExpression());
if (!pathExpr.isLiteral()) {
return null;
}

View File

@ -72,7 +72,7 @@ public class QueryLookupOperatorConversion implements SqlOperatorConversion
StringUtils.toLowerCase(calciteOperator().getName()),
inputExpressions -> {
final DruidExpression arg = inputExpressions.get(0);
final Expr lookupNameExpr = inputExpressions.get(1).parse(plannerContext.getExprMacroTable());
final Expr lookupNameExpr = plannerContext.parseExpression(inputExpressions.get(1).getExpression());
if (arg.isSimpleExtraction() && lookupNameExpr.isLiteral()) {
return arg.getSimpleExtraction().cascade(

View File

@ -66,9 +66,9 @@ public class RegexpExtractOperatorConversion implements SqlOperatorConversion
StringUtils.toLowerCase(calciteOperator().getName()),
inputExpressions -> {
final DruidExpression arg = inputExpressions.get(0);
final Expr patternExpr = inputExpressions.get(1).parse(plannerContext.getExprMacroTable());
final Expr patternExpr = plannerContext.parseExpression(inputExpressions.get(1).getExpression());
final Expr indexExpr = inputExpressions.size() > 2
? inputExpressions.get(2).parse(plannerContext.getExprMacroTable())
? plannerContext.parseExpression(inputExpressions.get(2).getExpression())
: null;
if (arg.isSimpleExtraction() && patternExpr.isLiteral() && (indexExpr == null || indexExpr.isLiteral())) {

View File

@ -32,7 +32,6 @@ import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.expression.TimestampFloorExprMacro;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
@ -69,7 +68,7 @@ public class TimeFloorOperatorConversion implements SqlOperatorConversion
public static DruidExpression applyTimestampFloor(
final DruidExpression input,
final PeriodGranularity granularity,
final ExprMacroTable macroTable
final PlannerContext plannerContext
)
{
Preconditions.checkNotNull(input, "input");
@ -79,7 +78,7 @@ public class TimeFloorOperatorConversion implements SqlOperatorConversion
if (granularity.getPeriod().equals(Period.days(1))) {
final TimestampFloorExprMacro.TimestampFloorExpr floorExpr = Expressions.asTimestampFloorExpr(
input,
macroTable
plannerContext.getExpressionParser()
);
if (floorExpr != null) {

View File

@ -62,7 +62,7 @@ public class TruncateOperatorConversion implements SqlOperatorConversion
inputExpressions -> {
final DruidExpression arg = inputExpressions.get(0);
final Expr digitsExpr = inputExpressions.size() > 1
? inputExpressions.get(1).parse(plannerContext.getExprMacroTable())
? plannerContext.parseExpression(inputExpressions.get(1).getExpression())
: null;
final String factorString;

View File

@ -0,0 +1,34 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.sql.calcite.planner;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.Parser;
/**
* Like {@link Parser#parse}, but allows caching via {@link ExpressionParserImpl}.
*/
public interface ExpressionParser
{
/**
* Parse a string into a flattened {@link Expr}. Parsing is eager and the results may be cached.
*/
Expr parse(String expr);
}

View File

@ -0,0 +1,56 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.sql.calcite.planner;
import com.github.benmanes.caffeine.cache.Cache;
import com.github.benmanes.caffeine.cache.Caffeine;
import com.github.benmanes.caffeine.cache.Weigher;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.Parser;
/**
* Member of {@link PlannerToolbox} that caches the result of {@link Parser#parse}.
*/
public class ExpressionParserImpl implements ExpressionParser
{
/**
* Maximum total expression length stored in the {@link #cache}, measured in characters of unparsed expressions.
*/
private static final int MAX_EXPRESSION_WEIGHT = 1_000_000;
private final Cache<String, Expr> cache;
private final ExprMacroTable macroTable;
public ExpressionParserImpl(final ExprMacroTable macroTable)
{
this.cache = Caffeine.newBuilder()
.maximumWeight(MAX_EXPRESSION_WEIGHT)
.weigher((Weigher<String, Expr>) (key, value) -> key.length())
.build();
this.macroTable = macroTable;
}
@Override
public Expr parse(final String expression)
{
return cache.get(expression, k -> Parser.parse(expression, macroTable));
}
}

View File

@ -32,7 +32,7 @@ import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Numbers;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
@ -89,6 +89,7 @@ public class PlannerContext
public static final String DATA_CTX_AUTHENTICATION_RESULT = "authenticationResult";
private final PlannerToolbox plannerToolbox;
private final ExpressionParser expressionParser;
private final String sql;
private final PlannerConfig plannerConfig;
private final DateTime localNow;
@ -126,6 +127,7 @@ public class PlannerContext
)
{
this.plannerToolbox = plannerToolbox;
this.expressionParser = new ExpressionParserImpl(plannerToolbox.exprMacroTable());
this.sql = sql;
this.plannerConfig = Preconditions.checkNotNull(plannerConfig, "plannerConfig");
this.engine = engine;
@ -229,10 +231,17 @@ public class PlannerContext
return plannerToolbox;
}
// Deprecated: prefer using the toolbox
public ExprMacroTable getExprMacroTable()
public ExpressionParser getExpressionParser()
{
return plannerToolbox.exprMacroTable();
return expressionParser;
}
/**
* Equivalent to {@link ExpressionParser#parse(String)} on {@link #getExpressionParser()}.
*/
public Expr parseExpression(final String expr)
{
return expressionParser.parse(expr);
}
// Deprecated: prefer using the toolbox

View File

@ -190,7 +190,7 @@ public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
expressionToUnnest.toVirtualColumn(
correlateRowSignature.getColumnName(correlateRowSignature.size() - 1),
Calcites.getColumnTypeForRelDataType(rexNodeToUnnest.getType()),
getPlannerContext().getExprMacroTable()
getPlannerContext().getExpressionParser()
),
unnestFilterOnDataSource
),

View File

@ -48,6 +48,7 @@ import org.apache.druid.query.QueryDataSource;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.join.JoinConditionAnalysis;
import org.apache.druid.segment.join.JoinType;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.Expressions;
@ -171,7 +172,7 @@ public class DruidJoinQueryRel extends DruidRel<DruidJoinQueryRel>
VirtualColumnRegistry virtualColumnRegistry = VirtualColumnRegistry.create(
prefixSignaturePair.rhs,
getPlannerContext().getExprMacroTable(),
getPlannerContext().getExpressionParser(),
getPlannerContext().getPlannerConfig().isForceExpressionVirtualColumns()
);
getPlannerContext().setJoinExpressionVirtualColumnRegistry(virtualColumnRegistry);
@ -198,10 +199,13 @@ public class DruidJoinQueryRel extends DruidRel<DruidJoinQueryRel>
leftDataSource,
rightDataSource,
prefixSignaturePair.lhs,
condition.getExpression(),
JoinConditionAnalysis.forExpression(
condition.getExpression(),
getPlannerContext().parseExpression(condition.getExpression()),
prefixSignaturePair.lhs
),
toDruidJoinType(joinRel.getJoinType()),
getDimFilter(getPlannerContext(), leftSignature, leftFilter),
getPlannerContext().getExprMacroTable(),
getPlannerContext().getJoinableFactoryWrapper()
),
prefixSignaturePair.rhs,

View File

@ -199,7 +199,7 @@ public class DruidQuery
if (virtualColumnRegistry == null) {
virtualColumnRegistry = VirtualColumnRegistry.create(
sourceRowSignature,
plannerContext.getExprMacroTable(),
plannerContext.getExpressionParser(),
plannerContext.getPlannerConfig().isForceExpressionVirtualColumns()
);
}
@ -1043,7 +1043,7 @@ public class DruidQuery
final DimensionExpression dimensionExpression = Iterables.getOnlyElement(grouping.getDimensions());
queryGranularity = Expressions.toQueryGranularity(
dimensionExpression.getDruidExpression(),
plannerContext.getExprMacroTable()
plannerContext.getExpressionParser()
);
if (queryGranularity == null) {
@ -1315,7 +1315,7 @@ public class DruidQuery
for (DimensionExpression dimensionExpression : grouping.getDimensions()) {
Granularity granularity = Expressions.toQueryGranularity(
dimensionExpression.getDruidExpression(),
plannerContext.getExprMacroTable()
plannerContext.getExpressionParser()
);
if (granularity == null || !canUseQueryGranularity(dataSource, filtration, granularity)) {
// Can't, or won't, convert this dimension to a query granularity.

View File

@ -27,7 +27,6 @@ import org.apache.calcite.plan.RelOptUtil;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.util.ImmutableBitSet;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.dimension.DimensionSpec;
@ -235,8 +234,8 @@ public class Grouping
for (int i = 0; i < dimensions.size(); i++) {
final DimensionExpression dimension = dimensions.get(i);
if (Parser.parse(dimension.getDruidExpression().getExpression(), plannerContext.getExprMacroTable())
.isLiteral() && !aggregateProjectBits.get(i)) {
if (plannerContext.parseExpression(dimension.getDruidExpression().getExpression()).isLiteral()
&& !aggregateProjectBits.get(i)) {
droppedDimensions = true;
newDimIndexes[i] = -1;
} else {

View File

@ -175,7 +175,7 @@ public class Projection
postAggregatorVisitor.getOutputNamePrefix() + postAggregatorVisitor.getAndIncrementCounter(),
postAggregatorExpression.getExpression(),
null,
plannerContext.getExprMacroTable()
plannerContext.parseExpression(postAggregatorExpression.getExpression())
);
postAggregatorVisitor.addPostAgg(postAggregator);
rowOrder.add(postAggregator.getName());

View File

@ -21,13 +21,13 @@ package org.apache.druid.sql.calcite.rel;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.ExpressionParser;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import javax.annotation.Nullable;
@ -47,7 +47,7 @@ import java.util.stream.Collectors;
*/
public class VirtualColumnRegistry
{
private final ExprMacroTable macroTable;
private final ExpressionParser expressionParser;
private final RowSignature baseRowSignature;
private final Map<ExpressionAndTypeHint, String> virtualColumnsByExpression;
private final Map<String, ExpressionAndTypeHint> virtualColumnsByName;
@ -57,14 +57,14 @@ public class VirtualColumnRegistry
private VirtualColumnRegistry(
RowSignature baseRowSignature,
ExprMacroTable macroTable,
ExpressionParser expressionParser,
String virtualColumnPrefix,
boolean forceExpressionVirtualColumns,
Map<ExpressionAndTypeHint, String> virtualColumnsByExpression,
Map<String, ExpressionAndTypeHint> virtualColumnsByName
)
{
this.macroTable = macroTable;
this.expressionParser = expressionParser;
this.baseRowSignature = baseRowSignature;
this.virtualColumnPrefix = virtualColumnPrefix;
this.virtualColumnsByExpression = virtualColumnsByExpression;
@ -74,13 +74,13 @@ public class VirtualColumnRegistry
public static VirtualColumnRegistry create(
final RowSignature rowSignature,
final ExprMacroTable macroTable,
final ExpressionParser expressionParser,
final boolean forceExpressionVirtualColumns
)
{
return new VirtualColumnRegistry(
rowSignature,
macroTable,
expressionParser,
Calcites.findUnusedPrefixForDigits("v", rowSignature.getColumnNames()),
forceExpressionVirtualColumns,
new HashMap<>(),
@ -158,8 +158,8 @@ public class VirtualColumnRegistry
DruidExpression expression = registeredColumn.getExpression();
ColumnType columnType = registeredColumn.getTypeHint();
return forceExpressionVirtualColumns
? expression.toExpressionVirtualColumn(virtualColumnName, columnType, macroTable)
: expression.toVirtualColumn(virtualColumnName, columnType, macroTable);
? expression.toExpressionVirtualColumn(virtualColumnName, columnType, expressionParser)
: expression.toVirtualColumn(virtualColumnName, columnType, expressionParser);
}
@Nullable
@ -187,7 +187,7 @@ public class VirtualColumnRegistry
final ColumnCapabilities virtualCapabilities = virtualColumn.getValue().getExpression().toVirtualColumn(
columnName,
typeHint,
macroTable
expressionParser
).capabilities(baseSignature, columnName);
// fall back to type hint
@ -300,7 +300,7 @@ public class VirtualColumnRegistry
* Wrapper class for a {@link DruidExpression} and the output {@link ColumnType} "hint" that callers can specify when
* adding a virtual column with {@link #getOrCreateVirtualColumnForExpression(DruidExpression, RelDataType)} or
* {@link #getOrCreateVirtualColumnForExpression(DruidExpression, ColumnType)}. This "hint" will be passed into
* {@link DruidExpression#toVirtualColumn(String, ColumnType, ExprMacroTable)}.
* {@link DruidExpression#toVirtualColumn(String, ColumnType, ExpressionParser)}.
*
* The type hint might be different than {@link DruidExpression#getDruidType()} since that value is the captured value
* of {@link org.apache.calcite.rex.RexNode#getType()} converted to the Druid type system, while callers might still

View File

@ -154,7 +154,7 @@ public class DruidUnnestRule extends RelOptRule
}
// Evaluate the expression. It's a constant, so no bindings are needed.
final Expr parsedExpression = expression.parse(plannerContext.getExprMacroTable());
final Expr parsedExpression = plannerContext.parseExpression(expression.getExpression());
final ExprEval<?> eval = parsedExpression.eval(InputBindings.nilBindings());
final List<Object[]> rows = new ArrayList<>();

View File

@ -35,8 +35,6 @@ import org.apache.druid.data.input.MapBasedRow;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.query.expression.TestExprMacroTable;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.filter.ValueMatcher;
import org.apache.druid.segment.RowAdapters;
@ -51,6 +49,7 @@ import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidTypeSystem;
import org.apache.druid.sql.calcite.planner.ExpressionParserImpl;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.planner.PlannerToolbox;
@ -336,8 +335,9 @@ class ExpressionTestHelper
Assert.assertEquals("Expression for: " + rexNode, expectedExpression.getExpression(), expression.getExpression());
}
ExprEval<?> result = Parser.parse(expression.getExpression(), PLANNER_CONTEXT.getExprMacroTable())
.eval(expressionBindings);
ExprEval<?> result = PLANNER_CONTEXT.parseExpression(expression.getExpression())
.eval(expressionBindings);
Assert.assertEquals("Result for: " + rexNode, expectedResult, result.value());
}
@ -351,7 +351,11 @@ class ExpressionTestHelper
)
{
final RexNode rexNode = rexBuilder.makeCall(op, exprs);
final VirtualColumnRegistry virtualColumnRegistry = VirtualColumnRegistry.create(rowSignature, TestExprMacroTable.INSTANCE, false);
final VirtualColumnRegistry virtualColumnRegistry = VirtualColumnRegistry.create(
rowSignature,
new ExpressionParserImpl(PLANNER_TOOLBOX.exprMacroTable()),
false
);
final DimFilter filter = Expressions.toFilter(PLANNER_CONTEXT, rowSignature, virtualColumnRegistry, rexNode);
Assert.assertEquals("Filter for: " + rexNode, expectedFilter, filter);

View File

@ -35,6 +35,7 @@ import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.join.JoinType;
import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.planner.ExpressionParserImpl;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.joda.time.Interval;
import org.junit.Assert;
@ -64,7 +65,11 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource,
selectorFilter,
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
VirtualColumnRegistry.create(
RowSignature.empty(),
new ExpressionParserImpl(TestExprMacroTable.INSTANCE),
false
),
CalciteTests.createJoinableFactoryWrapper()
);
verify(pair, dataSource, selectorFilter, Intervals.ETERNITY);
@ -77,7 +82,11 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource,
filterWithInterval,
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
VirtualColumnRegistry.create(
RowSignature.empty(),
new ExpressionParserImpl(TestExprMacroTable.INSTANCE),
false
),
CalciteTests.createJoinableFactoryWrapper()
);
verify(pair, dataSource, selectorFilter, Intervals.utc(100, 200));
@ -90,7 +99,11 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource,
filterWithInterval,
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
VirtualColumnRegistry.create(
RowSignature.empty(),
new ExpressionParserImpl(TestExprMacroTable.INSTANCE),
false
),
CalciteTests.createJoinableFactoryWrapper()
);
verify(pair, dataSource, selectorFilter, Intervals.utc(100, 200));
@ -104,7 +117,11 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource,
otherFilter,
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
VirtualColumnRegistry.create(
RowSignature.empty(),
new ExpressionParserImpl(TestExprMacroTable.INSTANCE),
false
),
CalciteTests.createJoinableFactoryWrapper()
);
verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200));
@ -118,7 +135,11 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource,
otherFilter,
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
VirtualColumnRegistry.create(
RowSignature.empty(),
new ExpressionParserImpl(TestExprMacroTable.INSTANCE),
false
),
CalciteTests.createJoinableFactoryWrapper()
);
verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200));
@ -132,7 +153,11 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource,
otherFilter,
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
VirtualColumnRegistry.create(
RowSignature.empty(),
new ExpressionParserImpl(TestExprMacroTable.INSTANCE),
false
),
CalciteTests.createJoinableFactoryWrapper()
);
verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200));
@ -146,7 +171,11 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource,
otherFilter,
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
VirtualColumnRegistry.create(
RowSignature.empty(),
new ExpressionParserImpl(TestExprMacroTable.INSTANCE),
false
),
CalciteTests.createJoinableFactoryWrapper()
);
verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200));
@ -165,7 +194,11 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource,
queryFilter,
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
VirtualColumnRegistry.create(
RowSignature.empty(),
new ExpressionParserImpl(TestExprMacroTable.INSTANCE),
false
),
CalciteTests.createJoinableFactoryWrapper()
);
verify(pair, expectedDataSource, otherFilter, Intervals.utc(150, 200));