bitwise aggregators, better null handling options for expression agg (#11280)

* bitwise aggregators, better nulls for expression agg

* correct behavior

* rework deserialize, better names

* fix json, share mask
diff --git a/core/src/main/java/org/apache/druid/math/expr/ExprEval.java b/core/src/main/java/org/apache/druid/math/expr/ExprEval.java
index 43a1454..8803583 100644
--- a/core/src/main/java/org/apache/druid/math/expr/ExprEval.java
+++ b/core/src/main/java/org/apache/druid/math/expr/ExprEval.java
@@ -47,9 +47,18 @@
    */
   public static ExprEval deserialize(ByteBuffer buffer, int position)
   {
-    // | expression type (byte) | expression bytes |
-    ExprType type = ExprType.fromByte(buffer.get(position));
-    int offset = position + 1;
+    final ExprType type = ExprType.fromByte(buffer.get(position));
+    return deserialize(buffer, position + 1, type);
+  }
+
+  /**
+   * Deserialize an expression stored in a bytebuffer, e.g. for an agg.
+   *
+   * This should be refactored to be consolidated with some of the standard type handling of aggregators probably
+   */
+  public static ExprEval deserialize(ByteBuffer buffer, int offset, ExprType type)
+  {
+    // | expression bytes |
     switch (type) {
       case LONG:
         // | expression type (byte) | is null (byte) | long bytes |
diff --git a/core/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java b/core/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java
index 0f1289b..fa3699d 100644
--- a/core/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java
+++ b/core/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java
@@ -297,9 +297,23 @@
   {
     ExprEval.serialize(buffer, position, expected, maxSizeBytes);
     if (ExprType.isArray(expected.type())) {
-      Assert.assertArrayEquals(expected.asArray(), ExprEval.deserialize(buffer, position).asArray());
+      Assert.assertArrayEquals(
+          expected.asArray(),
+          ExprEval.deserialize(buffer, position + 1, ExprType.fromByte(buffer.get(position))).asArray()
+      );
+      Assert.assertArrayEquals(
+          expected.asArray(),
+          ExprEval.deserialize(buffer, position).asArray()
+      );
     } else {
-      Assert.assertEquals(expected.value(), ExprEval.deserialize(buffer, position).value());
+      Assert.assertEquals(
+          expected.value(),
+          ExprEval.deserialize(buffer, position + 1, ExprType.fromByte(buffer.get(position))).value()
+      );
+      Assert.assertEquals(
+          expected.value(),
+          ExprEval.deserialize(buffer, position).value()
+      );
     }
     assertEstimatedBytes(expected, maxSizeBytes);
   }
diff --git a/docs/querying/sql.md b/docs/querying/sql.md
index 12d0da9..fd5903d 100644
--- a/docs/querying/sql.md
+++ b/docs/querying/sql.md
@@ -360,6 +360,9 @@
 |`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`|
+|`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`|
 
 For advice on choosing approximate aggregation functions, check out our [approximate aggregations documentation](aggregations.md#approx).
 
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregator.java
index 59bd6f2..3e56b5d 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregator.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregator.java
@@ -29,12 +29,19 @@
   private final Expr lambda;
   private final ExpressionLambdaAggregatorInputBindings bindings;
   private final int maxSizeBytes;
+  private boolean hasValue;
 
-  public ExpressionLambdaAggregator(Expr lambda, ExpressionLambdaAggregatorInputBindings bindings, int maxSizeBytes)
+  public ExpressionLambdaAggregator(
+      final Expr lambda,
+      final ExpressionLambdaAggregatorInputBindings bindings,
+      final boolean isNullUnlessAggregated,
+      final int maxSizeBytes
+  )
   {
     this.lambda = lambda;
     this.bindings = bindings;
     this.maxSizeBytes = maxSizeBytes;
+    this.hasValue = !isNullUnlessAggregated;
   }
 
   @Override
@@ -43,13 +50,14 @@
     final ExprEval<?> eval = lambda.eval(bindings);
     ExprEval.estimateAndCheckMaxBytes(eval, maxSizeBytes);
     bindings.accumulate(eval);
+    hasValue = true;
   }
 
   @Nullable
   @Override
   public Object get()
   {
-    return bindings.getAccumulator().value();
+    return hasValue ? bindings.getAccumulator().value() : null;
   }
 
   @Override
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 fd918fe..5d3c790 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
@@ -27,6 +27,7 @@
 import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
+import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.HumanReadableBytes;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.guava.Comparators;
@@ -74,6 +75,7 @@
   private final String foldExpressionString;
   private final String initialValueExpressionString;
   private final String initialCombineValueExpressionString;
+  private final boolean isNullUnlessAggregated;
 
   private final String combineExpressionString;
   @Nullable
@@ -105,6 +107,7 @@
       @JsonProperty("accumulatorIdentifier") @Nullable final String accumulatorIdentifier,
       @JsonProperty("initialValue") final String initialValue,
       @JsonProperty("initialCombineValue") @Nullable final String initialCombineValue,
+      @JsonProperty("isNullUnlessAggregated") @Nullable final Boolean isNullUnlessAggregated,
       @JsonProperty("fold") final String foldExpression,
       @JsonProperty("combine") @Nullable final String combineExpression,
       @JsonProperty("compare") @Nullable final String compareExpression,
@@ -121,6 +124,7 @@
 
     this.initialValueExpressionString = initialValue;
     this.initialCombineValueExpressionString = initialCombineValue == null ? initialValue : initialCombineValue;
+    this.isNullUnlessAggregated = isNullUnlessAggregated == null ? NullHandling.sqlCompatible() : isNullUnlessAggregated;
     this.foldExpressionString = foldExpression;
     if (combineExpression != null) {
       this.combineExpressionString = combineExpression;
@@ -195,6 +199,12 @@
     return initialCombineValueExpressionString;
   }
 
+  @JsonProperty("isNullUnlessAggregated")
+  public boolean getIsNullUnlessAggregated()
+  {
+    return isNullUnlessAggregated;
+  }
+
   @JsonProperty("fold")
   public String getFoldExpressionString()
   {
@@ -249,6 +259,7 @@
     return new ExpressionLambdaAggregator(
         thePlan.getExpression(),
         thePlan.getBindings(),
+        isNullUnlessAggregated,
         maxSizeBytes.getBytesInInt()
     );
   }
@@ -261,6 +272,7 @@
         thePlan.getExpression(),
         thePlan.getInitialValue(),
         thePlan.getBindings(),
+        isNullUnlessAggregated,
         maxSizeBytes.getBytesInInt()
     );
   }
@@ -329,6 +341,7 @@
         accumulatorId,
         initialValueExpressionString,
         initialCombineValueExpressionString,
+        isNullUnlessAggregated,
         foldExpressionString,
         combineExpressionString,
         compareExpressionString,
@@ -348,6 +361,7 @@
             accumulatorId,
             initialValueExpressionString,
             initialCombineValueExpressionString,
+            isNullUnlessAggregated,
             foldExpressionString,
             combineExpressionString,
             compareExpressionString,
@@ -407,6 +421,7 @@
            && foldExpressionString.equals(that.foldExpressionString)
            && initialValueExpressionString.equals(that.initialValueExpressionString)
            && initialCombineValueExpressionString.equals(that.initialCombineValueExpressionString)
+           && isNullUnlessAggregated == that.isNullUnlessAggregated
            && combineExpressionString.equals(that.combineExpressionString)
            && Objects.equals(compareExpressionString, that.compareExpressionString)
            && Objects.equals(finalizeExpressionString, that.finalizeExpressionString);
@@ -422,6 +437,7 @@
         foldExpressionString,
         initialValueExpressionString,
         initialCombineValueExpressionString,
+        isNullUnlessAggregated,
         combineExpressionString,
         compareExpressionString,
         finalizeExpressionString,
@@ -439,6 +455,7 @@
            ", foldExpressionString='" + foldExpressionString + '\'' +
            ", initialValueExpressionString='" + initialValueExpressionString + '\'' +
            ", initialCombineValueExpressionString='" + initialCombineValueExpressionString + '\'' +
+           ", nullUnlessAggregated='" + isNullUnlessAggregated + '\'' +
            ", combineExpressionString='" + combineExpressionString + '\'' +
            ", compareExpressionString='" + compareExpressionString + '\'' +
            ", finalizeExpressionString='" + finalizeExpressionString + '\'' +
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaBufferAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaBufferAggregator.java
index 357dd4b..82b954e 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaBufferAggregator.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaBufferAggregator.java
@@ -21,27 +21,33 @@
 
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
+import org.apache.druid.math.expr.ExprType;
 
 import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
 
 public class ExpressionLambdaBufferAggregator implements BufferAggregator
 {
+  private static final short NOT_AGGREGATED_BIT = 1 << 7;
+  private static final short IS_AGGREGATED_MASK = 0x3F;
   private final Expr lambda;
   private final ExprEval<?> initialValue;
   private final ExpressionLambdaAggregatorInputBindings bindings;
   private final int maxSizeBytes;
+  private final boolean isNullUnlessAggregated;
 
   public ExpressionLambdaBufferAggregator(
       Expr lambda,
       ExprEval<?> initialValue,
       ExpressionLambdaAggregatorInputBindings bindings,
+      boolean isNullUnlessAggregated,
       int maxSizeBytes
   )
   {
     this.lambda = lambda;
     this.initialValue = initialValue;
     this.bindings = bindings;
+    this.isNullUnlessAggregated = isNullUnlessAggregated;
     this.maxSizeBytes = maxSizeBytes;
   }
 
@@ -49,40 +55,49 @@
   public void init(ByteBuffer buf, int position)
   {
     ExprEval.serialize(buf, position, initialValue, maxSizeBytes);
+    // set a bit to indicate we haven't aggregated on top of expression type (not going to lie this could be nicer)
+    if (isNullUnlessAggregated) {
+      buf.put(position, (byte) (buf.get(position) | NOT_AGGREGATED_BIT));
+    }
   }
 
   @Override
   public void aggregate(ByteBuffer buf, int position)
   {
-    ExprEval<?> acc = ExprEval.deserialize(buf, position);
+    ExprEval<?> acc = ExprEval.deserialize(buf, position + 1, getType(buf, position));
     bindings.setAccumulator(acc);
     ExprEval<?> newAcc = lambda.eval(bindings);
     ExprEval.serialize(buf, position, newAcc, maxSizeBytes);
+    // scrub not aggregated bit
+    buf.put(position, (byte) (buf.get(position) & IS_AGGREGATED_MASK));
   }
 
   @Nullable
   @Override
   public Object get(ByteBuffer buf, int position)
   {
-    return ExprEval.deserialize(buf, position).value();
+    if (isNullUnlessAggregated && (buf.get(position) & NOT_AGGREGATED_BIT) != 0) {
+      return null;
+    }
+    return ExprEval.deserialize(buf, position + 1, getType(buf, position)).value();
   }
 
   @Override
   public float getFloat(ByteBuffer buf, int position)
   {
-    return (float) ExprEval.deserialize(buf, position).asDouble();
+    return (float) ExprEval.deserialize(buf, position + 1, getType(buf, position)).asDouble();
   }
 
   @Override
   public double getDouble(ByteBuffer buf, int position)
   {
-    return ExprEval.deserialize(buf, position).asDouble();
+    return ExprEval.deserialize(buf, position + 1, getType(buf, position)).asDouble();
   }
 
   @Override
   public long getLong(ByteBuffer buf, int position)
   {
-    return ExprEval.deserialize(buf, position).asLong();
+    return ExprEval.deserialize(buf, position + 1, getType(buf, position)).asLong();
   }
 
   @Override
@@ -90,4 +105,9 @@
   {
     // nothing to close
   }
+
+  private static ExprType getType(ByteBuffer buf, int position)
+  {
+    return ExprType.fromByte((byte) (buf.get(position) & IS_AGGREGATED_MASK));
+  }
 }
diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactoryTest.java
index a414379..f4e8a14 100644
--- a/processing/src/test/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactoryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactoryTest.java
@@ -58,6 +58,7 @@
         "customAccumulator",
         "0.0",
         "10.0",
+        true,
         "customAccumulator + some_column + some_other_column",
         "customAccumulator + expr_agg_name",
         "if (o1 > o2, if (o1 == o2, 0, 1), -1)",
@@ -102,6 +103,7 @@
         null,
         "x + y",
         null,
+        true,
         "__acc + some_column + some_other_column",
         "__acc + expr_agg_name",
         null,
@@ -125,6 +127,7 @@
         null,
         "0.0",
         "x + y",
+        true,
         "__acc + some_column + some_other_column",
         "__acc + expr_agg_name",
         null,
@@ -145,6 +148,7 @@
         null,
         "0",
         null,
+        true,
         "__acc + x",
         null,
         null,
@@ -165,6 +169,7 @@
         null,
         "0",
         null,
+        true,
         "__acc + x",
         null,
         null,
@@ -185,6 +190,7 @@
         null,
         "0",
         null,
+        true,
         "array_set_add(__acc, x)",
         "array_set_add_all(__acc, expr_agg_name)",
         null,
@@ -206,6 +212,7 @@
         null,
         "''",
         "''",
+        true,
         "concat(__acc, some_column, some_other_column)",
         "concat(__acc, expr_agg_name)",
         null,
@@ -228,6 +235,7 @@
         null,
         "0",
         null,
+        null,
         "__acc + some_column + some_other_column",
         "__acc + expr_agg_name",
         null,
@@ -250,6 +258,7 @@
         null,
         "0.0",
         null,
+        null,
         "__acc + some_column + some_other_column",
         "__acc + expr_agg_name",
         null,
@@ -272,6 +281,7 @@
         null,
         "''",
         "<STRING>[]",
+        null,
         "concat(__acc, some_column, some_other_column)",
         "array_set_add(__acc, expr_agg_name)",
         null,
@@ -294,6 +304,7 @@
         null,
         "''",
         "<STRING>[]",
+        null,
         "concat(__acc, some_column, some_other_column)",
         "array_set_add(__acc, expr_agg_name)",
         null,
@@ -316,6 +327,7 @@
         null,
         "0",
         "<LONG>[]",
+        null,
         "__acc + some_column + some_other_column",
         "array_set_add(__acc, expr_agg_name)",
         null,
@@ -338,6 +350,7 @@
         null,
         "0",
         "<LONG>[]",
+        null,
         "__acc + some_column + some_other_column",
         "array_set_add(__acc, expr_agg_name)",
         null,
@@ -360,6 +373,7 @@
         null,
         "0.0",
         "<DOUBLE>[]",
+        null,
         "__acc + some_column + some_other_column",
         "array_set_add(__acc, expr_agg_name)",
         null,
@@ -382,6 +396,7 @@
         null,
         "0.0",
         "<DOUBLE>[]",
+        null,
         "__acc + some_column + some_other_column",
         "array_set_add(__acc, expr_agg_name)",
         null,
@@ -410,6 +425,7 @@
                       null,
                       "''",
                       "''",
+                      null,
                       "concat(__acc, some_column, some_other_column)",
                       "concat(__acc, string_expr)",
                       null,
@@ -423,6 +439,7 @@
                       null,
                       "0.0",
                       null,
+                      null,
                       "__acc + some_column + some_other_column",
                       "__acc + double_expr",
                       null,
@@ -436,6 +453,7 @@
                       null,
                       "0",
                       null,
+                      null,
                       "__acc + some_column + some_other_column",
                       "__acc + long_expr",
                       null,
@@ -449,6 +467,7 @@
                       null,
                       "<STRING>[]",
                       "<STRING>[]",
+                      null,
                       "array_set_add(__acc, concat(some_column, some_other_column))",
                       "array_set_add_all(__acc, string_array_expr)",
                       null,
@@ -462,6 +481,7 @@
                       null,
                       "0.0",
                       "<DOUBLE>[]",
+                      null,
                       "__acc + some_column + some_other_column",
                       "array_set_add(__acc, double_array)",
                       null,
@@ -475,6 +495,7 @@
                       null,
                       "0",
                       "<LONG>[]",
+                      null,
                       "__acc + some_column + some_other_column",
                       "array_set_add(__acc, long_array_expr)",
                       null,
@@ -488,6 +509,7 @@
                       null,
                       "''",
                       "<STRING>[]",
+                      null,
                       "concat(__acc, some_column, some_other_column)",
                       "array_set_add(__acc, string_array_expr)",
                       null,
@@ -501,6 +523,7 @@
                       null,
                       "0.0",
                       "<DOUBLE>[]",
+                      null,
                       "__acc + some_column + some_other_column",
                       "array_set_add(__acc, double_array)",
                       null,
@@ -514,6 +537,7 @@
                       null,
                       "0",
                       "<LONG>[]",
+                      null,
                       "__acc + some_column + some_other_column",
                       "array_set_add(__acc, long_array_expr)",
                       null,
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
index bfd40ce..d6c2ca8 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
@@ -11228,6 +11228,7 @@
                 null,
                 "0",
                 null,
+                false,
                 "__acc + 1",
                 "__acc + rows",
                 null,
@@ -11241,6 +11242,7 @@
                 null,
                 "0.0",
                 null,
+                null,
                 "__acc + index",
                 null,
                 null,
@@ -11463,6 +11465,7 @@
                 null,
                 "0",
                 null,
+                false,
                 "__acc + 1",
                 "__acc + rows",
                 null,
@@ -11476,6 +11479,7 @@
                 null,
                 "0.0",
                 null,
+                true,
                 "__acc + index",
                 null,
                 null,
@@ -11489,6 +11493,7 @@
                 "acc",
                 "[]",
                 null,
+                null,
                 "array_set_add(acc, market)",
                 "array_set_add_all(acc, array_agg_distinct)",
                 null,
@@ -11747,6 +11752,7 @@
                 "acc",
                 "[]",
                 null,
+                null,
                 "array_set_add(acc, placementish)",
                 "array_set_add_all(acc, array_agg_distinct)",
                 null,
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
index c1de43c..63e5aa7 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
@@ -3030,6 +3030,7 @@
                                               null,
                                               "0",
                                               null,
+                                              false,
                                               "__acc + 1",
                                               "__acc + diy_count",
                                               null,
@@ -3043,6 +3044,7 @@
                                               null,
                                               "0.0",
                                               null,
+                                              null,
                                               "__acc + index",
                                               null,
                                               null,
@@ -3056,6 +3058,7 @@
                                               null,
                                               "0.0",
                                               "<DOUBLE>[]",
+                                              null,
                                               "__acc + index",
                                               "array_concat(__acc, diy_decomposed_sum)",
                                               null,
@@ -3069,6 +3072,7 @@
                                               "acc",
                                               "[]",
                                               null,
+                                              null,
                                               "array_set_add(acc, market)",
                                               "array_set_add_all(acc, array_agg_distinct)",
                                               null,
@@ -3132,6 +3136,7 @@
                                               "acc",
                                               "[]",
                                               null,
+                                              null,
                                               "array_set_add(acc, market)",
                                               "array_set_add_all(acc, array_agg_distinct)",
                                               null,
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
index 0f1c95c..c24b104 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
@@ -5987,6 +5987,7 @@
                         null,
                         "0",
                         null,
+                        false,
                         "__acc + 1",
                         "__acc + diy_count",
                         null,
@@ -6000,6 +6001,7 @@
                         null,
                         "0.0",
                         null,
+                        null,
                         "__acc + index",
                         null,
                         null,
@@ -6013,6 +6015,7 @@
                         null,
                         "0.0",
                         "<DOUBLE>[]",
+                        null,
                         "__acc + index",
                         "array_concat(__acc, diy_decomposed_sum)",
                         null,
@@ -6026,6 +6029,7 @@
                         "acc",
                         "[]",
                         null,
+                        null,
                         "array_set_add(acc, quality)",
                         "array_set_add_all(acc, array_agg_distinct)",
                         "if(array_length(o1) > array_length(o2), 1, if (array_length(o1) == array_length(o2), 0, -1))",
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 0f80daa..6c0b4d0 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
@@ -148,10 +148,11 @@
               null,
               initialvalue,
               null,
+              true,
               StringUtils.format("array_set_add(\"__acc\", \"%s\")", fieldName),
               StringUtils.format("array_set_add_all(\"__acc\", \"%s\")", name),
               null,
-              "if(array_length(o) == 0, null, o)",
+              null,
               maxSizeBytes != null ? new HumanReadableBytes(maxSizeBytes) : null,
               macroTable
           )
