Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 5 additions & 3 deletions docs/querying/sql.md
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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`|
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

looks like an extra change - can't find its usage

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it wasn't meant to be, its used in tests now


private final String name;
@Nullable
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,7 @@ public RelDataType inferReturnType(SqlOperatorBinding sqlOperatorBinding)
{
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(),
Expand All @@ -209,7 +209,7 @@ private static class ArrayAggFunction extends SqlAggFunction
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)
)
),
Expand Down
Original file line number Diff line number Diff line change
@@ -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);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should be aggregationColumnArg

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(
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the if-else can be collapsed

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah, it could if i pull out the two expressions into a separate conditional block; I did not make this change yet because it seems like minor shifting of the condition but will modify here (as well as ARRAY_AGG since I did the same thing there) if you feel more strongly about it.

// 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
);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
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;
Expand Down Expand Up @@ -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))
Expand Down
Loading