mirror of https://github.com/apache/druid.git
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:
parent
4bd6bd0d4f
commit
c78d885b80
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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<>();
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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())) {
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -303,7 +303,7 @@ public class OperatorConversions
|
|||
postAggregatorVisitor.getOutputNamePrefix() + postAggregatorVisitor.getAndIncrementCounter(),
|
||||
druidExpression.getExpression(),
|
||||
null,
|
||||
plannerContext.getExprMacroTable()
|
||||
plannerContext.parseExpression(druidExpression.getExpression())
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -50,7 +50,8 @@ public abstract class BaseExpressionDimFilterOperatorConversion extends DirectOp
|
|||
|
||||
return new ExpressionDimFilter(
|
||||
filterExpr,
|
||||
plannerContext.getExprMacroTable()
|
||||
plannerContext.parseExpression(filterExpr),
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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())) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
),
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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<>();
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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));
|
||||
|
|
Loading…
Reference in New Issue