mirror of https://github.com/apache/druid.git
expression language array constructor and sql multi-value string filtering support (#7973)
* expr array constructor and sql multi-value string support * doc fix * checkstyle * change from feedback
This commit is contained in:
parent
fb56f8d53c
commit
93b738bbfa
|
@ -100,17 +100,14 @@ public interface ApplyFunction
|
|||
}
|
||||
}
|
||||
|
||||
switch (elementType) {
|
||||
case STRING:
|
||||
stringsOut[i] = evaluated.asString();
|
||||
break;
|
||||
case LONG:
|
||||
longsOut[i] = evaluated.isNumericNull() ? null : evaluated.asLong();
|
||||
break;
|
||||
case DOUBLE:
|
||||
doublesOut[i] = evaluated.isNumericNull() ? null : evaluated.asDouble();
|
||||
break;
|
||||
}
|
||||
Function.ArrayConstructorFunction.setArrayOutputElement(
|
||||
stringsOut,
|
||||
longsOut,
|
||||
doublesOut,
|
||||
elementType,
|
||||
i,
|
||||
evaluated
|
||||
);
|
||||
}
|
||||
|
||||
switch (elementType) {
|
||||
|
@ -258,6 +255,9 @@ public interface ApplyFunction
|
|||
ExprEval evaluated = lambdaExpr.eval(bindings.accumulateWithIndex(i, accumulator));
|
||||
accumulator = evaluated.value();
|
||||
}
|
||||
if (accumulator instanceof Boolean) {
|
||||
return ExprEval.of((boolean) accumulator, ExprType.LONG);
|
||||
}
|
||||
return ExprEval.bestEffortOf(accumulator);
|
||||
}
|
||||
}
|
||||
|
@ -470,7 +470,7 @@ public interface ApplyFunction
|
|||
|
||||
final Object[] array = arrayEval.asArray();
|
||||
if (array == null) {
|
||||
return ExprEval.bestEffortOf(false);
|
||||
return ExprEval.of(false, ExprType.LONG);
|
||||
}
|
||||
|
||||
SettableLambdaBinding lambdaBinding = new SettableLambdaBinding(lambdaExpr, bindings);
|
||||
|
@ -519,7 +519,7 @@ public interface ApplyFunction
|
|||
{
|
||||
boolean anyMatch = Arrays.stream(values)
|
||||
.anyMatch(o -> expr.eval(bindings.withBinding(expr.getIdentifier(), o)).asBoolean());
|
||||
return ExprEval.bestEffortOf(anyMatch);
|
||||
return ExprEval.of(anyMatch, ExprType.LONG);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -542,7 +542,7 @@ public interface ApplyFunction
|
|||
{
|
||||
boolean allMatch = Arrays.stream(values)
|
||||
.allMatch(o -> expr.eval(bindings.withBinding(expr.getIdentifier(), o)).asBoolean());
|
||||
return ExprEval.bestEffortOf(allMatch);
|
||||
return ExprEval.of(allMatch, ExprType.LONG);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -1724,6 +1724,104 @@ interface Function
|
|||
}
|
||||
}
|
||||
|
||||
class ArrayConstructorFunction implements Function
|
||||
{
|
||||
@Override
|
||||
public String name()
|
||||
{
|
||||
return "array";
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings)
|
||||
{
|
||||
// this is copied from 'BaseMapFunction.applyMap', need to find a better way to consolidate, or construct arrays,
|
||||
// or.. something...
|
||||
final int length = args.size();
|
||||
String[] stringsOut = null;
|
||||
Long[] longsOut = null;
|
||||
Double[] doublesOut = null;
|
||||
|
||||
ExprType elementType = null;
|
||||
for (int i = 0; i < length; i++) {
|
||||
|
||||
ExprEval evaluated = args.get(i).eval(bindings);
|
||||
if (elementType == null) {
|
||||
elementType = evaluated.type();
|
||||
switch (elementType) {
|
||||
case STRING:
|
||||
stringsOut = new String[length];
|
||||
break;
|
||||
case LONG:
|
||||
longsOut = new Long[length];
|
||||
break;
|
||||
case DOUBLE:
|
||||
doublesOut = new Double[length];
|
||||
break;
|
||||
default:
|
||||
throw new RE("Unhandled array constructor element type [%s]", elementType);
|
||||
}
|
||||
}
|
||||
|
||||
setArrayOutputElement(stringsOut, longsOut, doublesOut, elementType, i, evaluated);
|
||||
}
|
||||
|
||||
switch (elementType) {
|
||||
case STRING:
|
||||
return ExprEval.ofStringArray(stringsOut);
|
||||
case LONG:
|
||||
return ExprEval.ofLongArray(longsOut);
|
||||
case DOUBLE:
|
||||
return ExprEval.ofDoubleArray(doublesOut);
|
||||
default:
|
||||
throw new RE("Unhandled array constructor element type [%s]", elementType);
|
||||
}
|
||||
}
|
||||
|
||||
static void setArrayOutputElement(
|
||||
String[] stringsOut,
|
||||
Long[] longsOut,
|
||||
Double[] doublesOut,
|
||||
ExprType elementType,
|
||||
int i,
|
||||
ExprEval evaluated
|
||||
)
|
||||
{
|
||||
switch (elementType) {
|
||||
case STRING:
|
||||
stringsOut[i] = evaluated.asString();
|
||||
break;
|
||||
case LONG:
|
||||
longsOut[i] = evaluated.isNumericNull() ? null : evaluated.asLong();
|
||||
break;
|
||||
case DOUBLE:
|
||||
doublesOut[i] = evaluated.isNumericNull() ? null : evaluated.asDouble();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Set<Expr> getArrayInputs(List<Expr> args)
|
||||
{
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void validateArguments(List<Expr> args)
|
||||
{
|
||||
if (!(args.size() > 0)) {
|
||||
throw new IAE("Function[%s] needs at least 1 argument", name());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Expr> getScalarInputs(List<Expr> args)
|
||||
{
|
||||
return ImmutableSet.copyOf(args);
|
||||
}
|
||||
}
|
||||
|
||||
class ArrayLengthFunction implements Function
|
||||
{
|
||||
@Override
|
||||
|
@ -2038,7 +2136,7 @@ interface Function
|
|||
{
|
||||
final Object[] array1 = lhsExpr.asArray();
|
||||
final Object[] array2 = rhsExpr.asArray();
|
||||
return ExprEval.bestEffortOf(Arrays.asList(array1).containsAll(Arrays.asList(array2)));
|
||||
return ExprEval.of(Arrays.asList(array1).containsAll(Arrays.asList(array2)), ExprType.LONG);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2059,7 +2157,7 @@ interface Function
|
|||
for (Object check : array1) {
|
||||
any |= array2.contains(check);
|
||||
}
|
||||
return ExprEval.bestEffortOf(any);
|
||||
return ExprEval.of(any, ExprType.LONG);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -113,19 +113,19 @@ public class ApplyFunctionTest
|
|||
@Test
|
||||
public void testAnyMatch()
|
||||
{
|
||||
assertExpr("any(x -> x > 3, [1, 2, 3, 4])", "true");
|
||||
assertExpr("any(x -> x > 3, [1, 2, 3])", "false");
|
||||
assertExpr("any(x -> x, map(x -> x > 3, [1, 2, 3, 4]))", "true");
|
||||
assertExpr("any(x -> x, map(x -> x > 3, [1, 2, 3]))", "false");
|
||||
assertExpr("any(x -> x > 3, [1, 2, 3, 4])", 1L);
|
||||
assertExpr("any(x -> x > 3, [1, 2, 3])", 0L);
|
||||
assertExpr("any(x -> x, map(x -> x > 3, [1, 2, 3, 4]))", 1L);
|
||||
assertExpr("any(x -> x, map(x -> x > 3, [1, 2, 3]))", 0L);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAllMatch()
|
||||
{
|
||||
assertExpr("all(x -> x > 0, [1, 2, 3, 4])", "true");
|
||||
assertExpr("all(x -> x > 1, [1, 2, 3, 4])", "false");
|
||||
assertExpr("all(x -> x, map(x -> x > 0, [1, 2, 3, 4]))", "true");
|
||||
assertExpr("all(x -> x, map(x -> x > 1, [1, 2, 3, 4]))", "false");
|
||||
assertExpr("all(x -> x > 0, [1, 2, 3, 4])", 1L);
|
||||
assertExpr("all(x -> x > 1, [1, 2, 3, 4])", 0L);
|
||||
assertExpr("all(x -> x, map(x -> x > 0, [1, 2, 3, 4]))", 1L);
|
||||
assertExpr("all(x -> x, map(x -> x > 1, [1, 2, 3, 4]))", 0L);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -159,6 +159,15 @@ public class FunctionTest
|
|||
assertExpr("rpad(null, 5, x)", null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArrayConstructor()
|
||||
{
|
||||
assertExpr("array(1, 2, 3, 4)", new Long[]{1L, 2L, 3L, 4L});
|
||||
assertExpr("array(1, 2, 3, 'bar')", new Long[]{1L, 2L, 3L, null});
|
||||
assertExpr("array(1.0)", new Double[]{1.0});
|
||||
assertExpr("array('foo', 'bar')", new String[]{"foo", "bar"});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArrayLength()
|
||||
{
|
||||
|
@ -201,18 +210,18 @@ public class FunctionTest
|
|||
@Test
|
||||
public void testArrayContains()
|
||||
{
|
||||
assertExpr("array_contains([1, 2, 3], 2)", "true");
|
||||
assertExpr("array_contains([1, 2, 3], 4)", "false");
|
||||
assertExpr("array_contains([1, 2, 3], [2, 3])", "true");
|
||||
assertExpr("array_contains([1, 2, 3], [3, 4])", "false");
|
||||
assertExpr("array_contains(b, [3, 4])", "true");
|
||||
assertExpr("array_contains([1, 2, 3], 2)", 1L);
|
||||
assertExpr("array_contains([1, 2, 3], 4)", 0L);
|
||||
assertExpr("array_contains([1, 2, 3], [2, 3])", 1L);
|
||||
assertExpr("array_contains([1, 2, 3], [3, 4])", 0L);
|
||||
assertExpr("array_contains(b, [3, 4])", 1L);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArrayOverlap()
|
||||
{
|
||||
assertExpr("array_overlap([1, 2, 3], [2, 4, 6])", "true");
|
||||
assertExpr("array_overlap([1, 2, 3], [4, 5, 6])", "false");
|
||||
assertExpr("array_overlap([1, 2, 3], [2, 4, 6])", 1L);
|
||||
assertExpr("array_overlap([1, 2, 3], [4, 5, 6])", 0L);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -168,11 +168,12 @@ See javadoc of java.lang.Math for detailed explanation for each function.
|
|||
|
||||
| function | description |
|
||||
| --- | --- |
|
||||
| `array(expr1,expr ...)` | constructs an array from the expression arguments, using the type of the first argument as the output array type |
|
||||
| `array_length(arr)` | returns length of array expression |
|
||||
| `array_offset(arr,long)` | returns the array element at the 0 based index supplied, or null for an out of range index|
|
||||
| `array_ordinal(arr,long)` | returns the array element at the 1 based index supplied, or null for an out of range index |
|
||||
| `array_contains(arr,expr)` | returns true if the array contains the element specified by expr, or contains all elements specified by expr if expr is an array |
|
||||
| `array_overlap(arr1,arr2)` | returns true if arr1 and arr2 have any elements in common |
|
||||
| `array_contains(arr,expr)` | returns 1 if the array contains the element specified by expr, or contains all elements specified by expr if expr is an array, else 0 |
|
||||
| `array_overlap(arr1,arr2)` | returns 1 if arr1 and arr2 have any elements in common, else 0 |
|
||||
| `array_offset_of(arr,expr)` | returns the 0 based index of the first occurrence of expr in the array, or `null` if no matching elements exist in the array. |
|
||||
| `array_ordinal_of(arr,expr)` | returns the 1 based index of the first occurrence of expr in the array, or `null` if no matching elements exist in the array. |
|
||||
| `array_append(arr1,expr)` | appends expr to arr, the resulting array type determined by the type of the first array |
|
||||
|
@ -192,5 +193,5 @@ See javadoc of java.lang.Math for detailed explanation for each function.
|
|||
| `filter(lambda,arr)` | filters arr by a single argument lambda, returning a new array with all matching elements, or null if no elements match |
|
||||
| `fold(lambda,arr)` | folds a 2 argument lambda across arr. The first argument of the lambda is the array element and the second the accumulator, returning a single accumulated value. |
|
||||
| `cartesian_fold(lambda,arr1,arr2,...)` | folds a multi argument lambda across the cartesian product of all input arrays. The first arguments of the lambda is the array element and the last is the accumulator, returning a single accumulated value. |
|
||||
| `any(lambda,arr)` | returns true if any element in the array matches the lambda expression |
|
||||
| `all(lambda,arr)` | returns true if all elements in the array matches the lambda expression |
|
||||
| `any(lambda,arr)` | returns 1 if any element in the array matches the lambda expression, else 0 |
|
||||
| `all(lambda,arr)` | returns 1 if all elements in the array matches the lambda expression, else 0 |
|
||||
|
|
|
@ -55,4 +55,9 @@ public class DirectOperatorConversion implements SqlOperatorConversion
|
|||
operands -> DruidExpression.fromExpression(DruidExpression.functionCall(druidFunctionName, operands))
|
||||
);
|
||||
}
|
||||
|
||||
public String getDruidFunctionName()
|
||||
{
|
||||
return druidFunctionName;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,58 @@
|
|||
/*
|
||||
* 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.expression.builtin;
|
||||
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
|
||||
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
||||
import org.apache.druid.sql.calcite.expression.OperatorConversions;
|
||||
import org.apache.druid.sql.calcite.expression.SqlOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.table.RowSignature;
|
||||
|
||||
public class ArrayConstructorOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final SqlOperator SQL_FUNCTION = SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR;
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.of(
|
||||
null,
|
||||
DruidExpression.functionCall("array", druidExpressions)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* 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.expression.builtin;
|
||||
|
||||
import org.apache.calcite.sql.SqlFunction;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.ReturnTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.druid.sql.calcite.expression.OperatorConversions;
|
||||
|
||||
public class ArrayContainsOperatorConversion extends BaseExpressionDimFilterOperatorConversion
|
||||
{
|
||||
private static final String exprFunction = "array_contains";
|
||||
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("ARRAY_CONTAINS")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.sequence(
|
||||
"(array,array)",
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.ARRAY),
|
||||
OperandTypes.family(SqlTypeFamily.STRING)
|
||||
),
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.ARRAY),
|
||||
OperandTypes.family(SqlTypeFamily.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
.returnTypeInference(ReturnTypes.BOOLEAN)
|
||||
.build();
|
||||
|
||||
public ArrayContainsOperatorConversion()
|
||||
{
|
||||
super(SQL_FUNCTION, exprFunction);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* 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.expression.builtin;
|
||||
|
||||
import org.apache.calcite.sql.SqlFunction;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.ReturnTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.druid.sql.calcite.expression.OperatorConversions;
|
||||
|
||||
public class ArrayOverlapOperatorConversion extends BaseExpressionDimFilterOperatorConversion
|
||||
{
|
||||
private static final String exprFunction = "array_overlap";
|
||||
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("ARRAY_OVERLAP")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.sequence(
|
||||
"(array,array)",
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.ARRAY),
|
||||
OperandTypes.family(SqlTypeFamily.STRING)
|
||||
),
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.ARRAY),
|
||||
OperandTypes.family(SqlTypeFamily.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
.returnTypeInference(ReturnTypes.BOOLEAN)
|
||||
.build();
|
||||
|
||||
public ArrayOverlapOperatorConversion()
|
||||
{
|
||||
super(SQL_FUNCTION, exprFunction);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,70 @@
|
|||
/*
|
||||
* 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.expression.builtin;
|
||||
|
||||
import org.apache.calcite.rex.RexCall;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.filter.ExpressionDimFilter;
|
||||
import org.apache.druid.sql.calcite.expression.DirectOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
||||
import org.apache.druid.sql.calcite.expression.Expressions;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
|
||||
import org.apache.druid.sql.calcite.table.RowSignature;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
|
||||
public abstract class BaseExpressionDimFilterOperatorConversion extends DirectOperatorConversion
|
||||
{
|
||||
public BaseExpressionDimFilterOperatorConversion(
|
||||
SqlOperator operator,
|
||||
String druidFunctionName
|
||||
)
|
||||
{
|
||||
super(operator, druidFunctionName);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public DimFilter toDruidFilter(
|
||||
final PlannerContext plannerContext,
|
||||
RowSignature rowSignature,
|
||||
@Nullable VirtualColumnRegistry virtualColumnRegistry,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
final List<RexNode> operands = ((RexCall) rexNode).getOperands();
|
||||
final List<DruidExpression> druidExpressions = Expressions.toDruidExpressions(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
operands
|
||||
);
|
||||
final String filterExpr = DruidExpression.functionCall(getDruidFunctionName(), druidExpressions);
|
||||
|
||||
ExpressionDimFilter expressionDimFilter = new ExpressionDimFilter(
|
||||
filterExpr,
|
||||
plannerContext.getExprMacroTable()
|
||||
);
|
||||
return expressionDimFilter;
|
||||
}
|
||||
}
|
|
@ -151,6 +151,9 @@ public class Calcites
|
|||
return ValueType.STRING;
|
||||
} else if (SqlTypeName.OTHER == sqlTypeName) {
|
||||
return ValueType.COMPLEX;
|
||||
} else if (sqlTypeName == SqlTypeName.ARRAY) {
|
||||
// until we have array ValueType, this will let us have array constants and use them at least
|
||||
return ValueType.STRING;
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -46,6 +46,9 @@ import org.apache.druid.sql.calcite.expression.SqlOperatorConversion;
|
|||
import org.apache.druid.sql.calcite.expression.UnaryFunctionOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.UnaryPrefixOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.UnarySuffixOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.ArrayConstructorOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.ArrayContainsOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.ArrayOverlapOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.BTrimOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.CastOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.CeilOperatorConversion;
|
||||
|
@ -203,6 +206,12 @@ public class DruidOperatorTable implements SqlOperatorTable
|
|||
// value coercion operators
|
||||
.add(new CastOperatorConversion())
|
||||
.add(new ReinterpretOperatorConversion())
|
||||
// array and multi-value string operators
|
||||
.add(new ArrayConstructorOperatorConversion())
|
||||
.add(new ArrayContainsOperatorConversion())
|
||||
.add(new ArrayOverlapOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayContainsOperatorConversion(), "MV_CONTAINS"))
|
||||
.add(new AliasedOperatorConversion(new ArrayOverlapOperatorConversion(), "MV_OVERLAP"))
|
||||
.build();
|
||||
|
||||
// Operators that have no conversion, but are handled in the convertlet table, so they still need to exist.
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.druid.sql.calcite.expression.Expressions;
|
|||
import org.apache.druid.sql.calcite.table.RowSignature;
|
||||
|
||||
import java.math.BigDecimal;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
@ -122,6 +123,9 @@ public class DruidRexExecutor implements RexExecutor
|
|||
}
|
||||
|
||||
literal = rexBuilder.makeLiteral(bigDecimal, constExp.getType(), true);
|
||||
} else if (sqlTypeName == SqlTypeName.ARRAY) {
|
||||
assert exprResult.isArray();
|
||||
literal = rexBuilder.makeLiteral(Arrays.asList(exprResult.asArray()), constExp.getType(), true);
|
||||
} else {
|
||||
literal = rexBuilder.makeLiteral(exprResult.value(), constExp.getType(), true);
|
||||
}
|
||||
|
|
|
@ -388,6 +388,13 @@ public class QueryMaker
|
|||
} else {
|
||||
coercedValue = value.getClass().getName();
|
||||
}
|
||||
} else if (sqlType == SqlTypeName.ARRAY) {
|
||||
try {
|
||||
coercedValue = jsonMapper.writeValueAsString(value);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
} else {
|
||||
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
|
||||
}
|
||||
|
|
|
@ -8122,4 +8122,140 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSelectConstantArrayExpressionFromTable() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT ARRAY[1,2] as arr, dim1 FROM foo LIMIT 1",
|
||||
ImmutableList.of(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(expressionVirtualColumn("v0", "array(1,2)", ValueType.STRING))
|
||||
.columns("dim1", "v0")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.limit(1)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"[\"1\",\"2\"]", ""}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSelectNonConstantArrayExpressionFromTable() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT ARRAY[CONCAT(dim1, 'word'),'up'] as arr, dim1 FROM foo LIMIT 5",
|
||||
ImmutableList.of(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(expressionVirtualColumn("v0", "array(concat(\"dim1\",'word'),'up')", ValueType.STRING))
|
||||
.columns("dim1", "v0")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.limit(5)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"[\"word\",\"up\"]", ""},
|
||||
new Object[]{"[\"10.1word\",\"up\"]", "10.1"},
|
||||
new Object[]{"[\"2word\",\"up\"]", "2"},
|
||||
new Object[]{"[\"1word\",\"up\"]", "1"},
|
||||
new Object[]{"[\"defword\",\"up\"]", "def"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSelectNonConstantArrayExpressionFromTableFailForMultival() throws Exception
|
||||
{
|
||||
// without expression output type inference to prevent this, the automatic translation will try to turn this into
|
||||
//
|
||||
// `map((dim3) -> array(concat(dim3,'word'),'up'), dim3)`
|
||||
//
|
||||
// This error message will get better in the future. The error without translation would be:
|
||||
//
|
||||
// org.apache.druid.java.util.common.RE: Unhandled array constructor element type [STRING_ARRAY]
|
||||
|
||||
expectedException.expect(RuntimeException.class);
|
||||
expectedException.expectMessage("Unhandled map function output type [STRING_ARRAY]");
|
||||
testQuery(
|
||||
"SELECT ARRAY[CONCAT(dim3, 'word'),'up'] as arr, dim1 FROM foo LIMIT 5",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringOverlapFilter() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT dim3 FROM druid.numfoo WHERE MV_OVERLAP(dim3, ARRAY['a','b']) LIMIT 5",
|
||||
ImmutableList.of(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.filters(expressionFilter("array_overlap(\"dim3\",array('a','b'))"))
|
||||
.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 testMultiValueStringOverlapFilterNonConstant() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT dim3 FROM druid.numfoo WHERE MV_OVERLAP(dim3, ARRAY['a','b']) LIMIT 5",
|
||||
ImmutableList.of(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.filters(expressionFilter("array_overlap(\"dim3\",array('a','b'))"))
|
||||
.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 testMultiValueStringContainsFilter() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT dim3 FROM druid.numfoo WHERE MV_CONTAINS(dim3, ARRAY['a','b']) LIMIT 5",
|
||||
ImmutableList.of(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.filters(expressionFilter("array_contains(\"dim3\",array('a','b'))"))
|
||||
.columns("dim3")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.limit(5)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"[\"a\",\"b\"]"}
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue