mirror of https://github.com/apache/druid.git
Simplify bounds/range vs selectors/equality logic in SQL planning. (#14619)
* Simplify bounds/range vs selectors/equality logic in SQL planning. 1) Consolidate duplicate code related to Expressions#buildTimeFloorFilter. 2) Cleaner logic in Expressions#toSimpleLeafFilter: choose bounds vs range filter based solely on plannerContext.isUseBoundsAndSelectors, not also considering rhs kind. Use parsed rhs in both paths (except for numerics in the bound path). 3) Fix ArrayContains, ArrayOverlap to avoid equality filters when there is an extractionFn present. Fixes a bug introduced in #14612. * Avoid sending nonprimitives down the bound path.
This commit is contained in:
parent
01e9a39544
commit
c2e6758580
|
@ -28,6 +28,7 @@ import org.apache.calcite.rex.RexCall;
|
|||
import org.apache.calcite.rex.RexInputRef;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.rex.RexUtil;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
|
@ -519,18 +520,18 @@ public class Expressions
|
|||
// column instead for filtering to ensure that results are correct
|
||||
if (druidExpression.isSimpleExtraction() &&
|
||||
!(isOutputNumeric && !rowSignature.isNumeric(druidExpression.getDirectColumn()))) {
|
||||
if (!plannerContext.isUseBoundsAndSelectors()) {
|
||||
if (druidExpression.getSimpleExtraction().getExtractionFn() != null) {
|
||||
// return null to fallback to using an expression filter
|
||||
return null;
|
||||
}
|
||||
equalFilter = NullFilter.forColumn(druidExpression.getDirectColumn());
|
||||
} else {
|
||||
if (plannerContext.isUseBoundsAndSelectors()) {
|
||||
equalFilter = new SelectorDimFilter(
|
||||
druidExpression.getSimpleExtraction().getColumn(),
|
||||
NullHandling.defaultStringValue(),
|
||||
druidExpression.getSimpleExtraction().getExtractionFn()
|
||||
);
|
||||
} else {
|
||||
if (druidExpression.getSimpleExtraction().getExtractionFn() != null) {
|
||||
// return null to fallback to using an expression filter
|
||||
return null;
|
||||
}
|
||||
equalFilter = NullFilter.forColumn(druidExpression.getDirectColumn());
|
||||
}
|
||||
} else if (virtualColumnRegistry != null) {
|
||||
final String virtualColumn = virtualColumnRegistry.getOrCreateVirtualColumnForExpression(
|
||||
|
@ -538,14 +539,10 @@ public class Expressions
|
|||
operand.getType()
|
||||
);
|
||||
|
||||
if (!plannerContext.isUseBoundsAndSelectors()) {
|
||||
equalFilter = NullFilter.forColumn(virtualColumn);
|
||||
if (plannerContext.isUseBoundsAndSelectors()) {
|
||||
equalFilter = new SelectorDimFilter(virtualColumn, NullHandling.defaultStringValue(), null);
|
||||
} else {
|
||||
equalFilter = new SelectorDimFilter(
|
||||
virtualColumn,
|
||||
NullHandling.defaultStringValue(),
|
||||
null
|
||||
);
|
||||
equalFilter = NullFilter.forColumn(virtualColumn);
|
||||
}
|
||||
} else {
|
||||
return null;
|
||||
|
@ -601,11 +598,11 @@ public class Expressions
|
|||
}
|
||||
|
||||
final DruidExpression rhsExpression = toDruidExpression(plannerContext, rowSignature, rhs);
|
||||
final Expr parsedRhsExpression = rhsExpression != null
|
||||
final Expr rhsParsed = rhsExpression != null
|
||||
? plannerContext.parseExpression(rhsExpression.getExpression())
|
||||
: null;
|
||||
// rhs must be a literal
|
||||
if (rhs.getKind() != SqlKind.LITERAL && (parsedRhsExpression == null || !parsedRhsExpression.isLiteral())) {
|
||||
if (rhsParsed == null || !rhsParsed.isLiteral()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -651,43 +648,37 @@ public class Expressions
|
|||
|
||||
final Granularity granularity = ExtractionFns.toQueryGranularity(extractionFn);
|
||||
if (granularity != null) {
|
||||
// lhs is FLOOR(__time TO granularity); rhs must be a timestamp
|
||||
// lhs is FLOOR(__time TO granularity); rhs must be a timestamp.
|
||||
final long rhsMillis = Calcites.calciteDateTimeLiteralToJoda(rhs, plannerContext.getTimeZone()).getMillis();
|
||||
final Interval rhsInterval = granularity.bucket(DateTimes.utc(rhsMillis));
|
||||
|
||||
// Is rhs aligned on granularity boundaries?
|
||||
final boolean rhsAligned = rhsInterval.getStartMillis() == rhsMillis;
|
||||
|
||||
if (plannerContext.isUseBoundsAndSelectors()) {
|
||||
// Create a BoundRefKey that strips the extractionFn and compares __time as a number.
|
||||
final BoundRefKey boundRefKey = new BoundRefKey(column, null, StringComparators.NUMERIC);
|
||||
|
||||
return getBoundTimeDimFilter(flippedKind, boundRefKey, rhsInterval, rhsAligned);
|
||||
} else {
|
||||
final RangeRefKey rangeRefKey = new RangeRefKey(column, ColumnType.LONG);
|
||||
return getRangeTimeDimFilter(flippedKind, rangeRefKey, rhsInterval, rhsAligned);
|
||||
}
|
||||
return buildTimeFloorFilter(column, granularity, flippedKind, rhsMillis, plannerContext);
|
||||
}
|
||||
}
|
||||
|
||||
if (plannerContext.isUseBoundsAndSelectors() && rhs instanceof RexLiteral) {
|
||||
final String val;
|
||||
final RexLiteral rhsLiteral = (RexLiteral) rhs;
|
||||
if (SqlTypeName.NUMERIC_TYPES.contains(rhsLiteral.getTypeName())) {
|
||||
val = String.valueOf(RexLiteral.value(rhsLiteral));
|
||||
} else if (SqlTypeName.CHAR_TYPES.contains(rhsLiteral.getTypeName())) {
|
||||
val = String.valueOf(RexLiteral.stringValue(rhsLiteral));
|
||||
} else if (SqlTypeName.TIMESTAMP == rhsLiteral.getTypeName() || SqlTypeName.DATE == rhsLiteral.getTypeName()) {
|
||||
val = String.valueOf(
|
||||
Calcites.calciteDateTimeLiteralToJoda(
|
||||
rhsLiteral,
|
||||
plannerContext.getTimeZone()
|
||||
).getMillis()
|
||||
);
|
||||
} else {
|
||||
// Don't know how to filter on this kind of literal.
|
||||
final ColumnType matchValueType = Calcites.getColumnTypeForRelDataType(rhs.getType());
|
||||
|
||||
if (plannerContext.isUseBoundsAndSelectors()) {
|
||||
if (matchValueType == null || !matchValueType.isPrimitive()) {
|
||||
// Fall back to expression filter.
|
||||
return null;
|
||||
}
|
||||
|
||||
final String stringVal;
|
||||
|
||||
if (rhsParsed.getLiteralValue() == null) {
|
||||
stringVal = NullHandling.defaultStringValue();
|
||||
} else if (RexUtil.isLiteral(rhs, true) && SqlTypeName.NUMERIC_TYPES.contains(rhs.getType().getSqlTypeName())) {
|
||||
// Peek inside the original rhs for numerics, rather than using the parsed version, for highest fidelity
|
||||
// to what the query originally contained. (It may be a BigDecimal.)
|
||||
stringVal = String.valueOf(RexLiteral.value(rhs));
|
||||
} else {
|
||||
stringVal = String.valueOf(rhsParsed.getLiteralValue());
|
||||
}
|
||||
|
||||
if (stringVal == null) {
|
||||
// Fall back to expression filter.
|
||||
return null;
|
||||
}
|
||||
|
||||
// Numeric lhs needs a numeric comparison.
|
||||
final StringComparator comparator = Calcites.getStringComparatorForRelDataType(lhs.getType());
|
||||
final BoundRefKey boundRefKey = new BoundRefKey(column, extractionFn, comparator);
|
||||
|
@ -696,22 +687,22 @@ public class Expressions
|
|||
// Always use BoundDimFilters, to simplify filter optimization later (it helps to remember the comparator).
|
||||
switch (flippedKind) {
|
||||
case EQUALS:
|
||||
filter = Bounds.equalTo(boundRefKey, val);
|
||||
filter = Bounds.equalTo(boundRefKey, stringVal);
|
||||
break;
|
||||
case NOT_EQUALS:
|
||||
filter = new NotDimFilter(Bounds.equalTo(boundRefKey, val));
|
||||
filter = new NotDimFilter(Bounds.equalTo(boundRefKey, stringVal));
|
||||
break;
|
||||
case GREATER_THAN:
|
||||
filter = Bounds.greaterThan(boundRefKey, val);
|
||||
filter = Bounds.greaterThan(boundRefKey, stringVal);
|
||||
break;
|
||||
case GREATER_THAN_OR_EQUAL:
|
||||
filter = Bounds.greaterThanOrEqualTo(boundRefKey, val);
|
||||
filter = Bounds.greaterThanOrEqualTo(boundRefKey, stringVal);
|
||||
break;
|
||||
case LESS_THAN:
|
||||
filter = Bounds.lessThan(boundRefKey, val);
|
||||
filter = Bounds.lessThan(boundRefKey, stringVal);
|
||||
break;
|
||||
case LESS_THAN_OR_EQUAL:
|
||||
filter = Bounds.lessThanOrEqualTo(boundRefKey, val);
|
||||
filter = Bounds.lessThanOrEqualTo(boundRefKey, stringVal);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Shouldn't have got here");
|
||||
|
@ -719,20 +710,13 @@ public class Expressions
|
|||
|
||||
return filter;
|
||||
} else {
|
||||
//noinspection VariableNotUsedInsideIf
|
||||
if (extractionFn != null) {
|
||||
final Object val = rhsParsed.getLiteralValue();
|
||||
|
||||
if (extractionFn != null || val == null) {
|
||||
// fall back to expression filter
|
||||
return null;
|
||||
}
|
||||
final Object val;
|
||||
if (parsedRhsExpression != null && parsedRhsExpression.isLiteral()) {
|
||||
val = parsedRhsExpression.getLiteralValue();
|
||||
} else {
|
||||
// Don't know how to filter on this kind of literal.
|
||||
return null;
|
||||
}
|
||||
|
||||
final ColumnType matchValueType = Calcites.getColumnTypeForRelDataType(rhs.getType());
|
||||
final RangeRefKey rangeRefKey = new RangeRefKey(column, matchValueType);
|
||||
final DimFilter filter;
|
||||
|
||||
|
@ -850,26 +834,20 @@ public class Expressions
|
|||
final PlannerContext plannerContext
|
||||
)
|
||||
{
|
||||
final Interval rhsInterval = granularity.bucket(DateTimes.utc(rhsMillis));
|
||||
|
||||
// Is rhs aligned on granularity boundaries?
|
||||
final boolean rhsAligned = rhsInterval.getStartMillis() == rhsMillis;
|
||||
|
||||
if (plannerContext.isUseBoundsAndSelectors()) {
|
||||
final BoundRefKey boundRefKey = new BoundRefKey(column, null, StringComparators.NUMERIC);
|
||||
final Interval rhsInterval = granularity.bucket(DateTimes.utc(rhsMillis));
|
||||
|
||||
// Is rhs aligned on granularity boundaries?
|
||||
final boolean rhsAligned = rhsInterval.getStartMillis() == rhsMillis;
|
||||
|
||||
return getBoundTimeDimFilter(operatorKind, boundRefKey, rhsInterval, rhsAligned);
|
||||
} else {
|
||||
final RangeRefKey rangeRefKey = new RangeRefKey(column, ColumnType.LONG);
|
||||
final Interval rhsInterval = granularity.bucket(DateTimes.utc(rhsMillis));
|
||||
|
||||
// Is rhs aligned on granularity boundaries?
|
||||
final boolean rhsAligned = rhsInterval.getStartMillis() == rhsMillis;
|
||||
|
||||
return getRangeTimeDimFilter(operatorKind, rangeRefKey, rhsInterval, rhsAligned);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private static DimFilter getBoundTimeDimFilter(
|
||||
SqlKind operatorKind,
|
||||
BoundRefKey boundRefKey,
|
||||
|
|
|
@ -41,9 +41,8 @@ import org.apache.druid.sql.calcite.planner.PlannerContext;
|
|||
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class ArrayContainsOperatorConversion extends BaseExpressionDimFilterOperatorConversion
|
||||
{
|
||||
|
@ -98,7 +97,9 @@ public class ArrayContainsOperatorConversion extends BaseExpressionDimFilterOper
|
|||
|
||||
// if the input column is not actually an ARRAY type, but rather an MVD, we can optimize this into
|
||||
// selector/equality filters on the individual array elements
|
||||
if (leftExpr.isSimpleExtraction() && !(leftExpr.isDirectColumnAccess() && leftExpr.getDruidType() != null && leftExpr.getDruidType().isArray())) {
|
||||
if (leftExpr.isSimpleExtraction()
|
||||
&& !isArray(leftExpr)
|
||||
&& (plannerContext.isUseBoundsAndSelectors() || leftExpr.isDirectColumnAccess())) {
|
||||
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
|
||||
|
@ -115,37 +116,34 @@ public class ArrayContainsOperatorConversion extends BaseExpressionDimFilterOper
|
|||
// However, since both Calcite's SqlMultisetValueConstructor and Druid's ArrayConstructorFunction don't allow
|
||||
// to create an empty array with no argument, we just return null.
|
||||
return null;
|
||||
} else if (arrayElements.length == 1) {
|
||||
if (plannerContext.isUseBoundsAndSelectors()) {
|
||||
return newSelectorDimFilter(leftExpr.getSimpleExtraction(), Evals.asString(arrayElements[0]));
|
||||
} else {
|
||||
return new EqualityFilter(
|
||||
leftExpr.getSimpleExtraction().getColumn(),
|
||||
ExpressionType.toColumnType(ExpressionType.elementType(exprEval.type())),
|
||||
arrayElements[0],
|
||||
null
|
||||
);
|
||||
}
|
||||
} else {
|
||||
final List<DimFilter> selectFilters = Arrays
|
||||
.stream(arrayElements)
|
||||
.map(val -> {
|
||||
if (plannerContext.isUseBoundsAndSelectors()) {
|
||||
return newSelectorDimFilter(leftExpr.getSimpleExtraction(), Evals.asString(val));
|
||||
} else {
|
||||
return new EqualityFilter(
|
||||
final List<DimFilter> filters = new ArrayList<>(arrayElements.length);
|
||||
|
||||
for (final Object val : arrayElements) {
|
||||
if (plannerContext.isUseBoundsAndSelectors()) {
|
||||
filters.add(newSelectorDimFilter(leftExpr.getSimpleExtraction(), Evals.asString(val)));
|
||||
} else {
|
||||
// Cannot handle extractionFn here. We won't get one due to the isDirectColumnAccess check above.
|
||||
filters.add(
|
||||
new EqualityFilter(
|
||||
leftExpr.getSimpleExtraction().getColumn(),
|
||||
ExpressionType.toColumnType(ExpressionType.elementType(exprEval.type())),
|
||||
val,
|
||||
null
|
||||
);
|
||||
}
|
||||
})
|
||||
.collect(Collectors.toList());
|
||||
return new AndDimFilter(selectFilters);
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
return filters.size() == 1 ? filters.get(0) : new AndDimFilter(filters);
|
||||
}
|
||||
}
|
||||
}
|
||||
return toExpressionFilter(plannerContext, getDruidFunctionName(), druidExpressions);
|
||||
}
|
||||
|
||||
private static boolean isArray(final DruidExpression expr)
|
||||
{
|
||||
return expr.getDruidType() != null && expr.getDruidType().isArray();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -114,7 +114,7 @@ public class ArrayOverlapOperatorConversion extends BaseExpressionDimFilterOpera
|
|||
}
|
||||
|
||||
Expr expr = plannerContext.parseExpression(complexExpr.getExpression());
|
||||
if (expr.isLiteral()) {
|
||||
if (expr.isLiteral() && (plannerContext.isUseBoundsAndSelectors() || simpleExtractionExpr.isDirectColumnAccess())) {
|
||||
// Evaluate the expression to take out the array elements.
|
||||
// We can safely pass null if the expression is literal.
|
||||
ExprEval<?> exprEval = expr.eval(InputBindings.nilBindings());
|
||||
|
@ -129,6 +129,7 @@ public class ArrayOverlapOperatorConversion extends BaseExpressionDimFilterOpera
|
|||
if (plannerContext.isUseBoundsAndSelectors()) {
|
||||
return newSelectorDimFilter(simpleExtractionExpr.getSimpleExtraction(), Evals.asString(arrayElements[0]));
|
||||
} else {
|
||||
// Cannot handle extractionFn here. We won't get one due to the isDirectColumnAccess check above.
|
||||
return new EqualityFilter(
|
||||
simpleExtractionExpr.getSimpleExtraction().getColumn(),
|
||||
ExpressionType.toColumnType(exprEval.type()),
|
||||
|
|
|
@ -119,11 +119,7 @@ public class CaseOperatorConversion implements SqlOperatorConversion
|
|||
DimFilter thenFilter = null, elseFilter = null;
|
||||
final DimFilter isNull;
|
||||
if (plannerContext.isUseBoundsAndSelectors()) {
|
||||
isNull = new SelectorDimFilter(
|
||||
condition.getArguments().get(0).getDirectColumn(),
|
||||
null,
|
||||
null
|
||||
);
|
||||
isNull = new SelectorDimFilter(condition.getArguments().get(0).getDirectColumn(), null, null);
|
||||
} else {
|
||||
isNull = NullFilter.forColumn(condition.getArguments().get(0).getDirectColumn());
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.druid.java.util.common.ISE;
|
|||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.math.expr.ExpressionProcessing;
|
||||
import org.apache.druid.math.expr.ExpressionProcessingConfig;
|
||||
import org.apache.druid.query.ordering.StringComparator;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
@ -132,9 +133,15 @@ public class Calcites
|
|||
}
|
||||
|
||||
/**
|
||||
* Convert {@link RelDataType} to the most appropriate {@link ValueType}
|
||||
* Caller who want to coerce all ARRAY types to STRING can set `druid.expressions.allowArrayToStringCast`
|
||||
* runtime property in {@link org.apache.druid.math.expr.ExpressionProcessingConfig}
|
||||
* Convert {@link RelDataType} to the most appropriate {@link ColumnType}, or null if there is no {@link ColumnType}
|
||||
* that is appropriate for this {@link RelDataType}.
|
||||
*
|
||||
* Equivalent to {@link #getValueTypeForRelDataTypeFull(RelDataType)}, except this method returns
|
||||
* {@link ColumnType#STRING} when {@link ColumnType#isArray()} if
|
||||
* {@link ExpressionProcessingConfig#processArraysAsMultiValueStrings()} is set via the server property
|
||||
* {@code druid.expressions.allowArrayToStringCast}.
|
||||
*
|
||||
* @return type, or null if there is no matching type
|
||||
*/
|
||||
@Nullable
|
||||
public static ColumnType getColumnTypeForRelDataType(final RelDataType type)
|
||||
|
@ -148,7 +155,13 @@ public class Calcites
|
|||
}
|
||||
|
||||
/**
|
||||
* Convert {@link RelDataType} to the most appropriate {@link ValueType}
|
||||
* Convert {@link RelDataType} to the most appropriate {@link ColumnType}, or null if there is no {@link ColumnType}
|
||||
* that is appropriate for this {@link RelDataType}.
|
||||
*
|
||||
* Equivalent to {@link #getColumnTypeForRelDataType(RelDataType)}, but ignores
|
||||
* {@link ExpressionProcessingConfig#processArraysAsMultiValueStrings()} (and acts as if it is false).
|
||||
*
|
||||
* @return type, or null if there is no matching type
|
||||
*/
|
||||
@Nullable
|
||||
public static ColumnType getValueTypeForRelDataTypeFull(final RelDataType type)
|
||||
|
|
|
@ -316,6 +316,14 @@ public class PlannerContext
|
|||
return stringifyArrays;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether we should use {@link org.apache.druid.query.filter.BoundDimFilter} and
|
||||
* {@link org.apache.druid.query.filter.SelectorDimFilter} (true) or {@link org.apache.druid.query.filter.RangeFilter},
|
||||
* {@link org.apache.druid.query.filter.EqualityFilter}, and {@link org.apache.druid.query.filter.NullFilter} (false).
|
||||
*
|
||||
* Typically true when {@link NullHandling#replaceWithDefault()} and false when {@link NullHandling#sqlCompatible()}.
|
||||
* Can be overriden by the undocumented context parameter {@link #CTX_SQL_USE_BOUNDS_AND_SELECTORS}.
|
||||
*/
|
||||
public boolean isUseBoundsAndSelectors()
|
||||
{
|
||||
return useBoundsAndSelectors;
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
|
|||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.query.extraction.SubstringDimExtractionFn;
|
||||
import org.apache.druid.query.filter.ExpressionDimFilter;
|
||||
import org.apache.druid.query.filter.InDimFilter;
|
||||
import org.apache.druid.query.filter.LikeDimFilter;
|
||||
|
@ -517,6 +518,33 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArrayOverlapFilterWithExtractionFn()
|
||||
{
|
||||
testQuery(
|
||||
"SELECT dim3 FROM druid.numfoo WHERE ARRAY_OVERLAP(SUBSTRING(dim3, 1, 1), ARRAY['a','b']) LIMIT 5",
|
||||
ImmutableList.of(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.filters(
|
||||
NullHandling.sqlCompatible()
|
||||
? expressionFilter("array_overlap(substring(\"dim3\", 0, 1),array('a','b'))")
|
||||
: new InDimFilter("dim3", ImmutableList.of("a", "b"), new SubstringDimExtractionFn(0, 1))
|
||||
)
|
||||
.columns("dim3")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.limit(5)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"[\"a\",\"b\"]"},
|
||||
new Object[]{"[\"b\",\"c\"]"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArrayOverlapFilterNonLiteral()
|
||||
{
|
||||
|
@ -566,6 +594,35 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArrayContainsFilterWithExtractionFn()
|
||||
{
|
||||
testQuery(
|
||||
"SELECT dim3 FROM druid.numfoo WHERE ARRAY_CONTAINS(SUBSTRING(dim3, 1, 1), ARRAY['a','b']) LIMIT 5",
|
||||
ImmutableList.of(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.filters(
|
||||
NullHandling.sqlCompatible()
|
||||
? expressionFilter("array_contains(substring(\"dim3\", 0, 1),array('a','b'))")
|
||||
: and(
|
||||
selector("dim3", "a", new SubstringDimExtractionFn(0, 1)),
|
||||
selector("dim3", "b", new SubstringDimExtractionFn(0, 1))
|
||||
)
|
||||
)
|
||||
.columns("dim3")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.limit(5)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"[\"a\",\"b\"]"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArrayContainsArrayOfOneElement()
|
||||
{
|
||||
|
|
Loading…
Reference in New Issue