You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by cw...@apache.org on 2021/08/10 20:47:30 UTC

[druid] branch master updated: STRING_AGG SQL aggregator function (#11241)

This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/druid.git


The following commit(s) were added to refs/heads/master by this push:
     new 9af7ba9  STRING_AGG SQL aggregator function (#11241)
9af7ba9 is described below

commit 9af7ba9d2a35e0fe7fc60cb0c737d10ffa794190
Author: Clint Wylie <cw...@apache.org>
AuthorDate: Tue Aug 10 13:47:09 2021 -0700

    STRING_AGG SQL aggregator function (#11241)
    
    * add string_agg
    
    * oops
    
    * style and fix test
    
    * spelling
    
    * fixup
    
    * review stuffs
---
 docs/querying/sql.md                               |   8 +-
 .../ExpressionLambdaAggregatorFactory.java         |   2 +-
 .../aggregation/builtin/ArraySqlAggregator.java    |   4 +-
 ...SqlAggregator.java => StringSqlAggregator.java} | 172 ++++---
 .../sql/calcite/planner/DruidOperatorTable.java    |   2 +
 .../druid/sql/calcite/CalciteArraysQueryTest.java  |  32 +-
 .../apache/druid/sql/calcite/CalciteQueryTest.java | 511 +++++++++++++++++++--
 website/.spelling                                  |   1 +
 8 files changed, 594 insertions(+), 138 deletions(-)

diff --git a/docs/querying/sql.md b/docs/querying/sql.md
index d4823cb..7783e5d 100644
--- a/docs/querying/sql.md
+++ b/docs/querying/sql.md
@@ -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 o [...]
+|`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 re [...]
 |`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`|
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactory.java
index 5d3c790..273c3b7 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactory.java
@@ -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
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java
index 6c0b4d0..9359b6e 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java
@@ -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)
             )
           ),
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java
similarity index 55%
copy from sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java
copy to sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java
index 6c0b4d0..596f69b 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java
@@ -22,25 +22,24 @@ 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.rel.type.RelDataType;
 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.SqlOperatorBinding;
 import org.apache.calcite.sql.type.InferTypes;
 import org.apache.calcite.sql.type.OperandTypes;
-import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeFamily;
-import org.apache.calcite.sql.type.SqlTypeUtil;
+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.ISE;
 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;
@@ -57,10 +56,10 @@ import java.util.List;
 import java.util.Objects;
 import java.util.stream.Collectors;
 
-public class ArraySqlAggregator implements SqlAggregator
+public class StringSqlAggregator implements SqlAggregator
 {
-  private static final String NAME = "ARRAY_AGG";
-  private static final SqlAggFunction FUNCTION = new ArrayAggFunction();
+  private static final String NAME = "STRING_AGG";
+  private static final SqlAggFunction FUNCTION = new StringAggFunction();
 
   @Override
   public SqlAggFunction calciteFunction()
@@ -82,7 +81,6 @@ public class ArraySqlAggregator implements SqlAggregator
       boolean finalizeAggregations
   )
   {
-
     final List<DruidExpression> arguments = aggregateCall
         .getArgList()
         .stream()
@@ -94,12 +92,28 @@ public class ArraySqlAggregator implements SqlAggregator
       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() > 1) {
+    if (arguments.size() > 2) {
       RexNode maxBytes = Expressions.fromFieldAccess(
           rowSignature,
           project,
-          aggregateCall.getArgList().get(1)
+          aggregateCall.getArgList().get(2)
       );
       if (!maxBytes.isA(SqlKind.LITERAL)) {
         // maxBytes must be a literal
@@ -110,29 +124,9 @@ public class ArraySqlAggregator implements SqlAggregator
     final DruidExpression arg = arguments.get(0);
     final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
 
+    final String initialvalue = "[]";
+    final ValueType elementType = ValueType.STRING;
     final String fieldName;
-    final String initialvalue;
-    final ValueType elementType;
-    final ValueType druidType = Calcites.getValueTypeForRelDataTypeFull(aggregateCall.getType());
-    if (druidType == null) {
-      initialvalue = "[]";
-      elementType = ValueType.STRING;
-    } else {
-      switch (druidType) {
-        case LONG_ARRAY:
-          initialvalue = "<LONG>[]";
-          elementType = ValueType.LONG;
-          break;
-        case DOUBLE_ARRAY:
-          initialvalue = "<DOUBLE>[]";
-          elementType = ValueType.DOUBLE;
-          break;
-        default:
-          initialvalue = "[]";
-          elementType = ValueType.STRING;
-          break;
-      }
-    }
     if (arg.isDirectColumnAccess()) {
       fieldName = arg.getDirectColumn();
     } else {
@@ -140,80 +134,84 @@ public class ArraySqlAggregator implements SqlAggregator
       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(
-          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,
-              null,
-              maxSizeBytes != null ? new HumanReadableBytes(maxSizeBytes) : null,
-              macroTable
+          // 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(
-          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,
-              null,
-              maxSizeBytes != null ? new HumanReadableBytes(maxSizeBytes) : null,
-              macroTable
+          // 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
           )
       );
     }
   }
 
-  static class ArrayAggReturnTypeInference implements SqlReturnTypeInference
+  private static class StringAggFunction extends SqlAggFunction
   {
-    @Override
-    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);
-      }
-      return Calcites.createSqlArrayTypeWithNullability(
-          sqlOperatorBinding.getTypeFactory(),
-          type.getSqlTypeName(),
-          true
-      );
-    }
-  }
-
-  private static class ArrayAggFunction extends SqlAggFunction
-  {
-    private static final ArrayAggReturnTypeInference RETURN_TYPE_INFERENCE = new ArrayAggReturnTypeInference();
-
-    ArrayAggFunction()
+    StringAggFunction()
     {
       super(
           NAME,
           null,
           SqlKind.OTHER_FUNCTION,
-          RETURN_TYPE_INFERENCE,
+          opBinding ->
+              Calcites.createSqlTypeWithNullability(opBinding.getTypeFactory(), SqlTypeName.VARCHAR, true),
           InferTypes.ANY_NULLABLE,
           OperandTypes.or(
-            OperandTypes.ANY,
-            OperandTypes.and(
-                OperandTypes.sequence(StringUtils.format("'%s'(expr, maxSizeBytes)", NAME), OperandTypes.ANY, OperandTypes.LITERAL),
-                OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC)
-            )
+              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.USER_DEFINED_FUNCTION,
+          SqlFunctionCategory.STRING,
           false,
           false,
           Optionality.IGNORED
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java
index 3e9288a..221bc95 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java
@@ -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))
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java
index 00aebaf..78382b2 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java
@@ -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
                                                   )
                                               )
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
index d05b150..6471fab 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
@@ -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,56 +12760,73 @@ 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",
+                                  "a9",
                                   ImmutableSet.of("l1"),
                                   "__acc",
                                   "0",
                                   "0",
                                   NullHandling.sqlCompatible(),
                                   "bitwiseOr(\"__acc\", \"l1\")",
-                                  "bitwiseOr(\"__acc\", \"a8\")",
+                                  "bitwiseOr(\"__acc\", \"a9\")",
                                   null,
                                   null,
-                                  new HumanReadableBytes(1024),
+                                  ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
                                   TestExprMacroTable.INSTANCE
                               ),
                               not(selector("l1", null, null))
                           ),
                           new FilteredAggregatorFactory(
                               new ExpressionLambdaAggregatorFactory(
-                                  "a9",
+                                  "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,50 +13040,70 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                                 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
+                                    ),
+                                    and(
+                                        not(selector("dim3", null, null)),
+                                        selector("dim1", "nonexistent", null)
+                                    )
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new ExpressionLambdaAggregatorFactory(
+                                        "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
                                     ),
                                     and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null))
                                 ),
                                 new FilteredAggregatorFactory(
                                     new ExpressionLambdaAggregatorFactory(
-                                        "a8",
+                                        "a9",
                                         ImmutableSet.of("l1"),
                                         "__acc",
                                         "0",
                                         "0",
                                         NullHandling.sqlCompatible(),
                                         "bitwiseOr(\"__acc\", \"l1\")",
-                                        "bitwiseOr(\"__acc\", \"a8\")",
+                                        "bitwiseOr(\"__acc\", \"a9\")",
                                         null,
                                         null,
-                                        new HumanReadableBytes(1024),
+                                        ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES,
                                         TestExprMacroTable.INSTANCE
                                     ),
                                     and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null))
                                 ),
                                 new FilteredAggregatorFactory(
                                     new ExpressionLambdaAggregatorFactory(
-                                        "a9",
+                                        "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"}
+        )
+    );
+  }
 }
diff --git a/website/.spelling b/website/.spelling
index 6d66b09..ce8feb2 100644
--- a/website/.spelling
+++ b/website/.spelling
@@ -1547,6 +1547,7 @@ SCHEMA_OWNER
 SERVER_SEGMENTS
 SMALLINT
 SQL_PATH
+STRING_AGG
 SYSTEM_TABLE
 TABLE_CATALOG
 TABLE_NAME

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org