@@ -164,10 +165,11 @@
               null,
               initialvalue,
               null,
+              true,
               StringUtils.format("array_append(\"__acc\", \"%s\")", fieldName),
               StringUtils.format("array_concat(\"__acc\", \"%s\")", name),
               null,
-              "if(array_length(o) == 0, null, o)",
+              null,
               maxSizeBytes != null ? new HumanReadableBytes(maxSizeBytes) : null,
               macroTable
           )
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java
new file mode 100644
index 0000000..a6fe93c
--- /dev/null
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java
@@ -0,0 +1,195 @@
+/*
+ * 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.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Optionality;
+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.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 BitwiseSqlAggregator implements SqlAggregator
+{
+  private static final SqlAggFunction XOR_FUNCTION = new BitwiseXorSqlAggFunction();
+
+  public enum Op
+  {
+    AND {
+      @Override
+      SqlAggFunction getCalciteFunction()
+      {
+        return SqlStdOperatorTable.BIT_AND;
+      }
+
+      @Override
+      String getDruidFunction()
+      {
+        return "bitwiseAnd";
+      }
+    },
+    OR {
+      @Override
+      SqlAggFunction getCalciteFunction()
+      {
+        return SqlStdOperatorTable.BIT_OR;
+      }
+
+      @Override
+      String getDruidFunction()
+      {
+        return "bitwiseOr";
+      }
+    },
+    XOR {
+      @Override
+      SqlAggFunction getCalciteFunction()
+      {
+        // newer versions of calcite have this built-in so someday we can drop this...
+        return XOR_FUNCTION;
+      }
+
+      @Override
+      String getDruidFunction()
+      {
+        return "bitwiseXor";
+      }
+    };
+
+    abstract SqlAggFunction getCalciteFunction();
+    abstract String getDruidFunction();
+  };
+
+  private final Op op;
+
+  public BitwiseSqlAggregator(Op op)
+  {
+    this.op = op;
+  }
+
+  @Override
+  public SqlAggFunction calciteFunction()
+  {
+    return op.getCalciteFunction();
+  }
+
+  @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;
+    }
+
+    final DruidExpression arg = arguments.get(0);
+    final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
+
+    final String fieldName;
+    if (arg.isDirectColumnAccess()) {
+      fieldName = arg.getDirectColumn();
+    } else {
+      VirtualColumn vc = virtualColumnRegistry.getOrCreateVirtualColumnForExpression(plannerContext, arg, ValueType.LONG);
+      fieldName = vc.getOutputName();
+    }
+
+    return Aggregation.create(
+        new FilteredAggregatorFactory(
+            new ExpressionLambdaAggregatorFactory(
+                name,
+                ImmutableSet.of(fieldName),
+                null,
+                "0",
+                null,
+                null,
+                StringUtils.format("%s(\"__acc\", \"%s\")", op.getDruidFunction(), fieldName),
+                null,
+                null,
+                null,
+                null,
+                macroTable
+            ),
+            new NotDimFilter(new SelectorDimFilter(fieldName, null, null))
+        )
+    );
+  }
+
+  private static class BitwiseXorSqlAggFunction extends SqlAggFunction
+  {
+    BitwiseXorSqlAggFunction()
+    {
+      super(
+          "BIT_XOR",
+          null,
+          SqlKind.OTHER_FUNCTION,
+          ReturnTypes.explicit(SqlTypeName.BIGINT),
+          InferTypes.ANY_NULLABLE,
+          OperandTypes.EXACT_NUMERIC,
+          SqlFunctionCategory.NUMERIC,
+          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 8b99dda..3e9288a 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
@@ -36,6 +36,7 @@
 import org.apache.druid.sql.calcite.aggregation.builtin.ApproxCountDistinctSqlAggregator;
 import org.apache.druid.sql.calcite.aggregation.builtin.ArraySqlAggregator;
 import org.apache.druid.sql.calcite.aggregation.builtin.AvgSqlAggregator;
+import org.apache.druid.sql.calcite.aggregation.builtin.BitwiseSqlAggregator;
 import org.apache.druid.sql.calcite.aggregation.builtin.CountSqlAggregator;
 import org.apache.druid.sql.calcite.aggregation.builtin.EarliestLatestAnySqlAggregator;
 import org.apache.druid.sql.calcite.aggregation.builtin.GroupingSqlAggregator;
@@ -136,6 +137,9 @@
                    .add(new SumZeroSqlAggregator())
                    .add(new GroupingSqlAggregator())
                    .add(new ArraySqlAggregator())
+                   .add(new BitwiseSqlAggregator(BitwiseSqlAggregator.Op.AND))
+                   .add(new BitwiseSqlAggregator(BitwiseSqlAggregator.Op.OR))
+                   .add(new BitwiseSqlAggregator(BitwiseSqlAggregator.Op.XOR))
                    .build();
 
 
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 aded4f8..00aebaf 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
@@ -1156,10 +1156,11 @@
                               "__acc",
                               "[]",
                               "[]",
+                              true,
                               "array_append(\"__acc\", \"dim1\")",
                               "array_concat(\"__acc\", \"a0\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
                           ),
@@ -1169,10 +1170,11 @@
                               "__acc",
                               "[]",
                               "[]",
+                              true,
                               "array_set_add(\"__acc\", \"dim1\")",
                               "array_set_add_all(\"__acc\", \"a1\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
                           ),
@@ -1183,10 +1185,11 @@
                                   "__acc",
                                   "[]",
                                   "[]",
+                                  true,
                                   "array_set_add(\"__acc\", \"dim1\")",
                                   "array_set_add_all(\"__acc\", \"a2\")",
                                   null,
-                                  "if(array_length(o) == 0, null, o)",
+                                  null,
                                   new HumanReadableBytes(1024),
                                   TestExprMacroTable.INSTANCE
                               ),
@@ -1228,10 +1231,11 @@
                               "__acc",
                               "[]",
                               "[]",
+                              true,
                               "array_append(\"__acc\", \"dim3\")",
                               "array_concat(\"__acc\", \"a0\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
                           ),
@@ -1241,10 +1245,11 @@
                               "__acc",
                               "[]",
                               "[]",
+                              true,
                               "array_set_add(\"__acc\", \"dim3\")",
                               "array_set_add_all(\"__acc\", \"a1\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
                           )
@@ -1280,10 +1285,11 @@
                               "__acc",
                               "<LONG>[]",
                               "<LONG>[]",
+                              true,
                               "array_append(\"__acc\", \"l1\")",
                               "array_concat(\"__acc\", \"a0\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
                           ),
@@ -1293,10 +1299,11 @@
                               "__acc",
                               "<LONG>[]",
                               "<LONG>[]",
+                              true,
                               "array_set_add(\"__acc\", \"l1\")",
                               "array_set_add_all(\"__acc\", \"a1\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
                           ),
@@ -1306,10 +1313,11 @@
                               "__acc",
                               "<DOUBLE>[]",
                               "<DOUBLE>[]",
+                              true,
                               "array_append(\"__acc\", \"d1\")",
                               "array_concat(\"__acc\", \"a2\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
                           ),
@@ -1319,10 +1327,11 @@
                               "__acc",
                               "<DOUBLE>[]",
                               "<DOUBLE>[]",
+                              true,
                               "array_set_add(\"__acc\", \"d1\")",
                               "array_set_add_all(\"__acc\", \"a3\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
                           ),
@@ -1332,10 +1341,11 @@
                               "__acc",
                               "<DOUBLE>[]",
                               "<DOUBLE>[]",
+                              true,
                               "array_append(\"__acc\", \"f1\")",
                               "array_concat(\"__acc\", \"a4\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
                           ),
@@ -1345,10 +1355,11 @@
                               "__acc",
                               "<DOUBLE>[]",
                               "<DOUBLE>[]",
+                              true,
                               "array_set_add(\"__acc\", \"f1\")",
                               "array_set_add_all(\"__acc\", \"a5\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
                           )
@@ -1399,10 +1410,11 @@
                               "__acc",
                               "[]",
                               "[]",
+                              true,
                               "array_set_add(\"__acc\", \"dim1\")",
                               "array_set_add_all(\"__acc\", \"a0\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
                           )
@@ -1440,10 +1452,11 @@
                               "__acc",
                               "[]",
                               "[]",
+                              true,
                               "array_set_add(\"__acc\", \"v0\")",
                               "array_set_add_all(\"__acc\", \"a0\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
                           )
@@ -1478,10 +1491,11 @@
                               "__acc",
                               "<LONG>[]",
                               "<LONG>[]",
+                              true,
                               "array_append(\"__acc\", \"l1\")",
                               "array_concat(\"__acc\", \"a0\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(128),
                               TestExprMacroTable.INSTANCE
                           ),
@@ -1491,10 +1505,11 @@
                               "__acc",
                               "<LONG>[]",
                               "<LONG>[]",
+                              true,
                               "array_set_add(\"__acc\", \"l1\")",
                               "array_set_add_all(\"__acc\", \"a1\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(128),
                               TestExprMacroTable.INSTANCE
                           )
@@ -1557,10 +1572,11 @@
                                                       "__acc",
                                                       "[]",
                                                       "[]",
+                                                      true,
                                                       "array_set_add(\"__acc\", \"dim1\")",
                                                       "array_set_add_all(\"__acc\", \"a0\")",
                                                       null,
-                                                      "if(array_length(o) == 0, null, o)",
+                                                      null,
                                                       new HumanReadableBytes(1024),
                                                       TestExprMacroTable.INSTANCE
                                                   )
@@ -1648,10 +1664,11 @@
                                                 "__acc",
                                                 "[]",
                                                 "[]",
+                                                true,
                                                 "array_set_add(\"__acc\", \"dim1\")",
                                                 "array_set_add_all(\"__acc\", \"a0\")",
                                                 null,
-                                                "if(array_length(o) == 0, null, o)",
+                                                null,
                                                 new HumanReadableBytes(1024),
                                                 TestExprMacroTable.INSTANCE
                                             )
@@ -1724,10 +1741,11 @@
                                                       "__acc",
                                                       "[]",
                                                       "[]",
+                                                      true,
                                                       "array_set_add(\"__acc\", \"dim1\")",
                                                       "array_set_add_all(\"__acc\", \"a0\")",
                                                       null,
-                                                      "if(array_length(o) == 0, null, o)",
+                                                      null,
                                                       new HumanReadableBytes(1024),
                                                       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 5014a35..2c4c87e 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
@@ -12650,7 +12650,10 @@
         + " EARLIEST(l1),\n"
         + " LATEST(dim1, 1024),\n"
         + " LATEST(l1),\n"
-        + " ARRAY_AGG(DISTINCT dim3)\n"
+        + " ARRAY_AGG(DISTINCT dim3),\n"
+        + " BIT_AND(l1),\n"
+        + " BIT_OR(l1),\n"
+        + " BIT_XOR(l1)\n"
         + "FROM druid.numfoo WHERE dim2 = 0",
         ImmutableList.of(
             Druids.newTimeseriesQueryBuilder()
@@ -12672,12 +12675,64 @@
                               "__acc",
                               "[]",
                               "[]",
+                              true,
                               "array_set_add(\"__acc\", \"dim3\")",
                               "array_set_add_all(\"__acc\", \"a6\")",
                               null,
-                              "if(array_length(o) == 0, null, o)",
+                              null,
                               new HumanReadableBytes(1024),
                               TestExprMacroTable.INSTANCE
+                          ),
+                          new FilteredAggregatorFactory(
+                            new ExpressionLambdaAggregatorFactory(
+                                "a7",
+                                ImmutableSet.of("l1"),
+                                "__acc",
+                                "0",
+                                "0",
+                                NullHandling.sqlCompatible(),
+                                "bitwiseAnd(\"__acc\", \"l1\")",
+                                "bitwiseAnd(\"__acc\", \"a7\")",
+                                null,
+                                null,
+                                new HumanReadableBytes(1024),
+                                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",
+                                  ImmutableSet.of("l1"),
+                                  "__acc",
+                                  "0",
+                                  "0",
+                                  NullHandling.sqlCompatible(),
+                                  "bitwiseXor(\"__acc\", \"l1\")",
+                                  "bitwiseXor(\"__acc\", \"a9\")",
+                                  null,
+                                  null,
+                                  new HumanReadableBytes(1024),
+                                  TestExprMacroTable.INSTANCE
+                              ),
+                              not(selector("l1", null, null))
                           )
                       )
                   )
@@ -12686,8 +12741,8 @@
         ),
         ImmutableList.of(
             useDefault
-            ? new Object[]{"", 0L, "", 0L, "", 0L, null}
-            : new Object[]{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}
         )
     );
   }
@@ -12828,7 +12883,10 @@
         + " EARLIEST(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
         + " LATEST(dim1, 1024) FILTER(WHERE dim1 = 'nonexistent'),\n"
         + " LATEST(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
-        + " ARRAY_AGG(DISTINCT dim3) FILTER(WHERE dim1 = 'nonexistent')"
+        + " ARRAY_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"
         + "FROM druid.numfoo WHERE dim2 = 'a' GROUP BY dim2",
         ImmutableList.of(
             GroupByQuery.builder()
@@ -12871,14 +12929,66 @@
                                         "__acc",
                                         "[]",
                                         "[]",
+                                        true,
                                         "array_set_add(\"__acc\", \"dim3\")",
                                         "array_set_add_all(\"__acc\", \"a6\")",
                                         null,
-                                        "if(array_length(o) == 0, null, o)",
+                                        null,
                                         new HumanReadableBytes(1024),
                                         TestExprMacroTable.INSTANCE
                                     ),
                                     selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new ExpressionLambdaAggregatorFactory(
+                                        "a7",
+                                        ImmutableSet.of("l1"),
+                                        "__acc",
+                                        "0",
+                                        "0",
+                                        NullHandling.sqlCompatible(),
+                                        "bitwiseAnd(\"__acc\", \"l1\")",
+                                        "bitwiseAnd(\"__acc\", \"a7\")",
+                                        null,
+                                        null,
+                                        new HumanReadableBytes(1024),
+                                        TestExprMacroTable.INSTANCE
+                                    ),
+                                    and(not(selector("l1", null, null)), selector("dim1", "nonexistent", 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
+                                    ),
+                                    and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null))
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new ExpressionLambdaAggregatorFactory(
+                                        "a9",
+                                        ImmutableSet.of("l1"),
+                                        "__acc",
+                                        "0",
+                                        "0",
+                                        NullHandling.sqlCompatible(),
+                                        "bitwiseXor(\"__acc\", \"l1\")",
+                                        "bitwiseXor(\"__acc\", \"a9\")",
+                                        null,
+                                        null,
+                                        new HumanReadableBytes(1024),
+                                        TestExprMacroTable.INSTANCE
+                                    ),
+                                    and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null))
                                 )
                             )
                         )
@@ -12887,8 +12997,8 @@
         ),
         ImmutableList.of(
             useDefault
-            ? new Object[]{"a", "", 0L, "", 0L, "", 0L, null}
-            : new Object[]{"a", 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}
         )
     );
   }
@@ -17611,4 +17721,186 @@
         )
     );
   }
+
+  @Test
+  public void testBitwiseAggregatorsTimeseries() throws Exception
+  {
+    cannotVectorize();
+    testQuery(
+        "SELECT\n"
+        + " BIT_AND(l1),\n"
+        + " BIT_OR(l1),\n"
+        + " BIT_XOR(l1)\n"
+        + "FROM druid.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",
+                                  "0",
+                                  "0",
+                                  NullHandling.sqlCompatible(),
+                                  "bitwiseAnd(\"__acc\", \"l1\")",
+                                  "bitwiseAnd(\"__acc\", \"a0\")",
+                                  null,
+                                  null,
+                                  new HumanReadableBytes(1024),
+                                  TestExprMacroTable.INSTANCE
+                              ),
+                              not(selector("l1", null, null))
+                          ),
+                          new FilteredAggregatorFactory(
+                              new ExpressionLambdaAggregatorFactory(
+                                  "a1",
+                                  ImmutableSet.of("l1"),
+                                  "__acc",
+                                  "0",
+                                  "0",
+                                  NullHandling.sqlCompatible(),
+                                  "bitwiseOr(\"__acc\", \"l1\")",
+                                  "bitwiseOr(\"__acc\", \"a1\")",
+                                  null,
+                                  null,
+                                  new HumanReadableBytes(1024),
+                                  TestExprMacroTable.INSTANCE
+                              ),
+                              not(selector("l1", null, null))
+                          ),
+                          new FilteredAggregatorFactory(
+                              new ExpressionLambdaAggregatorFactory(
+                                  "a2",
+                                  ImmutableSet.of("l1"),
+                                  "__acc",
+                                  "0",
+                                  "0",
+                                  NullHandling.sqlCompatible(),
+                                  "bitwiseXor(\"__acc\", \"l1\")",
+                                  "bitwiseXor(\"__acc\", \"a2\")",
+                                  null,
+                                  null,
+                                  new HumanReadableBytes(1024),
+                                  TestExprMacroTable.INSTANCE
+                              ),
+                              not(selector("l1", null, null))
+                          )
+                      )
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            useDefault
+            ? new Object[]{0L, 325327L, 325324L}
+            : new Object[]{0L, 325327L, 325324L}
+        )
+    );
+  }
+
+  @Test
+  public void testBitwiseAggregatorsGroupBy() throws Exception
+  {
+    cannotVectorize();
+    testQuery(
+        "SELECT\n"
+        + " dim2,\n"
+        + " BIT_AND(l1),\n"
+        + " BIT_OR(l1),\n"
+        + " BIT_XOR(l1)\n"
+        + "FROM druid.numfoo GROUP BY 1 ORDER BY 4",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE3)
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setDimensions(new DefaultDimensionSpec("dim2", "_d0", ValueType.STRING))
+                        .setAggregatorSpecs(
+                            aggregators(
+                                new FilteredAggregatorFactory(
+                                    new ExpressionLambdaAggregatorFactory(
+                                        "a0",
+                                        ImmutableSet.of("l1"),
+                                        "__acc",
+                                        "0",
+                                        "0",
+                                        NullHandling.sqlCompatible(),
+                                        "bitwiseAnd(\"__acc\", \"l1\")",
+                                        "bitwiseAnd(\"__acc\", \"a0\")",
+                                        null,
+                                        null,
+                                        new HumanReadableBytes(1024),
+                                        TestExprMacroTable.INSTANCE
+                                    ),
+                                    not(selector("l1", null, null))
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new ExpressionLambdaAggregatorFactory(
+                                        "a1",
+                                        ImmutableSet.of("l1"),
+                                        "__acc",
+                                        "0",
+                                        "0",
+                                        NullHandling.sqlCompatible(),
+                                        "bitwiseOr(\"__acc\", \"l1\")",
+                                        "bitwiseOr(\"__acc\", \"a1\")",
+                                        null,
+                                        null,
+                                        new HumanReadableBytes(1024),
+                                        TestExprMacroTable.INSTANCE
+                                    ),
+                                    not(selector("l1", null, null))
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new ExpressionLambdaAggregatorFactory(
+                                        "a2",
+                                        ImmutableSet.of("l1"),
+                                        "__acc",
+                                        "0",
+                                        "0",
+                                        NullHandling.sqlCompatible(),
+                                        "bitwiseXor(\"__acc\", \"l1\")",
+                                        "bitwiseXor(\"__acc\", \"a2\")",
+                                        null,
+                                        null,
+                                        new HumanReadableBytes(1024),
+                                        TestExprMacroTable.INSTANCE
+                                    ),
+                                    not(selector("l1", null, null))
+                                )
+                            )
+                        )
+                        .setLimitSpec(
+                            DefaultLimitSpec.builder()
+                                            .orderBy(
+                                                new OrderByColumnSpec(
+                                                    "a2",
+                                                    Direction.ASCENDING,
+                                                    StringComparators.NUMERIC
+                                                )
+                                            )
+                                            .build()
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        useDefault
+        ? ImmutableList.of(
+            new Object[]{"abc", 0L, 0L, 0L},
+            new Object[]{"a", 0L, 7L, 7L},
+            new Object[]{"", 0L, 325323L, 325323L}
+        )
+        : ImmutableList.of(
+            new Object[]{"abc", null, null, null},
+            new Object[]{"", 0L, 0L, 0L},
+            new Object[]{"a", 0L, 7L, 7L},
+            new Object[]{null, 0L, 325323L, 325323L}
+        )
+    );
+  }
 }