mirror of https://github.com/apache/druid.git
STRING_AGG SQL aggregator function (#11241)
* add string_agg * oops * style and fix test * spelling * fixup * review stuffs
This commit is contained in:
parent
2004a94675
commit
9af7ba9d2a
|
@ -316,7 +316,7 @@ no matches while aggregating values across an entire table without a grouping, o
|
|||
within a grouping. What this value is exactly varies per aggregator, but COUNT, and the various approximate count
|
||||
distinct sketch functions, will always return 0.
|
||||
|
||||
Only the COUNT and ARRAY_AGG aggregations can accept the DISTINCT keyword.
|
||||
Only the COUNT, ARRAY_AGG, and STRING_AGG aggregations can accept the DISTINCT keyword.
|
||||
|
||||
> The order of aggregation operations across segments is not deterministic. This means that non-commutative aggregation
|
||||
> functions can produce inconsistent results across the same query.
|
||||
|
@ -358,8 +358,10 @@ Only the COUNT and ARRAY_AGG aggregations can accept the DISTINCT keyword.
|
|||
|`ANY_VALUE(expr)`|Returns any value of `expr` including null. `expr` must be numeric. This aggregator can simplify and optimize the performance by returning the first encountered value (including null)|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
|
||||
|`ANY_VALUE(expr, maxBytesPerString)`|Like `ANY_VALUE(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `''`|
|
||||
|`GROUPING(expr, expr...)`|Returns a number to indicate which groupBy dimension is included in a row, when using `GROUPING SETS`. Refer to [additional documentation](aggregations.md#grouping-aggregator) on how to infer this number.|N/A|
|
||||
|`ARRAY_AGG(expr, [size])`|Collects all values of `expr` into an ARRAY, including null values, with `size` in bytes limit on aggregation size (default of 1024 bytes). Use of `ORDER BY` within the `ARRAY_AGG` expression is not currently supported, and the ordering of results within the output array may vary depending on processing order.|`null`|
|
||||
|`ARRAY_AGG(DISTINCT expr, [size])`|Collects all distinct values of `expr` into an ARRAY, including null values, with `size` in bytes limit on aggregation size (default of 1024 bytes) per aggregate. Use of `ORDER BY` within the `ARRAY_AGG` expression is not currently supported, and the ordering of results within the output array may vary depending on processing order.|`null`|
|
||||
|`ARRAY_AGG(expr, [size])`|Collects all values of `expr` into an ARRAY, including null values, with `size` in bytes limit on aggregation size (default of 1024 bytes). If the aggregated array grows larger than the maximum size in bytes, the query will fail. Use of `ORDER BY` within the `ARRAY_AGG` expression is not currently supported, and the ordering of results within the output array may vary depending on processing order.|`null`|
|
||||
|`ARRAY_AGG(DISTINCT expr, [size])`|Collects all distinct values of `expr` into an ARRAY, including null values, with `size` in bytes limit on aggregation size (default of 1024 bytes) per aggregate. If the aggregated array grows larger than the maximum size in bytes, the query will fail. Use of `ORDER BY` within the `ARRAY_AGG` expression is not currently supported, and the ordering of results within the output array may vary depending on processing order.|`null`|
|
||||
|`STRING_AGG(expr, separator, [size])`|Collects all values of `expr` into a single STRING, ignoring null values. Each value is joined by the `separator` which must be a literal STRING. An optional `size` in bytes can be supplied to limit aggregation size (default of 1024 bytes). If the aggregated string grows larger than the maximum size in bytes, the query will fail. Use of `ORDER BY` within the `STRING_AGG` expression is not currently supported, and the ordering of results within the output string may vary depending on processing order.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `''`|
|
||||
|`STRING_AGG(DISTINCT expr, separator, [size])`|Collects all distinct values of `expr` into a single STRING, ignoring null values. Each value is joined by the `separator` which must be a literal STRING. An optional `size` in bytes can be supplied to limit aggregation size (default of 1024 bytes). If the aggregated string grows larger than the maximum size in bytes, the query will fail. Use of `ORDER BY` within the `STRING_AGG` expression is not currently supported, and the ordering of results within the output string may vary depending on processing order.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `''`|
|
||||
|`BIT_AND(expr)`|Performs a bitwise AND operation on all input values.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
|
||||
|`BIT_OR(expr)`|Performs a bitwise OR operation on all input values.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
|
||||
|`BIT_XOR(expr)`|Performs a bitwise XOR operation on all input values.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
|
||||
|
|
|
@ -66,7 +66,7 @@ public class ExpressionLambdaAggregatorFactory extends AggregatorFactory
|
|||
// minimum permitted agg size is 10 bytes so it is at least large enough to hold primitive numerics (long, double)
|
||||
// | expression type byte | is_null byte | primitive value (8 bytes) |
|
||||
private static final int MIN_SIZE_BYTES = 10;
|
||||
private static final HumanReadableBytes DEFAULT_MAX_SIZE_BYTES = new HumanReadableBytes(1L << 10);
|
||||
public static final HumanReadableBytes DEFAULT_MAX_SIZE_BYTES = new HumanReadableBytes(1L << 10);
|
||||
|
||||
private final String name;
|
||||
@Nullable
|
||||
|
|
|
@ -184,7 +184,7 @@ public class ArraySqlAggregator implements SqlAggregator
|
|||
{
|
||||
RelDataType type = sqlOperatorBinding.getOperandType(0);
|
||||
if (SqlTypeUtil.isArray(type)) {
|
||||
throw new ISE("Cannot ARRAY_AGG on array inputs %s", type);
|
||||
throw new ISE("Cannot use ARRAY_AGG on array inputs %s", type);
|
||||
}
|
||||
return Calcites.createSqlArrayTypeWithNullability(
|
||||
sqlOperatorBinding.getTypeFactory(),
|
||||
|
@ -209,7 +209,7 @@ public class ArraySqlAggregator implements SqlAggregator
|
|||
OperandTypes.or(
|
||||
OperandTypes.ANY,
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(StringUtils.format("'%s'(expr, maxSizeBytes)", NAME), OperandTypes.ANY, OperandTypes.LITERAL),
|
||||
OperandTypes.sequence(StringUtils.format("'%s'(expr, maxSizeBytes)", NAME), OperandTypes.ANY, OperandTypes.POSITIVE_INTEGER_LITERAL),
|
||||
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC)
|
||||
)
|
||||
),
|
||||
|
|
|
@ -0,0 +1,221 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.aggregation.builtin;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.calcite.rel.core.AggregateCall;
|
||||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.sql.SqlAggFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.sql.type.InferTypes;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
import org.apache.calcite.util.Optionality;
|
||||
import org.apache.druid.java.util.common.HumanReadableBytes;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.segment.VirtualColumn;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.sql.calcite.aggregation.Aggregation;
|
||||
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
|
||||
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
||||
import org.apache.druid.sql.calcite.expression.Expressions;
|
||||
import org.apache.druid.sql.calcite.planner.Calcites;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class StringSqlAggregator implements SqlAggregator
|
||||
{
|
||||
private static final String NAME = "STRING_AGG";
|
||||
private static final SqlAggFunction FUNCTION = new StringAggFunction();
|
||||
|
||||
@Override
|
||||
public SqlAggFunction calciteFunction()
|
||||
{
|
||||
return FUNCTION;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Aggregation toDruidAggregation(
|
||||
PlannerContext plannerContext,
|
||||
RowSignature rowSignature,
|
||||
VirtualColumnRegistry virtualColumnRegistry,
|
||||
RexBuilder rexBuilder,
|
||||
String name,
|
||||
AggregateCall aggregateCall,
|
||||
Project project,
|
||||
List<Aggregation> existingAggregations,
|
||||
boolean finalizeAggregations
|
||||
)
|
||||
{
|
||||
final List<DruidExpression> arguments = aggregateCall
|
||||
.getArgList()
|
||||
.stream()
|
||||
.map(i -> Expressions.fromFieldAccess(rowSignature, project, i))
|
||||
.map(rexNode -> Expressions.toDruidExpression(plannerContext, rowSignature, rexNode))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
if (arguments.stream().anyMatch(Objects::isNull)) {
|
||||
return null;
|
||||
}
|
||||
|
||||
RexNode separatorNode = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(1)
|
||||
);
|
||||
if (!separatorNode.isA(SqlKind.LITERAL)) {
|
||||
// separator must be a literal
|
||||
return null;
|
||||
}
|
||||
String separator = RexLiteral.stringValue(separatorNode);
|
||||
|
||||
if (separator == null) {
|
||||
// separator must not be null
|
||||
return null;
|
||||
}
|
||||
|
||||
Integer maxSizeBytes = null;
|
||||
if (arguments.size() > 2) {
|
||||
RexNode maxBytes = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(2)
|
||||
);
|
||||
if (!maxBytes.isA(SqlKind.LITERAL)) {
|
||||
// maxBytes must be a literal
|
||||
return null;
|
||||
}
|
||||
maxSizeBytes = ((Number) RexLiteral.value(maxBytes)).intValue();
|
||||
}
|
||||
final DruidExpression arg = arguments.get(0);
|
||||
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
|
||||
|
||||
final String initialvalue = "[]";
|
||||
final ValueType elementType = ValueType.STRING;
|
||||
final String fieldName;
|
||||
if (arg.isDirectColumnAccess()) {
|
||||
fieldName = arg.getDirectColumn();
|
||||
} else {
|
||||
VirtualColumn vc = virtualColumnRegistry.getOrCreateVirtualColumnForExpression(plannerContext, arg, elementType);
|
||||
fieldName = vc.getOutputName();
|
||||
}
|
||||
|
||||
final String finalizer = StringUtils.format("if(array_length(o) == 0, null, array_to_string(o, '%s'))", separator);
|
||||
final NotDimFilter dimFilter = new NotDimFilter(new SelectorDimFilter(fieldName, null, null));
|
||||
if (aggregateCall.isDistinct()) {
|
||||
return Aggregation.create(
|
||||
// string_agg ignores nulls
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
name,
|
||||
ImmutableSet.of(fieldName),
|
||||
null,
|
||||
initialvalue,
|
||||
null,
|
||||
true,
|
||||
StringUtils.format("array_set_add(\"__acc\", \"%s\")", fieldName),
|
||||
StringUtils.format("array_set_add_all(\"__acc\", \"%s\")", name),
|
||||
null,
|
||||
finalizer,
|
||||
maxSizeBytes != null ? new HumanReadableBytes(maxSizeBytes) : null,
|
||||
macroTable
|
||||
),
|
||||
dimFilter
|
||||
)
|
||||
);
|
||||
} else {
|
||||
return Aggregation.create(
|
||||
// string_agg ignores nulls
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
name,
|
||||
ImmutableSet.of(fieldName),
|
||||
null,
|
||||
initialvalue,
|
||||
null,
|
||||
true,
|
||||
StringUtils.format("array_append(\"__acc\", \"%s\")", fieldName),
|
||||
StringUtils.format("array_concat(\"__acc\", \"%s\")", name),
|
||||
null,
|
||||
finalizer,
|
||||
maxSizeBytes != null ? new HumanReadableBytes(maxSizeBytes) : null,
|
||||
macroTable
|
||||
),
|
||||
dimFilter
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private static class StringAggFunction extends SqlAggFunction
|
||||
{
|
||||
StringAggFunction()
|
||||
{
|
||||
super(
|
||||
NAME,
|
||||
null,
|
||||
SqlKind.OTHER_FUNCTION,
|
||||
opBinding ->
|
||||
Calcites.createSqlTypeWithNullability(opBinding.getTypeFactory(), SqlTypeName.VARCHAR, true),
|
||||
InferTypes.ANY_NULLABLE,
|
||||
OperandTypes.or(
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(
|
||||
StringUtils.format("'%s'(expr, separator)", NAME),
|
||||
OperandTypes.ANY,
|
||||
OperandTypes.STRING
|
||||
),
|
||||
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.STRING)
|
||||
),
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(
|
||||
StringUtils.format("'%s'(expr, separator, maxSizeBytes)", NAME),
|
||||
OperandTypes.ANY,
|
||||
OperandTypes.STRING,
|
||||
OperandTypes.POSITIVE_INTEGER_LITERAL
|
||||
),
|
||||
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.STRING, SqlTypeFamily.NUMERIC)
|
||||
)
|
||||
),
|
||||
SqlFunctionCategory.STRING,
|
||||
false,
|
||||
false,
|
||||
Optionality.IGNORED
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -42,6 +42,7 @@ import org.apache.druid.sql.calcite.aggregation.builtin.EarliestLatestAnySqlAggr
|
|||
import org.apache.druid.sql.calcite.aggregation.builtin.GroupingSqlAggregator;
|
||||
import org.apache.druid.sql.calcite.aggregation.builtin.MaxSqlAggregator;
|
||||
import org.apache.druid.sql.calcite.aggregation.builtin.MinSqlAggregator;
|
||||
import org.apache.druid.sql.calcite.aggregation.builtin.StringSqlAggregator;
|
||||
import org.apache.druid.sql.calcite.aggregation.builtin.SumSqlAggregator;
|
||||
import org.apache.druid.sql.calcite.aggregation.builtin.SumZeroSqlAggregator;
|
||||
import org.apache.druid.sql.calcite.expression.AliasedOperatorConversion;
|
||||
|
@ -137,6 +138,7 @@ public class DruidOperatorTable implements SqlOperatorTable
|
|||
.add(new SumZeroSqlAggregator())
|
||||
.add(new GroupingSqlAggregator())
|
||||
.add(new ArraySqlAggregator())
|
||||
.add(new StringSqlAggregator())
|
||||
.add(new BitwiseSqlAggregator(BitwiseSqlAggregator.Op.AND))
|
||||
.add(new BitwiseSqlAggregator(BitwiseSqlAggregator.Op.OR))
|
||||
.add(new BitwiseSqlAggregator(BitwiseSqlAggregator.Op.XOR))
|
||||
|
|
|
@ -1161,7 +1161,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_concat(\"__acc\", \"a0\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
|
@ -1175,7 +1175,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a1\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
|
@ -1190,7 +1190,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a2\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
selector("dim1", "shazbot", null)
|
||||
|
@ -1236,7 +1236,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_concat(\"__acc\", \"a0\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
|
@ -1250,7 +1250,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a1\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
|
@ -1290,7 +1290,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_concat(\"__acc\", \"a0\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
|
@ -1304,7 +1304,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a1\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
|
@ -1318,7 +1318,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_concat(\"__acc\", \"a2\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
|
@ -1332,7 +1332,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a3\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
|
@ -1346,7 +1346,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_concat(\"__acc\", \"a4\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
|
@ -1360,7 +1360,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a5\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
|
@ -1415,7 +1415,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a0\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
|
@ -1457,7 +1457,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a0\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
|
@ -1577,7 +1577,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a0\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
|
@ -1669,7 +1669,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a0\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
|
@ -1746,7 +1746,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a0\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
|
|
|
@ -12730,6 +12730,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
+ " LATEST(dim1, 1024),\n"
|
||||
+ " LATEST(l1),\n"
|
||||
+ " ARRAY_AGG(DISTINCT dim3),\n"
|
||||
+ " STRING_AGG(DISTINCT dim3, '|'),\n"
|
||||
+ " BIT_AND(l1),\n"
|
||||
+ " BIT_OR(l1),\n"
|
||||
+ " BIT_XOR(l1)\n"
|
||||
|
@ -12759,43 +12760,43 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a6\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a7",
|
||||
ImmutableSet.of("dim3"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_set_add(\"__acc\", \"dim3\")",
|
||||
"array_set_add_all(\"__acc\", \"a7\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, '|'))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("dim3", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a7",
|
||||
"a8",
|
||||
ImmutableSet.of("l1"),
|
||||
"__acc",
|
||||
"0",
|
||||
"0",
|
||||
NullHandling.sqlCompatible(),
|
||||
"bitwiseAnd(\"__acc\", \"l1\")",
|
||||
"bitwiseAnd(\"__acc\", \"a7\")",
|
||||
"bitwiseAnd(\"__acc\", \"a8\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a8",
|
||||
ImmutableSet.of("l1"),
|
||||
"__acc",
|
||||
"0",
|
||||
"0",
|
||||
NullHandling.sqlCompatible(),
|
||||
"bitwiseOr(\"__acc\", \"l1\")",
|
||||
"bitwiseOr(\"__acc\", \"a8\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a9",
|
||||
|
@ -12804,11 +12805,28 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"0",
|
||||
"0",
|
||||
NullHandling.sqlCompatible(),
|
||||
"bitwiseOr(\"__acc\", \"l1\")",
|
||||
"bitwiseOr(\"__acc\", \"a9\")",
|
||||
null,
|
||||
null,
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a10",
|
||||
ImmutableSet.of("l1"),
|
||||
"__acc",
|
||||
"0",
|
||||
"0",
|
||||
NullHandling.sqlCompatible(),
|
||||
"bitwiseXor(\"__acc\", \"l1\")",
|
||||
"bitwiseXor(\"__acc\", \"a9\")",
|
||||
"bitwiseXor(\"__acc\", \"a10\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
|
@ -12820,8 +12838,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
),
|
||||
ImmutableList.of(
|
||||
useDefault
|
||||
? new Object[]{"", 0L, "", 0L, "", 0L, null, 0L, 0L, 0L}
|
||||
: new Object[]{null, null, null, null, null, null, null, null, null, null}
|
||||
? new Object[]{"", 0L, "", 0L, "", 0L, null, "", 0L, 0L, 0L}
|
||||
: new Object[]{null, null, null, null, null, null, null, null, null, null, null}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -12963,6 +12981,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
+ " LATEST(dim1, 1024) FILTER(WHERE dim1 = 'nonexistent'),\n"
|
||||
+ " LATEST(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
|
||||
+ " ARRAY_AGG(DISTINCT dim3) FILTER(WHERE dim1 = 'nonexistent'),\n"
|
||||
+ " STRING_AGG(DISTINCT dim3, '|') FILTER(WHERE dim1 = 'nonexistent'),\n"
|
||||
+ " BIT_AND(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
|
||||
+ " BIT_OR(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
|
||||
+ " BIT_XOR(l1) FILTER(WHERE dim1 = 'nonexistent')\n"
|
||||
|
@ -13013,7 +13032,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"array_set_add_all(\"__acc\", \"a6\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
|
@ -13021,19 +13040,22 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a7",
|
||||
ImmutableSet.of("l1"),
|
||||
ImmutableSet.of("dim3"),
|
||||
"__acc",
|
||||
"0",
|
||||
"0",
|
||||
NullHandling.sqlCompatible(),
|
||||
"bitwiseAnd(\"__acc\", \"l1\")",
|
||||
"bitwiseAnd(\"__acc\", \"a7\")",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_set_add(\"__acc\", \"dim3\")",
|
||||
"array_set_add_all(\"__acc\", \"a7\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
"if(array_length(o) == 0, null, array_to_string(o, '|'))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null))
|
||||
and(
|
||||
not(selector("dim3", null, null)),
|
||||
selector("dim1", "nonexistent", null)
|
||||
)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
|
@ -13043,11 +13065,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"0",
|
||||
"0",
|
||||
NullHandling.sqlCompatible(),
|
||||
"bitwiseOr(\"__acc\", \"l1\")",
|
||||
"bitwiseOr(\"__acc\", \"a8\")",
|
||||
"bitwiseAnd(\"__acc\", \"l1\")",
|
||||
"bitwiseAnd(\"__acc\", \"a8\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null))
|
||||
|
@ -13060,11 +13082,28 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"0",
|
||||
"0",
|
||||
NullHandling.sqlCompatible(),
|
||||
"bitwiseOr(\"__acc\", \"l1\")",
|
||||
"bitwiseOr(\"__acc\", \"a9\")",
|
||||
null,
|
||||
null,
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a10",
|
||||
ImmutableSet.of("l1"),
|
||||
"__acc",
|
||||
"0",
|
||||
"0",
|
||||
NullHandling.sqlCompatible(),
|
||||
"bitwiseXor(\"__acc\", \"l1\")",
|
||||
"bitwiseXor(\"__acc\", \"a9\")",
|
||||
"bitwiseXor(\"__acc\", \"a10\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null))
|
||||
|
@ -13076,8 +13115,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
),
|
||||
ImmutableList.of(
|
||||
useDefault
|
||||
? new Object[]{"a", "", 0L, "", 0L, "", 0L, null, 0L, 0L, 0L}
|
||||
: new Object[]{"a", null, null, null, null, null, null, null, null, null, null}
|
||||
? new Object[]{"a", "", 0L, "", 0L, "", 0L, null, "", 0L, 0L, 0L}
|
||||
: new Object[]{"a", null, null, null, null, null, null, null, null, null, null, null}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -17932,7 +17971,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"bitwiseAnd(\"__acc\", \"a0\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
|
@ -17949,7 +17988,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"bitwiseOr(\"__acc\", \"a1\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
|
@ -17966,7 +18005,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"bitwiseXor(\"__acc\", \"a2\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
|
@ -18015,7 +18054,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"bitwiseAnd(\"__acc\", \"a0\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
|
@ -18032,7 +18071,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"bitwiseOr(\"__acc\", \"a1\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
|
@ -18049,7 +18088,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"bitwiseXor(\"__acc\", \"a2\")",
|
||||
null,
|
||||
null,
|
||||
new HumanReadableBytes(1024),
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
|
@ -18261,4 +18300,418 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStringAgg() throws Exception
|
||||
{
|
||||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT STRING_AGG(dim1,','), STRING_AGG(DISTINCT dim1, ','), STRING_AGG(DISTINCT dim1,',') FILTER(WHERE dim1 = 'shazbot') FROM foo WHERE dim1 is not null",
|
||||
ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
.filters(not(selector("dim1", null, null)))
|
||||
.aggregators(
|
||||
aggregators(
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a0",
|
||||
ImmutableSet.of("dim1"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_append(\"__acc\", \"dim1\")",
|
||||
"array_concat(\"__acc\", \"a0\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("dim1", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a1",
|
||||
ImmutableSet.of("dim1"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_set_add(\"__acc\", \"dim1\")",
|
||||
"array_set_add_all(\"__acc\", \"a1\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("dim1", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a2",
|
||||
ImmutableSet.of("dim1"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_set_add(\"__acc\", \"dim1\")",
|
||||
"array_set_add_all(\"__acc\", \"a2\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
and(
|
||||
not(selector("dim1", null, null)),
|
||||
selector("dim1", "shazbot", null)
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
useDefault
|
||||
? new Object[]{"10.1,2,1,def,abc", "1,2,abc,def,10.1", ""}
|
||||
: new Object[]{",10.1,2,1,def,abc", ",1,2,abc,def,10.1", null}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStringAggMultiValue() throws Exception
|
||||
{
|
||||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT STRING_AGG(dim3, ','), STRING_AGG(DISTINCT dim3, ',') FROM foo",
|
||||
ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(
|
||||
aggregators(
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a0",
|
||||
ImmutableSet.of("dim3"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_append(\"__acc\", \"dim3\")",
|
||||
"array_concat(\"__acc\", \"a0\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("dim3", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a1",
|
||||
ImmutableSet.of("dim3"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_set_add(\"__acc\", \"dim3\")",
|
||||
"array_set_add_all(\"__acc\", \"a1\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("dim3", null, null))
|
||||
)
|
||||
)
|
||||
)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
useDefault
|
||||
? new Object[]{"a,b,b,c,d", "a,b,c,d"}
|
||||
: new Object[]{"a,b,b,c,d,", ",a,b,c,d"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStringAggNumeric() throws Exception
|
||||
{
|
||||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT STRING_AGG(l1, ','), STRING_AGG(DISTINCT l1, ','), STRING_AGG(d1, ','), STRING_AGG(DISTINCT d1, ','), STRING_AGG(f1, ','), STRING_AGG(DISTINCT f1, ',') FROM numfoo",
|
||||
ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(
|
||||
aggregators(
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a0",
|
||||
ImmutableSet.of("l1"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_append(\"__acc\", \"l1\")",
|
||||
"array_concat(\"__acc\", \"a0\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a1",
|
||||
ImmutableSet.of("l1"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_set_add(\"__acc\", \"l1\")",
|
||||
"array_set_add_all(\"__acc\", \"a1\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a2",
|
||||
ImmutableSet.of("d1"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_append(\"__acc\", \"d1\")",
|
||||
"array_concat(\"__acc\", \"a2\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("d1", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a3",
|
||||
ImmutableSet.of("d1"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_set_add(\"__acc\", \"d1\")",
|
||||
"array_set_add_all(\"__acc\", \"a3\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("d1", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a4",
|
||||
ImmutableSet.of("f1"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_append(\"__acc\", \"f1\")",
|
||||
"array_concat(\"__acc\", \"a4\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("f1", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a5",
|
||||
ImmutableSet.of("f1"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_set_add(\"__acc\", \"f1\")",
|
||||
"array_set_add_all(\"__acc\", \"a5\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("f1", null, null))
|
||||
)
|
||||
)
|
||||
)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
useDefault
|
||||
? new Object[]{
|
||||
"7,325323,0,0,0,0",
|
||||
"0,7,325323",
|
||||
"1.0,1.7,0.0,0.0,0.0,0.0",
|
||||
"0.0,1.0,1.7",
|
||||
"1.0,0.10000000149011612,0.0,0.0,0.0,0.0",
|
||||
"0.10000000149011612,0.0,1.0"
|
||||
}
|
||||
: new Object[]{
|
||||
"7,325323,0",
|
||||
"0,7,325323",
|
||||
"1.0,1.7,0.0",
|
||||
"0.0,1.0,1.7",
|
||||
"1.0,0.10000000149011612,0.0",
|
||||
"0.10000000149011612,0.0,1.0"
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStringAggExpression() throws Exception
|
||||
{
|
||||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT STRING_AGG(DISTINCT CONCAT(dim1, dim2), ','), STRING_AGG(DISTINCT CONCAT(dim1, dim2), CONCAT('|', '|')) FROM foo",
|
||||
ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
.virtualColumns(
|
||||
expressionVirtualColumn("v0", "concat(\"dim1\",\"dim2\")", ValueType.STRING)
|
||||
)
|
||||
.aggregators(
|
||||
aggregators(
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a0",
|
||||
ImmutableSet.of("v0"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_set_add(\"__acc\", \"v0\")",
|
||||
"array_set_add_all(\"__acc\", \"a0\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("v0", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a1",
|
||||
ImmutableSet.of("v0"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_set_add(\"__acc\", \"v0\")",
|
||||
"array_set_add_all(\"__acc\", \"a1\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, '||'))",
|
||||
ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("v0", null, null))
|
||||
)
|
||||
)
|
||||
)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
useDefault ? new Object[]{"1a,a,2,abc,10.1,defabc", "1a||a||2||abc||10.1||defabc"} : new Object[]{"1a,a,2,defabc", "1a||a||2||defabc"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test(expected = RelOptPlanner.CannotPlanException.class)
|
||||
public void testStringAggExpressionNonConstantSeparator() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT STRING_AGG(DISTINCT CONCAT(dim1, dim2), CONCAT('|', dim1)) FROM foo",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStringAggMaxBytes() throws Exception
|
||||
{
|
||||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT STRING_AGG(l1, ',', 128), STRING_AGG(DISTINCT l1, ',', 128) FROM numfoo",
|
||||
ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(
|
||||
aggregators(
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a0",
|
||||
ImmutableSet.of("l1"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_append(\"__acc\", \"l1\")",
|
||||
"array_concat(\"__acc\", \"a0\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
new HumanReadableBytes(128),
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
"a1",
|
||||
ImmutableSet.of("l1"),
|
||||
"__acc",
|
||||
"[]",
|
||||
"[]",
|
||||
true,
|
||||
"array_set_add(\"__acc\", \"l1\")",
|
||||
"array_set_add_all(\"__acc\", \"a1\")",
|
||||
null,
|
||||
"if(array_length(o) == 0, null, array_to_string(o, ','))",
|
||||
new HumanReadableBytes(128),
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
not(selector("l1", null, null))
|
||||
)
|
||||
)
|
||||
)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
useDefault
|
||||
? new Object[]{"7,325323,0,0,0,0", "0,7,325323"}
|
||||
: new Object[]{"7,325323,0", "0,7,325323"}
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1547,6 +1547,7 @@ SCHEMA_OWNER
|
|||
SERVER_SEGMENTS
|
||||
SMALLINT
|
||||
SQL_PATH
|
||||
STRING_AGG
|
||||
SYSTEM_TABLE
|
||||
TABLE_CATALOG
|
||||
TABLE_NAME
|
||||
|
|
Loading…
Reference in New Issue