mirror of https://github.com/apache/druid.git
more sql support for expression array functions (#7974)
* more sql support for expression array functions * prepend/slice * doc fixes * fix imports * fix tests * add null numeric expr for proper conversions between ExprEval and Expr and back to ExprEval * re-arrange * imports :( * add append/prepend test
This commit is contained in:
parent
f7283378ac
commit
c556d44a19
|
@ -270,6 +270,21 @@ abstract class ConstantExpr implements Expr
|
|||
}
|
||||
}
|
||||
|
||||
abstract class NullNumericConstantExpr extends ConstantExpr
|
||||
{
|
||||
@Override
|
||||
public Object getLiteralValue()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "null";
|
||||
}
|
||||
}
|
||||
|
||||
class LongExpr extends ConstantExpr
|
||||
{
|
||||
private final Long value;
|
||||
|
@ -298,6 +313,16 @@ class LongExpr extends ConstantExpr
|
|||
}
|
||||
}
|
||||
|
||||
class NullLongExpr extends NullNumericConstantExpr
|
||||
{
|
||||
@Override
|
||||
public ExprEval eval(ObjectBinding bindings)
|
||||
{
|
||||
return ExprEval.ofLong(null);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
class LongArrayExpr extends ConstantExpr
|
||||
{
|
||||
private final Long[] value;
|
||||
|
@ -412,6 +437,15 @@ class DoubleExpr extends ConstantExpr
|
|||
}
|
||||
}
|
||||
|
||||
class NullDoubleExpr extends NullNumericConstantExpr
|
||||
{
|
||||
@Override
|
||||
public ExprEval eval(ObjectBinding bindings)
|
||||
{
|
||||
return ExprEval.ofDouble(null);
|
||||
}
|
||||
}
|
||||
|
||||
class DoubleArrayExpr extends ConstantExpr
|
||||
{
|
||||
private final Double[] value;
|
||||
|
|
|
@ -293,6 +293,9 @@ public abstract class ExprEval<T>
|
|||
@Override
|
||||
public Expr toExpr()
|
||||
{
|
||||
if (isNumericNull()) {
|
||||
return new NullDoubleExpr();
|
||||
}
|
||||
return new DoubleExpr(value.doubleValue());
|
||||
}
|
||||
}
|
||||
|
@ -357,9 +360,11 @@ public abstract class ExprEval<T>
|
|||
@Override
|
||||
public Expr toExpr()
|
||||
{
|
||||
if (isNumericNull()) {
|
||||
return new NullLongExpr();
|
||||
}
|
||||
return new LongExpr(value.longValue());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class StringExprEval extends ExprEval<String>
|
||||
|
|
|
@ -1915,8 +1915,12 @@ interface Function
|
|||
ExprEval doApply(ExprEval arrayExpr, ExprEval scalarExpr)
|
||||
{
|
||||
final String join = scalarExpr.asString();
|
||||
final Object[] raw = arrayExpr.asArray();
|
||||
if (raw == null || raw.length == 1 && raw[0] == null) {
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
return ExprEval.of(
|
||||
Arrays.stream(arrayExpr.asArray()).map(String::valueOf).collect(Collectors.joining(join != null ? join : ""))
|
||||
Arrays.stream(raw).map(String::valueOf).collect(Collectors.joining(join != null ? join : ""))
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -1987,7 +1991,7 @@ interface Function
|
|||
break;
|
||||
}
|
||||
}
|
||||
return index < 0 ? ExprEval.of(null) : ExprEval.ofLong(index);
|
||||
return index < 0 ? ExprEval.ofLong(NullHandling.replaceWithDefault() ? -1 : null) : ExprEval.ofLong(index);
|
||||
default:
|
||||
throw new IAE("Function[%s] 2nd argument must be a a scalar type", name());
|
||||
}
|
||||
|
@ -2017,7 +2021,7 @@ interface Function
|
|||
break;
|
||||
}
|
||||
}
|
||||
return index < 0 ? ExprEval.of(null) : ExprEval.ofLong(index + 1);
|
||||
return index < 0 ? ExprEval.ofLong(NullHandling.replaceWithDefault() ? -1 : null) : ExprEval.ofLong(index + 1);
|
||||
default:
|
||||
throw new IAE("Function[%s] 2nd argument must be a a scalar type", name());
|
||||
}
|
||||
|
|
|
@ -195,7 +195,7 @@ public class FunctionTest
|
|||
public void testArrayOffsetOf()
|
||||
{
|
||||
assertExpr("array_offset_of([1, 2, 3], 3)", 2L);
|
||||
assertExpr("array_offset_of([1, 2, 3], 4)", null);
|
||||
assertExpr("array_offset_of([1, 2, 3], 4)", NullHandling.replaceWithDefault() ? -1L : null);
|
||||
assertExpr("array_offset_of(a, 'baz')", 2);
|
||||
}
|
||||
|
||||
|
@ -203,7 +203,7 @@ public class FunctionTest
|
|||
public void testArrayOrdinalOf()
|
||||
{
|
||||
assertExpr("array_ordinal_of([1, 2, 3], 3)", 3L);
|
||||
assertExpr("array_ordinal_of([1, 2, 3], 4)", null);
|
||||
assertExpr("array_ordinal_of([1, 2, 3], 4)", NullHandling.replaceWithDefault() ? -1L : null);
|
||||
assertExpr("array_ordinal_of(a, 'baz')", 3);
|
||||
}
|
||||
|
||||
|
|
|
@ -176,12 +176,13 @@ See javadoc of java.lang.Math for detailed explanation for each function.
|
|||
| `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_prepend(expr,arr)` | adds expr to arr at the beginning, the resulting array type determined by the type of the array |
|
||||
| `array_append(arr1,expr)` | appends expr to arr, the resulting array type determined by the type of the first array |
|
||||
| `array_concat(arr1,arr2)` | concatenates 2 arrays, the resulting array type determined by the type of the first array |
|
||||
| `array_slice(arr,start,end)` | return the subarray of arr from the 0 based index start(inclusive) to end(exclusive), or `null`, if start is less than 0, greater than length of arr or less than end|
|
||||
| `array_to_string(arr,str)` | joins all elements of arr by the delimiter specified by str |
|
||||
| `string_to_array(str1,str2)` | splits str1 into an array on the delimiter specified by str2 |
|
||||
| `array_slice(arr,start,end)` | return the subarray of arr from the 0 based index start(inclusive) to end(exclusive), or `null`, if start is less than 0, greater than length of arr or less than end|
|
||||
| `array_prepend(expr,arr)` | adds expr to arr at the beginning, the resulting array type determined by the type of the array |
|
||||
|
||||
|
||||
|
||||
## Apply Functions
|
||||
|
|
|
@ -664,6 +664,47 @@ public class MultiValuedDimensionTest
|
|||
TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-auto");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByExpressionArrayExpressionFilter()
|
||||
{
|
||||
if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) {
|
||||
expectedException.expect(RuntimeException.class);
|
||||
expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING.");
|
||||
}
|
||||
GroupByQuery query = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource("xx")
|
||||
.setQuerySegmentSpec(new LegacySegmentSpec("1970/3000"))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(new DefaultDimensionSpec("tt", "tt", ValueType.LONG))
|
||||
.setVirtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"tt",
|
||||
"array_offset_of(tags, 't2')",
|
||||
ValueType.LONG,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(context)
|
||||
.build();
|
||||
|
||||
Sequence<Row> result = helper.runQueryOnSegmentsObjs(
|
||||
ImmutableList.of(
|
||||
new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
|
||||
new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
|
||||
),
|
||||
query
|
||||
);
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", NullHandling.replaceWithDefault() ? -1L : null, "count", 6L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tt", 1L, "count", 2L)
|
||||
);
|
||||
|
||||
TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-auto");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByExpressionArrayFnArg()
|
||||
{
|
||||
|
|
|
@ -0,0 +1,73 @@
|
|||
/*
|
||||
* 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.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
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 ArrayLengthOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("ARRAY_LENGTH")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.ARRAY),
|
||||
OperandTypes.family(SqlTypeFamily.MULTISET)
|
||||
)
|
||||
)
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
.returnType(SqlTypeName.INTEGER)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.of(
|
||||
null,
|
||||
DruidExpression.functionCall("array_length", druidExpressions)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* 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.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
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 ArrayOffsetOfOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("ARRAY_OFFSET_OF")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.sequence(
|
||||
"(array,expr)",
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.ARRAY),
|
||||
OperandTypes.family(SqlTypeFamily.MULTISET)
|
||||
),
|
||||
OperandTypes.family(SqlTypeFamily.ANY)
|
||||
)
|
||||
)
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
.returnType(SqlTypeName.INTEGER)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.of(
|
||||
null,
|
||||
DruidExpression.functionCall("array_offset_of", druidExpressions)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* 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.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
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 ArrayOffsetOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("ARRAY_OFFSET")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.sequence(
|
||||
"(array,expr)",
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.ARRAY),
|
||||
OperandTypes.family(SqlTypeFamily.MULTISET)
|
||||
),
|
||||
OperandTypes.family(SqlTypeFamily.NUMERIC)
|
||||
)
|
||||
)
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
.returnType(SqlTypeName.VARCHAR)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.of(
|
||||
null,
|
||||
DruidExpression.functionCall("array_offset", druidExpressions)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* 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.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
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 ArrayOrdinalOfOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("ARRAY_ORDINAL_OF")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.sequence(
|
||||
"(array,expr)",
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.ARRAY),
|
||||
OperandTypes.family(SqlTypeFamily.MULTISET)
|
||||
),
|
||||
OperandTypes.family(SqlTypeFamily.ANY)
|
||||
)
|
||||
)
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
.returnType(SqlTypeName.INTEGER)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.of(
|
||||
null,
|
||||
DruidExpression.functionCall("array_ordinal_of", druidExpressions)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* 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.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
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 ArrayOrdinalOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("ARRAY_ORDINAL")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.sequence(
|
||||
"(array,expr)",
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.ARRAY),
|
||||
OperandTypes.family(SqlTypeFamily.MULTISET)
|
||||
),
|
||||
OperandTypes.family(SqlTypeFamily.NUMERIC)
|
||||
)
|
||||
)
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
.returnType(SqlTypeName.VARCHAR)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.of(
|
||||
null,
|
||||
DruidExpression.functionCall("array_ordinal", druidExpressions)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* 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.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
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 ArrayToStringOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("ARRAY_TO_STRING")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.sequence(
|
||||
"(array,expr)",
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.ARRAY),
|
||||
OperandTypes.family(SqlTypeFamily.MULTISET)
|
||||
),
|
||||
OperandTypes.family(SqlTypeFamily.ANY)
|
||||
)
|
||||
)
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
.returnType(SqlTypeName.VARCHAR)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.of(
|
||||
null,
|
||||
DruidExpression.functionCall("array_to_string", druidExpressions)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* 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.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
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 MultiValueStringAppendOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("MV_APPEND")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.sequence(
|
||||
"(array,expr)",
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.NUMERIC)
|
||||
)
|
||||
)
|
||||
)
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
.returnType(SqlTypeName.VARCHAR)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.of(
|
||||
null,
|
||||
DruidExpression.functionCall("array_append", druidExpressions)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,73 @@
|
|||
/*
|
||||
* 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.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
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 MultiValueStringConcatOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("MV_CONCAT")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.sequence(
|
||||
"(array,array)",
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.STRING)
|
||||
)
|
||||
)
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
.returnType(SqlTypeName.VARCHAR)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.of(
|
||||
null,
|
||||
DruidExpression.functionCall("array_concat", druidExpressions)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* 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.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
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 MultiValueStringPrependOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("MV_PREPEND")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.sequence(
|
||||
"(expr,array)",
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.NUMERIC)
|
||||
),
|
||||
OperandTypes.family(SqlTypeFamily.STRING)
|
||||
)
|
||||
)
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
.returnType(SqlTypeName.VARCHAR)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.of(
|
||||
null,
|
||||
DruidExpression.functionCall("array_prepend", druidExpressions)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,81 @@
|
|||
/*
|
||||
* 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.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
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 MultiValueStringSliceOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("MV_SLICE")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.or(
|
||||
OperandTypes.sequence(
|
||||
"(expr,start)",
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.NUMERIC)
|
||||
),
|
||||
OperandTypes.sequence(
|
||||
"(expr,start,end)",
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.NUMERIC),
|
||||
OperandTypes.family(SqlTypeFamily.NUMERIC)
|
||||
)
|
||||
)
|
||||
)
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
.returnType(SqlTypeName.VARCHAR)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.of(
|
||||
null,
|
||||
DruidExpression.functionCall("array_slice", druidExpressions)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* 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.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
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 StringToMultiValueStringOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
// note: since this function produces an array
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("STRING_TO_MV")
|
||||
.operandTypeChecker(
|
||||
OperandTypes.sequence(
|
||||
"(string,expr)",
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.STRING)
|
||||
)
|
||||
)
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
.returnType(SqlTypeName.VARCHAR)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.of(
|
||||
null,
|
||||
DruidExpression.functionCall("string_to_array", druidExpressions)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -48,7 +48,13 @@ 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.ArrayLengthOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.ArrayOffsetOfOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.ArrayOffsetOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.ArrayOrdinalOfOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.ArrayOrdinalOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.ArrayOverlapOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.ArrayToStringOperatorConversion;
|
||||
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;
|
||||
|
@ -61,6 +67,10 @@ import org.apache.druid.sql.calcite.expression.builtin.LTrimOperatorConversion;
|
|||
import org.apache.druid.sql.calcite.expression.builtin.LeftOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.LikeOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.MillisToTimestampOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.MultiValueStringAppendOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.MultiValueStringConcatOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.MultiValueStringPrependOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.MultiValueStringSliceOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.ParseLongOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.PositionOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.RPadOperatorConversion;
|
||||
|
@ -72,6 +82,7 @@ import org.apache.druid.sql.calcite.expression.builtin.ReverseOperatorConversion
|
|||
import org.apache.druid.sql.calcite.expression.builtin.RightOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.RoundOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.StringFormatOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.StringToMultiValueStringOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.StrposOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.SubstringOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.TextcatOperatorConversion;
|
||||
|
@ -212,6 +223,24 @@ public class DruidOperatorTable implements SqlOperatorTable
|
|||
.add(new ArrayOverlapOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayContainsOperatorConversion(), "MV_CONTAINS"))
|
||||
.add(new AliasedOperatorConversion(new ArrayOverlapOperatorConversion(), "MV_OVERLAP"))
|
||||
.add(new ArrayLengthOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayLengthOperatorConversion(), "MV_LENGTH"))
|
||||
.add(new ArrayOffsetOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayOffsetOperatorConversion(), "MV_OFFSET"))
|
||||
.add(new ArrayOrdinalOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayOrdinalOperatorConversion(), "MV_ORDINAL"))
|
||||
.add(new ArrayOffsetOfOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayOffsetOfOperatorConversion(), "MV_OFFSET_OF"))
|
||||
.add(new ArrayOrdinalOfOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayOrdinalOfOperatorConversion(), "MV_ORDINAL_OF"))
|
||||
.add(new ArrayToStringOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayToStringOperatorConversion(), "MV_TO_STRING"))
|
||||
// multi-value string operators
|
||||
.add(new MultiValueStringAppendOperatorConversion())
|
||||
.add(new MultiValueStringConcatOperatorConversion())
|
||||
.add(new MultiValueStringPrependOperatorConversion())
|
||||
.add(new MultiValueStringSliceOperatorConversion())
|
||||
.add(new StringToMultiValueStringOperatorConversion())
|
||||
.build();
|
||||
|
||||
// Operators that have no conversion, but are handled in the convertlet table, so they still need to exist.
|
||||
|
|
|
@ -7446,7 +7446,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
public void testConcat() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT CONCAt(dim1, '-', dim1, '_', dim1) as dimX FROM foo",
|
||||
"SELECT CONCAT(dim1, '-', dim1, '_', dim1) as dimX FROM foo",
|
||||
ImmutableList.of(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
|
@ -7889,8 +7889,16 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
expressionVirtualColumn("v3", "div((\"__time\" - 946684683000),60000)", ValueType.LONG),
|
||||
expressionVirtualColumn("v4", "div((\"__time\" - 946684743000),1000)", ValueType.LONG),
|
||||
expressionVirtualColumn("v5", "subtract_months(\"__time\",941414400000,'UTC')", ValueType.LONG),
|
||||
expressionVirtualColumn("v6", "div(subtract_months(\"__time\",846806400000,'UTC'),12)", ValueType.LONG),
|
||||
expressionVirtualColumn("v7", "div(subtract_months(\"__time\",844128000000,'UTC'),3)", ValueType.LONG),
|
||||
expressionVirtualColumn(
|
||||
"v6",
|
||||
"div(subtract_months(\"__time\",846806400000,'UTC'),12)",
|
||||
ValueType.LONG
|
||||
),
|
||||
expressionVirtualColumn(
|
||||
"v7",
|
||||
"div(subtract_months(\"__time\",844128000000,'UTC'),3)",
|
||||
ValueType.LONG
|
||||
),
|
||||
expressionVirtualColumn("v8", "div(div((\"__time\" - 907200000000),1000),604800)", ValueType.LONG)
|
||||
)
|
||||
.columns("v0", "v1", "v2", "v3", "v4", "v5", "v6", "v7", "v8")
|
||||
|
@ -8258,4 +8266,535 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringSlice() throws Exception
|
||||
{
|
||||
final String nullVal = NullHandling.replaceWithDefault() ? "[\"foo\"]" : "[null]";
|
||||
testQuery(
|
||||
"SELECT MV_SLICE(dim3, 1) FROM druid.numfoo",
|
||||
ImmutableList.of(
|
||||
new Druids.ScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(expressionVirtualColumn("v0", "array_slice(\"dim3\",1)", ValueType.STRING))
|
||||
.columns(ImmutableList.of("v0"))
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"[\"b\"]"},
|
||||
new Object[]{"[\"c\"]"},
|
||||
new Object[]{"[]"},
|
||||
new Object[]{"[]"},
|
||||
new Object[]{"[]"},
|
||||
new Object[]{"[]"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringLength() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT dim1, MV_LENGTH(dim3), SUM(cnt) FROM druid.numfoo GROUP BY 1, 2 ORDER BY 2 DESC",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "array_length(\"dim3\")", ValueType.LONG))
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("dim1", "_d0", ValueType.STRING),
|
||||
new DefaultDimensionSpec("v0", "v0", ValueType.LONG)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setLimitSpec(new DefaultLimitSpec(
|
||||
ImmutableList.of(new OrderByColumnSpec(
|
||||
"v0",
|
||||
Direction.DESCENDING,
|
||||
StringComparators.NUMERIC
|
||||
)),
|
||||
Integer.MAX_VALUE
|
||||
))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"", 2, 1L},
|
||||
new Object[]{"10.1", 2, 1L},
|
||||
new Object[]{"1", 1, 1L},
|
||||
new Object[]{"2", 1, 1L},
|
||||
new Object[]{"abc", 1, 1L},
|
||||
new Object[]{"def", 1, 1L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringAppend() throws Exception
|
||||
{
|
||||
ImmutableList<Object[]> results;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
results = ImmutableList.of(
|
||||
new Object[]{"foo", 6L},
|
||||
new Object[]{"", 3L},
|
||||
new Object[]{"b", 2L},
|
||||
new Object[]{"a", 1L},
|
||||
new Object[]{"c", 1L},
|
||||
new Object[]{"d", 1L}
|
||||
);
|
||||
} else {
|
||||
results = ImmutableList.of(
|
||||
new Object[]{"foo", 6L},
|
||||
new Object[]{null, 2L},
|
||||
new Object[]{"b", 2L},
|
||||
new Object[]{"", 1L},
|
||||
new Object[]{"a", 1L},
|
||||
new Object[]{"c", 1L},
|
||||
new Object[]{"d", 1L}
|
||||
);
|
||||
}
|
||||
testQuery(
|
||||
"SELECT MV_APPEND(dim3, 'foo'), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "array_append(\"dim3\",'foo')", ValueType.STRING))
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "v0", ValueType.STRING)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setLimitSpec(new DefaultLimitSpec(
|
||||
ImmutableList.of(new OrderByColumnSpec(
|
||||
"a0",
|
||||
Direction.DESCENDING,
|
||||
StringComparators.NUMERIC
|
||||
)),
|
||||
Integer.MAX_VALUE
|
||||
))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
results
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringPrepend() throws Exception
|
||||
{
|
||||
ImmutableList<Object[]> results;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
results = ImmutableList.of(
|
||||
new Object[]{"foo", 6L},
|
||||
new Object[]{"", 3L},
|
||||
new Object[]{"b", 2L},
|
||||
new Object[]{"a", 1L},
|
||||
new Object[]{"c", 1L},
|
||||
new Object[]{"d", 1L}
|
||||
);
|
||||
} else {
|
||||
results = ImmutableList.of(
|
||||
new Object[]{"foo", 6L},
|
||||
new Object[]{null, 2L},
|
||||
new Object[]{"b", 2L},
|
||||
new Object[]{"", 1L},
|
||||
new Object[]{"a", 1L},
|
||||
new Object[]{"c", 1L},
|
||||
new Object[]{"d", 1L}
|
||||
);
|
||||
}
|
||||
testQuery(
|
||||
"SELECT MV_PREPEND('foo', dim3), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "array_prepend('foo',\"dim3\")", ValueType.STRING))
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "v0", ValueType.STRING)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setLimitSpec(new DefaultLimitSpec(
|
||||
ImmutableList.of(new OrderByColumnSpec(
|
||||
"a0",
|
||||
Direction.DESCENDING,
|
||||
StringComparators.NUMERIC
|
||||
)),
|
||||
Integer.MAX_VALUE
|
||||
))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
results
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringPrependAppend() throws Exception
|
||||
{
|
||||
ImmutableList<Object[]> results;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
results = ImmutableList.of(
|
||||
new Object[]{"foo,null", "null,foo", 3L},
|
||||
new Object[]{"foo,a,b", "a,b,foo", 1L},
|
||||
new Object[]{"foo,b,c", "b,c,foo", 1L},
|
||||
new Object[]{"foo,d", "d,foo", 1L}
|
||||
);
|
||||
} else {
|
||||
results = ImmutableList.of(
|
||||
new Object[]{"foo,null", "null,foo", 2L},
|
||||
new Object[]{"foo,", ",foo", 1L},
|
||||
new Object[]{"foo,a,b", "a,b,foo", 1L},
|
||||
new Object[]{"foo,b,c", "b,c,foo", 1L},
|
||||
new Object[]{"foo,d", "d,foo", 1L}
|
||||
);
|
||||
}
|
||||
testQuery(
|
||||
"SELECT MV_TO_STRING(MV_PREPEND('foo', dim3), ','), MV_TO_STRING(MV_APPEND(dim3, 'foo'), ','), SUM(cnt) FROM druid.numfoo GROUP BY 1,2 ORDER BY 3 DESC",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(
|
||||
expressionVirtualColumn("v0", "array_to_string(array_prepend('foo',\"dim3\"),',')", ValueType.STRING),
|
||||
expressionVirtualColumn("v1", "array_to_string(array_append(\"dim3\",'foo'),',')", ValueType.STRING)
|
||||
)
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "v0", ValueType.STRING),
|
||||
new DefaultDimensionSpec("v1", "v1", ValueType.STRING)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setLimitSpec(new DefaultLimitSpec(
|
||||
ImmutableList.of(new OrderByColumnSpec(
|
||||
"a0",
|
||||
Direction.DESCENDING,
|
||||
StringComparators.NUMERIC
|
||||
)),
|
||||
Integer.MAX_VALUE
|
||||
))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
results
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringConcat() throws Exception
|
||||
{
|
||||
ImmutableList<Object[]> results;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
results = ImmutableList.of(
|
||||
new Object[]{"", 6L},
|
||||
new Object[]{"b", 4L},
|
||||
new Object[]{"a", 2L},
|
||||
new Object[]{"c", 2L},
|
||||
new Object[]{"d", 2L}
|
||||
);
|
||||
} else {
|
||||
results = ImmutableList.of(
|
||||
new Object[]{null, 4L},
|
||||
new Object[]{"b", 4L},
|
||||
new Object[]{"", 2L},
|
||||
new Object[]{"a", 2L},
|
||||
new Object[]{"c", 2L},
|
||||
new Object[]{"d", 2L}
|
||||
);
|
||||
}
|
||||
testQuery(
|
||||
"SELECT MV_CONCAT(dim3, dim3), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "array_concat(\"dim3\",\"dim3\")", ValueType.STRING))
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "v0", ValueType.STRING)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setLimitSpec(new DefaultLimitSpec(
|
||||
ImmutableList.of(new OrderByColumnSpec(
|
||||
"a0",
|
||||
Direction.DESCENDING,
|
||||
StringComparators.NUMERIC
|
||||
)),
|
||||
Integer.MAX_VALUE
|
||||
))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
results
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringOffset() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT MV_OFFSET(dim3, 1), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "array_offset(\"dim3\",1)", ValueType.STRING))
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "v0", ValueType.STRING)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setLimitSpec(new DefaultLimitSpec(
|
||||
ImmutableList.of(new OrderByColumnSpec(
|
||||
"a0",
|
||||
Direction.DESCENDING,
|
||||
StringComparators.NUMERIC
|
||||
)),
|
||||
Integer.MAX_VALUE
|
||||
))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultStringValue(), 4L},
|
||||
new Object[]{"b", 1L},
|
||||
new Object[]{"c", 1L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringOrdinal() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT MV_ORDINAL(dim3, 2), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "array_ordinal(\"dim3\",2)", ValueType.STRING))
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "v0", ValueType.STRING)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setLimitSpec(new DefaultLimitSpec(
|
||||
ImmutableList.of(new OrderByColumnSpec(
|
||||
"a0",
|
||||
Direction.DESCENDING,
|
||||
StringComparators.NUMERIC
|
||||
)),
|
||||
Integer.MAX_VALUE
|
||||
))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultStringValue(), 4L},
|
||||
new Object[]{"b", 1L},
|
||||
new Object[]{"c", 1L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringOffsetOf() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT MV_OFFSET_OF(dim3, 'b'), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "array_offset_of(\"dim3\",'b')", ValueType.LONG))
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "v0", ValueType.LONG)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setLimitSpec(new DefaultLimitSpec(
|
||||
ImmutableList.of(new OrderByColumnSpec(
|
||||
"a0",
|
||||
Direction.DESCENDING,
|
||||
StringComparators.NUMERIC
|
||||
)),
|
||||
Integer.MAX_VALUE
|
||||
))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.replaceWithDefault() ? -1 : null, 4L},
|
||||
new Object[]{0, 1L},
|
||||
new Object[]{1, 1L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringOrdinalOf() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT MV_ORDINAL_OF(dim3, 'b'), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "array_ordinal_of(\"dim3\",'b')", ValueType.LONG))
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "v0", ValueType.LONG)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setLimitSpec(new DefaultLimitSpec(
|
||||
ImmutableList.of(new OrderByColumnSpec(
|
||||
"a0",
|
||||
Direction.DESCENDING,
|
||||
StringComparators.NUMERIC
|
||||
)),
|
||||
Integer.MAX_VALUE
|
||||
))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.replaceWithDefault() ? -1 : null, 4L},
|
||||
new Object[]{1, 1L},
|
||||
new Object[]{2, 1L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringToString() throws Exception
|
||||
{
|
||||
ImmutableList<Object[]> results;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
results = ImmutableList.of(
|
||||
new Object[]{"", 3L},
|
||||
new Object[]{"a,b", 1L},
|
||||
new Object[]{"b,c", 1L},
|
||||
new Object[]{"d", 1L}
|
||||
);
|
||||
} else {
|
||||
results = ImmutableList.of(
|
||||
new Object[]{null, 2L},
|
||||
new Object[]{"", 1L},
|
||||
new Object[]{"a,b", 1L},
|
||||
new Object[]{"b,c", 1L},
|
||||
new Object[]{"d", 1L}
|
||||
);
|
||||
}
|
||||
testQuery(
|
||||
"SELECT MV_TO_STRING(dim3, ','), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "array_to_string(\"dim3\",',')", ValueType.STRING))
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "v0", ValueType.STRING)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setLimitSpec(new DefaultLimitSpec(
|
||||
ImmutableList.of(new OrderByColumnSpec(
|
||||
"a0",
|
||||
Direction.DESCENDING,
|
||||
StringComparators.NUMERIC
|
||||
)),
|
||||
Integer.MAX_VALUE
|
||||
))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
results
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringToStringToMultiValueString() throws Exception
|
||||
{
|
||||
ImmutableList<Object[]> results;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
results = ImmutableList.of(
|
||||
new Object[]{"d", 7L},
|
||||
new Object[]{"", 3L},
|
||||
new Object[]{"b", 2L},
|
||||
new Object[]{"a", 1L},
|
||||
new Object[]{"c", 1L}
|
||||
);
|
||||
} else {
|
||||
results = ImmutableList.of(
|
||||
new Object[]{"d", 5L},
|
||||
new Object[]{null, 2L},
|
||||
new Object[]{"b", 2L},
|
||||
new Object[]{"", 1L},
|
||||
new Object[]{"a", 1L},
|
||||
new Object[]{"c", 1L}
|
||||
);
|
||||
}
|
||||
testQuery(
|
||||
"SELECT STRING_TO_MV(CONCAT(MV_TO_STRING(dim3, ','), ',d'), ','), SUM(cnt) FROM druid.numfoo WHERE MV_LENGTH(dim3) > 0 GROUP BY 1 ORDER BY 2 DESC",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(
|
||||
expressionVirtualColumn("v0", "array_length(\"dim3\")", ValueType.LONG),
|
||||
expressionVirtualColumn(
|
||||
"v1",
|
||||
"string_to_array(concat(array_to_string(\"dim3\",','),',d'),',')",
|
||||
ValueType.STRING
|
||||
)
|
||||
)
|
||||
.setDimFilter(bound("v0", "0", null, true, false, null, StringComparators.NUMERIC))
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v1", "v1", ValueType.STRING)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setLimitSpec(new DefaultLimitSpec(
|
||||
ImmutableList.of(new OrderByColumnSpec(
|
||||
"a0",
|
||||
Direction.DESCENDING,
|
||||
StringComparators.NUMERIC
|
||||
)),
|
||||
Integer.MAX_VALUE
|
||||
))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
results
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue