SQL timeseries no longer skip empty buckets with all granularity (#11188)

* SQL timeseries no longer skip empty buckets with all granularity

* add comment, fix tests

* the ol switcheroo

* revert unintended change

* docs and more tests

* style

* make checkstyle happy

* docs fixes and more tests

* add docs, tests for array_agg

* fixes

* oops

* doc stuffs

* fix compile, match doc style
diff --git a/docs/querying/sql.md b/docs/querying/sql.md
index b4bf211..1fbed13 100644
--- a/docs/querying/sql.md
+++ b/docs/querying/sql.md
@@ -293,7 +293,7 @@
 The `druid.generic.useDefaultValueForNull` [runtime property](../configuration/index.md#sql-compatible-null-handling)
 controls Druid's NULL handling mode.
 
-In the default mode (`true`), Druid treats NULLs and empty strings interchangeably, rather than according to the SQL
+In the default value mode (`true`), Druid treats NULLs and empty strings interchangeably, rather than according to the SQL
 standard. In this mode Druid SQL only has partial support for NULLs. For example, the expressions `col IS NULL` and
 `col = ''` are equivalent, and both will evaluate to true if `col` contains an empty string. Similarly, the expression
 `COALESCE(col1, col2)` will return `col2` if `col1` is an empty string. While the `COUNT(*)` aggregator counts all rows,
@@ -301,17 +301,22 @@
 columns in this mode are not nullable; any null or missing values will be treated as zeroes.
 
 In SQL compatible mode (`false`), NULLs are treated more closely to the SQL standard. The property affects both storage
-and querying, so for best behavior, it should be set at both ingestion time and query time. There is some overhead
-associated with the ability to handle NULLs; see the [segment internals](../design/segments.md#sql-compatible-null-handling)
-documentation for more details.
+and querying, so for correct behavior, it should be set on all Druid service types to be available at both ingestion
+time and query time. There is some overhead associated with the ability to handle NULLs; see
+the [segment internals](../design/segments.md#sql-compatible-null-handling)documentation for more details.
 
 ## Aggregation functions
 
-Aggregation functions can appear in the SELECT clause of any query. Any aggregator can be filtered using syntax like
-`AGG(expr) FILTER(WHERE whereExpr)`. Filtered aggregators will only aggregate rows that match their filter. It's
+Aggregation functions can appear in the SELECT clause of any query. Any aggregator can be filtered using syntax
+like `AGG(expr) FILTER(WHERE whereExpr)`. Filtered aggregators will only aggregate rows that match their filter. It's
 possible for two aggregators in the same SQL query to have different filters.
 
-Only the COUNT and ARRAY_AGG aggregations can accept DISTINCT.
+When no rows are selected, aggregate functions will return their initial value. This can occur when filtering results in
+no matches while aggregating values across an entire table without a grouping, or, when using filtered aggregations
+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.
 
 > 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. 
@@ -320,41 +325,41 @@
 > results across multiple query runs because of this. If precisely the same value is desired across multiple query runs,
 > consider using the `ROUND` function to smooth out the inconsistencies between queries.  
 
-|Function|Notes|
-|--------|-----|
-|`COUNT(*)`|Counts the number of rows.|
-|`COUNT(DISTINCT expr)`|Counts distinct values of expr, which can be string, numeric, or hyperUnique. By default this is approximate, using a variant of [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf). To get exact counts set "useApproximateCountDistinct" to "false". If you do this, expr must be string or numeric, since exact counts are not possible using hyperUnique columns. See also `APPROX_COUNT_DISTINCT(expr)`. In exact mode, only one distinct count per query is permitted unless `useGroupingSetForExactDistinct` is set to true in query contexts or broker configurations.|
-|`SUM(expr)`|Sums numbers.|
-|`MIN(expr)`|Takes the minimum of numbers.|
-|`MAX(expr)`|Takes the maximum of numbers.|
-|`AVG(expr)`|Averages numbers.|
-|`APPROX_COUNT_DISTINCT(expr)`|Counts distinct values of expr, which can be a regular column or a hyperUnique column. This is always approximate, regardless of the value of "useApproximateCountDistinct". This uses Druid's built-in "cardinality" or "hyperUnique" aggregators. See also `COUNT(DISTINCT expr)`.|
-|`APPROX_COUNT_DISTINCT_DS_HLL(expr, [lgK, tgtHllType])`|Counts distinct values of expr, which can be a regular column or an [HLL sketch](../development/extensions-core/datasketches-hll.md) column. The `lgK` and `tgtHllType` parameters are described in the HLL sketch documentation. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`. The [DataSketches extension](../development/extensions-core/datasketches-extension.md) must be loaded to use this function.|
-|`APPROX_COUNT_DISTINCT_DS_THETA(expr, [size])`|Counts distinct values of expr, which can be a regular column or a [Theta sketch](../development/extensions-core/datasketches-theta.md) column. The `size` parameter is described in the Theta sketch documentation. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`. The [DataSketches extension](../development/extensions-core/datasketches-extension.md) must be loaded to use this function.|
-|`DS_HLL(expr, [lgK, tgtHllType])`|Creates an [HLL sketch](../development/extensions-core/datasketches-hll.md) on the values of expr, which can be a regular column or a column containing HLL sketches. The `lgK` and `tgtHllType` parameters are described in the HLL sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extension.md) must be loaded to use this function.|
-|`DS_THETA(expr, [size])`|Creates a [Theta sketch](../development/extensions-core/datasketches-theta.md) on the values of expr, which can be a regular column or a column containing Theta sketches. The `size` parameter is described in the Theta sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extension.md) must be loaded to use this function.|
-|`APPROX_QUANTILE(expr, probability, [resolution])`|Computes approximate quantiles on numeric or [approxHistogram](../development/extensions-core/approximate-histograms.md#approximate-histogram-aggregator) exprs. The "probability" should be between 0 and 1 (exclusive). The "resolution" is the number of centroids to use for the computation. Higher resolutions will give more precise results but also have higher overhead. If not provided, the default resolution is 50. The [approximate histogram extension](../development/extensions-core/approximate-histograms.md) must be loaded to use this function.|
-|`APPROX_QUANTILE_DS(expr, probability, [k])`|Computes approximate quantiles on numeric or [Quantiles sketch](../development/extensions-core/datasketches-quantiles.md) exprs. The "probability" should be between 0 and 1 (exclusive). The `k` parameter is described in the Quantiles sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extension.md) must be loaded to use this function.|
-|`APPROX_QUANTILE_FIXED_BUCKETS(expr, probability, numBuckets, lowerLimit, upperLimit, [outlierHandlingMode])`|Computes approximate quantiles on numeric or [fixed buckets histogram](../development/extensions-core/approximate-histograms.md#fixed-buckets-histogram) exprs. The "probability" should be between 0 and 1 (exclusive). The `numBuckets`, `lowerLimit`, `upperLimit`, and `outlierHandlingMode` parameters are described in the fixed buckets histogram documentation. The [approximate histogram extension](../development/extensions-core/approximate-histograms.md) must be loaded to use this function.|
-|`DS_QUANTILES_SKETCH(expr, [k])`|Creates a [Quantiles sketch](../development/extensions-core/datasketches-quantiles.md) on the values of expr, which can be a regular column or a column containing quantiles sketches. The `k` parameter is described in the Quantiles sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extension.md) must be loaded to use this function.|
-|`BLOOM_FILTER(expr, numEntries)`|Computes a bloom filter from values produced by `expr`, with `numEntries` maximum number of distinct values before false positive rate increases. See [bloom filter extension](../development/extensions-core/bloom-filter.md) documentation for additional details.|
-|`TDIGEST_QUANTILE(expr, quantileFraction, [compression])`|Builds a T-Digest sketch on values produced by `expr` and returns the value for the quantile. Compression parameter (default value 100) determines the accuracy and size of the sketch. Higher compression means higher accuracy but more space to store sketches. See [t-digest extension](../development/extensions-contrib/tdigestsketch-quantiles.md) documentation for additional details.|
-|`TDIGEST_GENERATE_SKETCH(expr, [compression])`|Builds a T-Digest sketch on values produced by `expr`. Compression parameter (default value 100) determines the accuracy and size of the sketch Higher compression means higher accuracy but more space to store sketches. See [t-digest extension](../development/extensions-contrib/tdigestsketch-quantiles.md) documentation for additional details.|
-|`VAR_POP(expr)`|Computes variance population of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|
-|`VAR_SAMP(expr)`|Computes variance sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|
-|`VARIANCE(expr)`|Computes variance sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|
-|`STDDEV_POP(expr)`|Computes standard deviation population of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|
-|`STDDEV_SAMP(expr)`|Computes standard deviation sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|
-|`STDDEV(expr)`|Computes standard deviation sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|
-|`EARLIEST(expr)`|Returns the earliest value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "earliest" is the value first encountered with the minimum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the first value encountered.|
-|`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(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.|
-|`LATEST(expr)`|Returns the latest value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|
-|`LATEST(expr, maxBytesPerString)`|Like `LATEST(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.|
-|`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)|
-|`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.|
-|`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.|
-|`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.|
-|`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.|
+|Function|Notes|Default|
+|--------|-----|-------|
+|`COUNT(*)`|Counts the number of rows.|`0`|
+|`COUNT(DISTINCT expr)`|Counts distinct values of expr, which can be string, numeric, or hyperUnique. By default this is approximate, using a variant of [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf). To get exact counts set "useApproximateCountDistinct" to "false". If you do this, expr must be string or numeric, since exact counts are not possible using hyperUnique columns. See also `APPROX_COUNT_DISTINCT(expr)`. In exact mode, only one distinct count per query is permitted unless `useGroupingSetForExactDistinct` is set to true in query contexts or broker configurations.|`0`|
+|`SUM(expr)`|Sums numbers.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
+|`MIN(expr)`|Takes the minimum of numbers.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `9223372036854775807` (maximum LONG value)|
+|`MAX(expr)`|Takes the maximum of numbers.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `-9223372036854775808` (minimum LONG value)|
+|`AVG(expr)`|Averages numbers.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
+|`APPROX_COUNT_DISTINCT(expr)`|Counts distinct values of expr, which can be a regular column or a hyperUnique column. This is always approximate, regardless of the value of "useApproximateCountDistinct". This uses Druid's built-in "cardinality" or "hyperUnique" aggregators. See also `COUNT(DISTINCT expr)`.|`0`|
+|`APPROX_COUNT_DISTINCT_DS_HLL(expr, [lgK, tgtHllType])`|Counts distinct values of expr, which can be a regular column or an [HLL sketch](../development/extensions-core/datasketches-hll.md) column. The `lgK` and `tgtHllType` parameters are described in the HLL sketch documentation. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`. The [DataSketches extension](../development/extensions-core/datasketches-extension.md) must be loaded to use this function.|`0`|
+|`APPROX_COUNT_DISTINCT_DS_THETA(expr, [size])`|Counts distinct values of expr, which can be a regular column or a [Theta sketch](../development/extensions-core/datasketches-theta.md) column. The `size` parameter is described in the Theta sketch documentation. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`. The [DataSketches extension](../development/extensions-core/datasketches-extension.md) must be loaded to use this function.|`0`|
+|`DS_HLL(expr, [lgK, tgtHllType])`|Creates an [HLL sketch](../development/extensions-core/datasketches-hll.md) on the values of expr, which can be a regular column or a column containing HLL sketches. The `lgK` and `tgtHllType` parameters are described in the HLL sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extension.md) must be loaded to use this function.|`'0'` (STRING)|
+|`DS_THETA(expr, [size])`|Creates a [Theta sketch](../development/extensions-core/datasketches-theta.md) on the values of expr, which can be a regular column or a column containing Theta sketches. The `size` parameter is described in the Theta sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extension.md) must be loaded to use this function.|`'0.0'` (STRING)|
+|`APPROX_QUANTILE(expr, probability, [resolution])`|Computes approximate quantiles on numeric or [approxHistogram](../development/extensions-core/approximate-histograms.md#approximate-histogram-aggregator) exprs. The "probability" should be between 0 and 1 (exclusive). The "resolution" is the number of centroids to use for the computation. Higher resolutions will give more precise results but also have higher overhead. If not provided, the default resolution is 50. The [approximate histogram extension](../development/extensions-core/approximate-histograms.md) must be loaded to use this function.|`NaN`|
+|`APPROX_QUANTILE_DS(expr, probability, [k])`|Computes approximate quantiles on numeric or [Quantiles sketch](../development/extensions-core/datasketches-quantiles.md) exprs. The "probability" should be between 0 and 1 (exclusive). The `k` parameter is described in the Quantiles sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extension.md) must be loaded to use this function.|`NaN`|
+|`APPROX_QUANTILE_FIXED_BUCKETS(expr, probability, numBuckets, lowerLimit, upperLimit, [outlierHandlingMode])`|Computes approximate quantiles on numeric or [fixed buckets histogram](../development/extensions-core/approximate-histograms.md#fixed-buckets-histogram) exprs. The "probability" should be between 0 and 1 (exclusive). The `numBuckets`, `lowerLimit`, `upperLimit`, and `outlierHandlingMode` parameters are described in the fixed buckets histogram documentation. The [approximate histogram extension](../development/extensions-core/approximate-histograms.md) must be loaded to use this function.|`0.0`|
+|`DS_QUANTILES_SKETCH(expr, [k])`|Creates a [Quantiles sketch](../development/extensions-core/datasketches-quantiles.md) on the values of expr, which can be a regular column or a column containing quantiles sketches. The `k` parameter is described in the Quantiles sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extension.md) must be loaded to use this function.|`'0'` (STRING)|
+|`BLOOM_FILTER(expr, numEntries)`|Computes a bloom filter from values produced by `expr`, with `numEntries` maximum number of distinct values before false positive rate increases. See [bloom filter extension](../development/extensions-core/bloom-filter.md) documentation for additional details.|Empty base64 encoded bloom filter STRING|
+|`TDIGEST_QUANTILE(expr, quantileFraction, [compression])`|Builds a T-Digest sketch on values produced by `expr` and returns the value for the quantile. Compression parameter (default value 100) determines the accuracy and size of the sketch. Higher compression means higher accuracy but more space to store sketches. See [t-digest extension](../development/extensions-contrib/tdigestsketch-quantiles.md) documentation for additional details.|`Double.NaN`|
+|`TDIGEST_GENERATE_SKETCH(expr, [compression])`|Builds a T-Digest sketch on values produced by `expr`. Compression parameter (default value 100) determines the accuracy and size of the sketch Higher compression means higher accuracy but more space to store sketches. See [t-digest extension](../development/extensions-contrib/tdigestsketch-quantiles.md) documentation for additional details.|Empty base64 encoded T-Digest sketch STRING|
+|`VAR_POP(expr)`|Computes variance population of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
+|`VAR_SAMP(expr)`|Computes variance sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
+|`VARIANCE(expr)`|Computes variance sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
+|`STDDEV_POP(expr)`|Computes standard deviation population of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
+|`STDDEV_SAMP(expr)`|Computes standard deviation sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
+|`STDDEV(expr)`|Computes standard deviation sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
+|`EARLIEST(expr)`|Returns the earliest value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "earliest" is the value first encountered with the minimum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the first value encountered.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
+|`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(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 `''`|
+|`LATEST(expr)`|Returns the latest value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
+|`LATEST(expr, maxBytesPerString)`|Like `LATEST(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 `''`|
+|`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`|
 
 For advice on choosing approximate aggregation functions, check out our [approximate aggregations documentation](aggregations.md#approx).
 
diff --git a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java
index ec17e7e..c1737f9 100644
--- a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java
+++ b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java
@@ -21,9 +21,7 @@
 
 import com.fasterxml.jackson.databind.Module;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.math.expr.ExprMacroTable;
@@ -31,6 +29,7 @@
 import org.apache.druid.query.QueryDataSource;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
+import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
 import org.apache.druid.query.aggregation.PostAggregator;
 import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
 import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchAggregatorFactory;
@@ -38,6 +37,7 @@
 import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchToQuantilePostAggregator;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
 import org.apache.druid.segment.IndexBuilder;
 import org.apache.druid.segment.QueryableIndex;
@@ -45,16 +45,11 @@
 import org.apache.druid.segment.incremental.IncrementalIndexSchema;
 import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
 import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
-import org.apache.druid.server.security.AuthenticationResult;
-import org.apache.druid.sql.SqlLifecycle;
 import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
 import org.apache.druid.sql.calcite.filtration.Filtration;
 import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
-import org.apache.druid.sql.calcite.planner.PlannerConfig;
-import org.apache.druid.sql.calcite.planner.PlannerContext;
 import org.apache.druid.sql.calcite.util.CalciteTests;
 import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
-import org.apache.druid.sql.http.SqlParameter;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.partition.LinearShardSpec;
 import org.junit.Assert;
@@ -62,12 +57,9 @@
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
-import java.util.stream.Stream;
 
 public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
 {
-  private static final AuthenticationResult AUTH_RESULT = CalciteTests.REGULAR_USER_AUTH_RESULT;
   private static final DruidOperatorTable OPERATOR_TABLE = new DruidOperatorTable(
       ImmutableSet.of(new TDigestSketchQuantileSqlAggregator(), new TDigestGenerateSketchSqlAggregator()),
       ImmutableSet.of()
@@ -115,73 +107,36 @@
   }
 
   @Override
-  public List<Object[]> getResults(
-      final PlannerConfig plannerConfig,
-      final Map<String, Object> queryContext,
-      final List<SqlParameter> parameters,
-      final String sql,
-      final AuthenticationResult authenticationResult
-  ) throws Exception
+  public DruidOperatorTable createOperatorTable()
   {
-    return getResults(
-        plannerConfig,
-        queryContext,
-        parameters,
-        sql,
-        authenticationResult,
-        OPERATOR_TABLE,
-        CalciteTests.createExprMacroTable(),
-        CalciteTests.TEST_AUTHORIZER_MAPPER,
-        CalciteTests.getJsonMapper()
-    );
-  }
-
-  private SqlLifecycle getSqlLifecycle()
-  {
-    return getSqlLifecycleFactory(
-        BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT,
-        OPERATOR_TABLE,
-        CalciteTests.createExprMacroTable(),
-        CalciteTests.TEST_AUTHORIZER_MAPPER,
-        CalciteTests.getJsonMapper()
-    ).factorize();
+    return OPERATOR_TABLE;
   }
 
   @Test
   public void testComputingSketchOnNumericValues() throws Exception
   {
-    SqlLifecycle sqlLifecycle = getSqlLifecycle();
-    final String sql = "SELECT\n"
-                       + "TDIGEST_GENERATE_SKETCH(m1, 200)"
-                       + "FROM foo";
+    cannotVectorize();
 
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        TIMESERIES_CONTEXT_DEFAULT,
-        DEFAULT_PARAMETERS,
-        AUTH_RESULT
-    ).toList();
-    final List<String[]> expectedResults = ImmutableList.of(
-        new String[]{
-            "\"AAAAAT/wAAAAAAAAQBgAAAAAAABAaQAAAAAAAAAAAAY/8AAAAAAAAD/wAAAAAAAAP/AAAAAAAABAAAAAAAAAAD/wAAAAAAAAQAgAAAAAAAA/8AAAAAAAAEAQAAAAAAAAP/AAAAAAAABAFAAAAAAAAD/wAAAAAAAAQBgAAAAAAAA=\""
-        }
-    );
-
-    Assert.assertEquals(expectedResults.size(), results.size());
-
-    // Verify query
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(ImmutableList.of(
-                  new TDigestSketchAggregatorFactory("a0:agg", "m1", 200)
-              ))
-              .context(TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+    testQuery(
+        "SELECT\n"
+        + "TDIGEST_GENERATE_SKETCH(m1, 200)"
+        + "FROM foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(ImmutableList.of(
+                      new TDigestSketchAggregatorFactory("a0:agg", "m1", 200)
+                  ))
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new String[]{
+                "\"AAAAAT/wAAAAAAAAQBgAAAAAAABAaQAAAAAAAAAAAAY/8AAAAAAAAD/wAAAAAAAAP/AAAAAAAABAAAAAAAAAAD/wAAAAAAAAQAgAAAAAAAA/8AAAAAAAAEAQAAAAAAAAP/AAAAAAAABAFAAAAAAAAD/wAAAAAAAAQBgAAAAAAAA=\""
+            }
+        )
     );
   }
 
@@ -210,7 +165,7 @@
                   .aggregators(ImmutableList.of(
                       new TDigestSketchAggregatorFactory("a0:agg", "v0", 200)
                   ))
-                  .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -228,54 +183,36 @@
   @Test
   public void testDefaultCompressionForTDigestGenerateSketchAgg() throws Exception
   {
-    SqlLifecycle sqlLifecycle = getSqlLifecycle();
-    final String sql = "SELECT\n"
-                       + "TDIGEST_GENERATE_SKETCH(m1)"
-                       + "FROM foo";
+    cannotVectorize();
 
-    // Log query
-    sqlLifecycle.runSimple(
-        sql,
-        TIMESERIES_CONTEXT_DEFAULT,
-        DEFAULT_PARAMETERS,
-        AUTH_RESULT
-    ).toList();
-
-    // Verify query
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(ImmutableList.of(
-                  new TDigestSketchAggregatorFactory("a0:agg", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION)
-              ))
-              .context(TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+    testQuery(
+        "SELECT\n"
+        + "TDIGEST_GENERATE_SKETCH(m1)"
+        + "FROM foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(ImmutableList.of(
+                      new TDigestSketchAggregatorFactory("a0:agg", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION)
+                  ))
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{"\"AAAAAT/wAAAAAAAAQBgAAAAAAABAWQAAAAAAAAAAAAY/8AAAAAAAAD/wAAAAAAAAP/AAAAAAAABAAAAAAAAAAD/wAAAAAAAAQAgAAAAAAAA/8AAAAAAAAEAQAAAAAAAAP/AAAAAAAABAFAAAAAAAAD/wAAAAAAAAQBgAAAAAAAA=\""}
+        )
     );
   }
 
   @Test
   public void testComputingQuantileOnPreAggregatedSketch() throws Exception
   {
-    SqlLifecycle sqlLifecycle = getSqlLifecycle();
-    final String sql = "SELECT\n"
-                       + "TDIGEST_QUANTILE(qsketch_m1, 0.1),\n"
-                       + "TDIGEST_QUANTILE(qsketch_m1, 0.4),\n"
-                       + "TDIGEST_QUANTILE(qsketch_m1, 0.8),\n"
-                       + "TDIGEST_QUANTILE(qsketch_m1, 1.0)\n"
-                       + "FROM foo";
+    cannotVectorize();
 
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        TIMESERIES_CONTEXT_DEFAULT,
-        DEFAULT_PARAMETERS,
-        AUTH_RESULT
-    ).toList();
-    final List<double[]> expectedResults = ImmutableList.of(
-        new double[]{
+    final List<Object[]> expectedResults = ImmutableList.of(
+        new Object[]{
             1.1,
             2.9,
             5.3,
@@ -283,200 +220,161 @@
         }
     );
 
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Object[] objects = results.get(i);
-      Assert.assertArrayEquals(
-          expectedResults.get(i),
-          Stream.of(objects).mapToDouble(value -> ((Double) value).doubleValue()).toArray(),
-          0.000001
-      );
-    }
-
-    // Verify query
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(ImmutableList.of(
-                  new TDigestSketchAggregatorFactory("a0:agg", "qsketch_m1", 100)
-              ))
-              .postAggregators(
-                  new TDigestSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.1f),
-                  new TDigestSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a0:agg"), 0.4f),
-                  new TDigestSketchToQuantilePostAggregator("a2", makeFieldAccessPostAgg("a0:agg"), 0.8f),
-                  new TDigestSketchToQuantilePostAggregator("a3", makeFieldAccessPostAgg("a0:agg"), 1.0f)
-              )
-              .context(TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+    testQuery(
+        "SELECT\n"
+        + "TDIGEST_QUANTILE(qsketch_m1, 0.1),\n"
+        + "TDIGEST_QUANTILE(qsketch_m1, 0.4),\n"
+        + "TDIGEST_QUANTILE(qsketch_m1, 0.8),\n"
+        + "TDIGEST_QUANTILE(qsketch_m1, 1.0)\n"
+        + "FROM foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(ImmutableList.of(
+                      new TDigestSketchAggregatorFactory("a0:agg", "qsketch_m1", 100)
+                  ))
+                  .postAggregators(
+                      new TDigestSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.1f),
+                      new TDigestSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a0:agg"), 0.4f),
+                      new TDigestSketchToQuantilePostAggregator("a2", makeFieldAccessPostAgg("a0:agg"), 0.8f),
+                      new TDigestSketchToQuantilePostAggregator("a3", makeFieldAccessPostAgg("a0:agg"), 1.0f)
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        expectedResults
     );
   }
 
   @Test
   public void testGeneratingSketchAndComputingQuantileOnFly() throws Exception
   {
-    SqlLifecycle sqlLifecycle = getSqlLifecycle();
-    final String sql = "SELECT TDIGEST_QUANTILE(x, 0.0), TDIGEST_QUANTILE(x, 0.5), TDIGEST_QUANTILE(x, 1.0)\n"
-                       + "FROM (SELECT dim1, TDIGEST_GENERATE_SKETCH(m1, 200) AS x FROM foo group by dim1)";
+    cannotVectorize();
 
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        TIMESERIES_CONTEXT_DEFAULT,
-        DEFAULT_PARAMETERS,
-        AUTH_RESULT
-    ).toList();
-    final List<double[]> expectedResults = ImmutableList.of(
-        new double[]{
+    final List<Object[]> expectedResults = ImmutableList.of(
+        new Object[]{
             1.0,
             3.5,
             6.0
         }
     );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Object[] objects = results.get(i);
-      Assert.assertArrayEquals(
-          expectedResults.get(i),
-          Stream.of(objects).mapToDouble(value -> ((Double) value).doubleValue()).toArray(),
-          0.000001
-      );
-    }
 
-    // Verify query
-    Assert.assertEquals(
-        GroupByQuery.builder()
-                    .setDataSource(
-                        new QueryDataSource(
-                            GroupByQuery.builder()
-                                        .setDataSource(CalciteTests.DATASOURCE1)
-                                        .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-                                        .setGranularity(Granularities.ALL)
-                                        .setDimensions(new DefaultDimensionSpec("dim1", "d0"))
-                                        .setAggregatorSpecs(
-                                            ImmutableList.of(
-                                                new TDigestSketchAggregatorFactory("a0:agg", "m1", 200)
+    testQuery(
+        "SELECT TDIGEST_QUANTILE(x, 0.0), TDIGEST_QUANTILE(x, 0.5), TDIGEST_QUANTILE(x, 1.0)\n"
+        + "FROM (SELECT dim1, TDIGEST_GENERATE_SKETCH(m1, 200) AS x FROM foo group by dim1)",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                GroupByQuery.builder()
+                                            .setDataSource(CalciteTests.DATASOURCE1)
+                                            .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                                            .setGranularity(Granularities.ALL)
+                                            .setDimensions(new DefaultDimensionSpec("dim1", "d0"))
+                                            .setAggregatorSpecs(
+                                                ImmutableList.of(
+                                                    new TDigestSketchAggregatorFactory("a0:agg", "m1", 200)
+                                                )
                                             )
-                                        )
-                                        .setContext(TIMESERIES_CONTEXT_DEFAULT)
-                                        .build()
+                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                            .build()
+                            )
                         )
-                    )
-                    .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-                    .setGranularity(Granularities.ALL)
-                    .setAggregatorSpecs(
-                        ImmutableList.of(
-                            new TDigestSketchAggregatorFactory("_a0:agg", "a0:agg", 100)
+                        .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(
+                            ImmutableList.of(
+                                new TDigestSketchAggregatorFactory("_a0:agg", "a0:agg", 100)
+                            )
                         )
-                    )
-                    .setPostAggregatorSpecs(
-                        ImmutableList.of(
-                            new TDigestSketchToQuantilePostAggregator("_a0", makeFieldAccessPostAgg("_a0:agg"), 0.0f),
-                            new TDigestSketchToQuantilePostAggregator("_a1", makeFieldAccessPostAgg("_a0:agg"), 0.5f),
-                            new TDigestSketchToQuantilePostAggregator("_a2", makeFieldAccessPostAgg("_a0:agg"), 1.0f)
+                        .setPostAggregatorSpecs(
+                            ImmutableList.of(
+                                new TDigestSketchToQuantilePostAggregator("_a0", makeFieldAccessPostAgg("_a0:agg"), 0.0f),
+                                new TDigestSketchToQuantilePostAggregator("_a1", makeFieldAccessPostAgg("_a0:agg"), 0.5f),
+                                new TDigestSketchToQuantilePostAggregator("_a2", makeFieldAccessPostAgg("_a0:agg"), 1.0f)
+                            )
                         )
-                    )
-                    .setContext(TIMESERIES_CONTEXT_DEFAULT)
-                    .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        expectedResults
     );
   }
 
   @Test
   public void testQuantileOnNumericValues() throws Exception
   {
-    SqlLifecycle sqlLifecycle = getSqlLifecycle();
-    final String sql = "SELECT\n"
-                       + "TDIGEST_QUANTILE(m1, 0.0), TDIGEST_QUANTILE(m1, 0.5), TDIGEST_QUANTILE(m1, 1.0)\n"
-                       + "FROM foo";
+    cannotVectorize();
 
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        TIMESERIES_CONTEXT_DEFAULT,
-        DEFAULT_PARAMETERS,
-        AUTH_RESULT
-    ).toList();
-    final List<double[]> expectedResults = ImmutableList.of(
-        new double[]{
+    final List<Object[]> expectedResults = ImmutableList.of(
+        new Object[]{
             1.0,
             3.5,
             6.0
         }
     );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Object[] objects = results.get(i);
-      Assert.assertArrayEquals(
-          expectedResults.get(i),
-          Stream.of(objects).mapToDouble(value -> ((Double) value).doubleValue()).toArray(),
-          0.000001
-      );
-    }
 
-    // Verify query
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(ImmutableList.of(
-                  new TDigestSketchAggregatorFactory("a0:agg", "m1", null)
-              ))
-              .postAggregators(
-                  new TDigestSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.0f),
-                  new TDigestSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a0:agg"), 0.5f),
-                  new TDigestSketchToQuantilePostAggregator("a2", makeFieldAccessPostAgg("a0:agg"), 1.0f)
-              )
-              .context(TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+    testQuery(
+        "SELECT\n"
+        + "TDIGEST_QUANTILE(m1, 0.0), TDIGEST_QUANTILE(m1, 0.5), TDIGEST_QUANTILE(m1, 1.0)\n"
+        + "FROM foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(ImmutableList.of(
+                      new TDigestSketchAggregatorFactory("a0:agg", "m1", null)
+                  ))
+                  .postAggregators(
+                      new TDigestSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.0f),
+                      new TDigestSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a0:agg"), 0.5f),
+                      new TDigestSketchToQuantilePostAggregator("a2", makeFieldAccessPostAgg("a0:agg"), 1.0f)
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        expectedResults
     );
   }
 
   @Test
   public void testCompressionParamForTDigestQuantileAgg() throws Exception
   {
-    SqlLifecycle sqlLifecycle = getSqlLifecycle();
-    final String sql = "SELECT\n"
-                       + "TDIGEST_QUANTILE(m1, 0.0), TDIGEST_QUANTILE(m1, 0.5, 200), TDIGEST_QUANTILE(m1, 1.0, 300)\n"
-                       + "FROM foo";
-
-    // Log query
-    sqlLifecycle.runSimple(
-        sql,
-        TIMESERIES_CONTEXT_DEFAULT,
-        DEFAULT_PARAMETERS,
-        AUTH_RESULT
-    ).toList();
-
-    // Verify query
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(ImmutableList.of(
-                  new TDigestSketchAggregatorFactory("a0:agg", "m1",
-                                                     TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION
-                  ),
-                  new TDigestSketchAggregatorFactory("a1:agg", "m1",
-                                                     200
-                  ),
-                  new TDigestSketchAggregatorFactory("a2:agg", "m1",
-                                                     300
+    cannotVectorize();
+    testQuery(
+        "SELECT\n"
+        + "TDIGEST_QUANTILE(m1, 0.0), TDIGEST_QUANTILE(m1, 0.5, 200), TDIGEST_QUANTILE(m1, 1.0, 300)\n"
+        + "FROM foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(ImmutableList.of(
+                      new TDigestSketchAggregatorFactory("a0:agg", "m1",
+                                                         TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION
+                      ),
+                      new TDigestSketchAggregatorFactory("a1:agg", "m1",
+                                                         200
+                      ),
+                      new TDigestSketchAggregatorFactory("a2:agg", "m1",
+                                                         300
+                      )
+                  ))
+                  .postAggregators(
+                      new TDigestSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.0f),
+                      new TDigestSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a1:agg"), 0.5f),
+                      new TDigestSketchToQuantilePostAggregator("a2", makeFieldAccessPostAgg("a2:agg"), 1.0f)
                   )
-              ))
-              .postAggregators(
-                  new TDigestSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.0f),
-                  new TDigestSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a1:agg"), 0.5f),
-                  new TDigestSketchToQuantilePostAggregator("a2", makeFieldAccessPostAgg("a2:agg"), 1.0f)
-              )
-              .context(TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{1.0, 3.5, 6.0}
+        )
     );
   }
 
@@ -522,7 +420,7 @@
                           1.0
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -533,6 +431,102 @@
     );
   }
 
+  @Test
+  public void testEmptyTimeseriesResults() throws Exception
+  {
+    cannotVectorize();
+
+    testQuery(
+        "SELECT\n"
+        + "TDIGEST_GENERATE_SKETCH(m1),"
+        + "TDIGEST_QUANTILE(qsketch_m1, 0.1)"
+        + "FROM foo WHERE dim2 = 0",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
+                  .granularity(Granularities.ALL)
+                  .aggregators(ImmutableList.of(
+                      new TDigestSketchAggregatorFactory("a0:agg", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION),
+                      new TDigestSketchAggregatorFactory("a1:agg", "qsketch_m1", 100)
+                  ))
+                  .postAggregators(
+                      new TDigestSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a1:agg"), 0.1f)
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{"\"AAAAAX/wAAAAAAAA//AAAAAAAABAWQAAAAAAAAAAAAA=\"", Double.NaN}
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByAggregatorDefaultValues() throws Exception
+  {
+    cannotVectorize();
+    testQuery(
+        "SELECT\n"
+        + "dim2,\n"
+        + "TDIGEST_GENERATE_SKETCH(m1) FILTER(WHERE dim1 = 'nonexistent'),"
+        + "TDIGEST_QUANTILE(qsketch_m1, 0.1) FILTER(WHERE dim1 = 'nonexistent')"
+        + "FROM foo WHERE dim2 = 'a' GROUP BY dim2",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE1)
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setDimFilter(selector("dim2", "a", null))
+                        .setGranularity(Granularities.ALL)
+                        .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING))
+                        .setDimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING))
+                        .setAggregatorSpecs(
+                            aggregators(
+                                new FilteredAggregatorFactory(
+                                    new TDigestSketchAggregatorFactory("a0:agg", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new TDigestSketchAggregatorFactory("a1:agg", "qsketch_m1", 100),
+                                    selector("dim1", "nonexistent", null)
+                                )
+                            )
+                        )
+                        .setPostAggregatorSpecs(
+                            ImmutableList.of(
+                                new TDigestSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a1:agg"), 0.1f)
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{"a", "\"AAAAAX/wAAAAAAAA//AAAAAAAABAWQAAAAAAAAAAAAA=\"", Double.NaN}
+        )
+    );
+  }
+
+  @Override
+  public void assertResultsEquals(String sql, List<Object[]> expectedResults, List<Object[]> results)
+  {
+    Assert.assertEquals(expectedResults.size(), results.size());
+    for (int i = 0; i < expectedResults.size(); i++) {
+      Object[] expectedResult = expectedResults.get(i);
+      Object[] result = results.get(i);
+      Assert.assertEquals(expectedResult.length, result.length);
+      for (int j = 0; j < expectedResult.length; j++) {
+        if (expectedResult[j] instanceof Float) {
+          Assert.assertEquals((Float) expectedResult[j], (Float) result[j], 0.000001);
+        } else if (expectedResult[j] instanceof Double) {
+          Assert.assertEquals((Double) expectedResult[j], (Double) result[j], 0.000001);
+        } else {
+          Assert.assertEquals(expectedResult[j], result[j]);
+        }
+      }
+    }
+  }
+
   private static PostAggregator makeFieldAccessPostAgg(String name)
   {
     return new FieldAccessPostAggregator(name, name);
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
index b1faed1..2c71fa4 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
@@ -23,18 +23,12 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import org.apache.calcite.schema.SchemaPlus;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.granularity.PeriodGranularity;
-import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.query.Druids;
-import org.apache.druid.query.Query;
-import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.QueryDataSource;
-import org.apache.druid.query.QueryRunnerFactoryConglomerate;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
 import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
@@ -49,9 +43,12 @@
 import org.apache.druid.query.aggregation.post.ExpressionPostAggregator;
 import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
 import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.expression.TestExprMacroTable;
 import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
 import org.apache.druid.segment.IndexBuilder;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.segment.TestHelper;
@@ -59,108 +56,34 @@
 import org.apache.druid.segment.incremental.IncrementalIndexSchema;
 import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
 import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
-import org.apache.druid.server.QueryStackTests;
-import org.apache.druid.server.security.AuthTestUtils;
-import org.apache.druid.server.security.AuthenticationResult;
-import org.apache.druid.sql.SqlLifecycle;
-import org.apache.druid.sql.SqlLifecycleFactory;
 import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
 import org.apache.druid.sql.calcite.filtration.Filtration;
 import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
-import org.apache.druid.sql.calcite.planner.PlannerConfig;
-import org.apache.druid.sql.calcite.planner.PlannerContext;
-import org.apache.druid.sql.calcite.planner.PlannerFactory;
-import org.apache.druid.sql.calcite.util.CalciteTestBase;
 import org.apache.druid.sql.calcite.util.CalciteTests;
-import org.apache.druid.sql.calcite.util.QueryLogHook;
 import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.partition.LinearShardSpec;
 import org.joda.time.DateTimeZone;
 import org.joda.time.Period;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 
-@RunWith(Parameterized.class)
-public class HllSketchSqlAggregatorTest extends CalciteTestBase
+public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
 {
-  private static final String DATA_SOURCE = "foo";
   private static final boolean ROUND = true;
-  private static QueryRunnerFactoryConglomerate conglomerate;
-  private static Closer resourceCloser;
-  private static AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
 
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-  @Rule
-  public QueryLogHook queryLogHook = QueryLogHook.create(TestHelper.JSON_MAPPER);
-
-  private final Map<String, Object> queryContext;
-  private SpecificSegmentsQuerySegmentWalker walker;
-  private SqlLifecycleFactory sqlLifecycleFactory;
-
-  public HllSketchSqlAggregatorTest(final String vectorize)
-  {
-    this.queryContext = ImmutableMap.of(
-        PlannerContext.CTX_SQL_QUERY_ID, "dummy",
-        QueryContexts.VECTORIZE_KEY, vectorize,
-        QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize
-    );
-  }
-
-  @Parameterized.Parameters(name = "vectorize = {0}")
-  public static Collection<?> constructorFeeder()
-  {
-    final List<Object[]> constructors = new ArrayList<>();
-    for (String vectorize : new String[]{"false", "true", "force"}) {
-      constructors.add(new Object[]{vectorize});
-    }
-    return constructors;
-  }
-
-  @BeforeClass
-  public static void setUpClass()
-  {
-    resourceCloser = Closer.create();
-    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(resourceCloser);
-  }
-
-  @AfterClass
-  public static void tearDownClass() throws IOException
-  {
-    resourceCloser.close();
-  }
-
-  @Before
-  public void setUp() throws Exception
+  @Override
+  public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker() throws IOException
   {
     HllSketchModule.registerSerde();
     for (Module mod : new HllSketchModule().getJacksonModules()) {
       CalciteTests.getJsonMapper().registerModule(mod);
       TestHelper.JSON_MAPPER.registerModule(mod);
     }
-
     final QueryableIndex index = IndexBuilder.create()
                                              .tmpDir(temporaryFolder.newFolder())
                                              .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
@@ -185,7 +108,7 @@
 
     walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
         DataSegment.builder()
-                   .dataSource(DATA_SOURCE)
+                   .dataSource(CalciteTests.DATASOURCE1)
                    .interval(index.getDataInterval())
                    .version("1")
                    .shardSpec(new LinearShardSpec(0))
@@ -193,164 +116,115 @@
                    .build(),
         index
     );
+    return walker;
+  }
 
-    final PlannerConfig plannerConfig = new PlannerConfig();
-    final DruidOperatorTable operatorTable = new DruidOperatorTable(
+  @Override
+  public DruidOperatorTable createOperatorTable()
+  {
+    return new DruidOperatorTable(
         ImmutableSet.of(
             new HllSketchApproxCountDistinctSqlAggregator(),
             new HllSketchObjectSqlAggregator()
         ),
         ImmutableSet.of(
-            new HllSketchEstimateOperatorConversion(),
-            new HllSketchEstimateWithErrorBoundsOperatorConversion(),
             new HllSketchSetUnionOperatorConversion(),
-            new HllSketchToStringOperatorConversion()
+            new HllSketchEstimateOperatorConversion(),
+            new HllSketchToStringOperatorConversion(),
+            new HllSketchEstimateWithErrorBoundsOperatorConversion()
         )
     );
-
-    SchemaPlus rootSchema =
-        CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
-    sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
-        new PlannerFactory(
-            rootSchema,
-            CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
-            operatorTable,
-            CalciteTests.createExprMacroTable(),
-            plannerConfig,
-            AuthTestUtils.TEST_AUTHORIZER_MAPPER,
-            CalciteTests.getJsonMapper(),
-            CalciteTests.DRUID_SCHEMA_NAME
-        )
-    );
-  }
-
-  @After
-  public void tearDown() throws Exception
-  {
-    walker.close();
-    walker = null;
   }
 
   @Test
   public void testApproxCountDistinctHllSketch() throws Exception
   {
-    // Can't vectorize due to CONCAT expression.
+    // Can't vectorize due to SUBSTRING expression.
     cannotVectorize();
 
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-
-    final String sql = "SELECT\n"
-                       + "  SUM(cnt),\n"
-                       + "  APPROX_COUNT_DISTINCT_DS_HLL(dim2),\n" // uppercase
-                       + "  APPROX_COUNT_DISTINCT_DS_HLL(dim2) FILTER(WHERE dim2 <> ''),\n" // lowercase; also, filtered
-                       + "  APPROX_COUNT_DISTINCT_DS_HLL(SUBSTRING(dim2, 1, 1)),\n" // on extractionFn
-                       + "  APPROX_COUNT_DISTINCT_DS_HLL(SUBSTRING(dim2, 1, 1) || 'x'),\n" // on expression
-                       + "  APPROX_COUNT_DISTINCT_DS_HLL(hllsketch_dim1, 21, 'HLL_8'),\n" // on native HllSketch column
-                       + "  APPROX_COUNT_DISTINCT_DS_HLL(hllsketch_dim1)\n" // on native HllSketch column
-                       + "FROM druid.foo";
-
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        queryContext,
-        DEFAULT_PARAMETERS,
-        authenticationResult
-    ).toList();
     final List<Object[]> expectedResults;
 
     if (NullHandling.replaceWithDefault()) {
       expectedResults = ImmutableList.of(
-          new Object[]{
-              6L,
-              2L,
-              2L,
-              1L,
-              2L,
-              5L,
-              5L
-          }
+          new Object[]{6L, 2L, 2L, 1L, 2L, 5L, 5L}
       );
     } else {
       expectedResults = ImmutableList.of(
-          new Object[]{
-              6L,
-              2L,
-              2L,
-              1L,
-              1L,
-              5L,
-              5L
-          }
+          new Object[]{6L, 2L, 2L, 1L, 1L, 5L, 5L}
       );
     }
 
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
-
-    // Verify query
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .virtualColumns(
-                  new ExpressionVirtualColumn(
-                      "v0",
-                      "substring(\"dim2\", 0, 1)",
-                      ValueType.STRING,
-                      TestExprMacroTable.INSTANCE
-                  ),
-                  new ExpressionVirtualColumn(
-                      "v1",
-                      "concat(substring(\"dim2\", 0, 1),'x')",
-                      ValueType.STRING,
-                      TestExprMacroTable.INSTANCE
-                  )
-              )
-              .aggregators(
-                  ImmutableList.of(
-                      new LongSumAggregatorFactory("a0", "cnt"),
-                      new HllSketchBuildAggregatorFactory(
-                          "a1",
-                          "dim2",
-                          null,
-                          null,
-                          ROUND
+    testQuery(
+        "SELECT\n"
+        + "  SUM(cnt),\n"
+        + "  APPROX_COUNT_DISTINCT_DS_HLL(dim2),\n" // uppercase
+        + "  APPROX_COUNT_DISTINCT_DS_HLL(dim2) FILTER(WHERE dim2 <> ''),\n" // lowercase; also, filtered
+        + "  APPROX_COUNT_DISTINCT_DS_HLL(SUBSTRING(dim2, 1, 1)),\n" // on extractionFn
+        + "  APPROX_COUNT_DISTINCT_DS_HLL(SUBSTRING(dim2, 1, 1) || 'x'),\n" // on expression
+        + "  APPROX_COUNT_DISTINCT_DS_HLL(hllsketch_dim1, 21, 'HLL_8'),\n" // on native HllSketch column
+        + "  APPROX_COUNT_DISTINCT_DS_HLL(hllsketch_dim1)\n" // on native HllSketch column
+        + "FROM druid.foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .virtualColumns(
+                      new ExpressionVirtualColumn(
+                          "v0",
+                          "substring(\"dim2\", 0, 1)",
+                          ValueType.STRING,
+                          TestExprMacroTable.INSTANCE
                       ),
-                      new FilteredAggregatorFactory(
+                      new ExpressionVirtualColumn(
+                          "v1",
+                          "concat(substring(\"dim2\", 0, 1),'x')",
+                          ValueType.STRING,
+                          TestExprMacroTable.INSTANCE
+                      )
+                  )
+                  .aggregators(
+                      ImmutableList.of(
+                          new LongSumAggregatorFactory("a0", "cnt"),
                           new HllSketchBuildAggregatorFactory(
-                              "a2",
+                              "a1",
                               "dim2",
                               null,
                               null,
                               ROUND
                           ),
-                          BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null))
-                      ),
-                      new HllSketchBuildAggregatorFactory(
-                          "a3",
-                          "v0",
-                          null,
-                          null,
-                          ROUND
-                      ),
-                      new HllSketchBuildAggregatorFactory(
-                          "a4",
-                          "v1",
-                          null,
-                          null,
-                          ROUND
-                      ),
-                      new HllSketchMergeAggregatorFactory("a5", "hllsketch_dim1", 21, "HLL_8", ROUND),
-                      new HllSketchMergeAggregatorFactory("a6", "hllsketch_dim1", null, null, ROUND)
+                          new FilteredAggregatorFactory(
+                              new HllSketchBuildAggregatorFactory(
+                                  "a2",
+                                  "dim2",
+                                  null,
+                                  null,
+                                  ROUND
+                              ),
+                              BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null))
+                          ),
+                          new HllSketchBuildAggregatorFactory(
+                              "a3",
+                              "v0",
+                              null,
+                              null,
+                              ROUND
+                          ),
+                          new HllSketchBuildAggregatorFactory(
+                              "a4",
+                              "v1",
+                              null,
+                              null,
+                              ROUND
+                          ),
+                          new HllSketchMergeAggregatorFactory("a5", "hllsketch_dim1", 21, "HLL_8", ROUND),
+                          new HllSketchMergeAggregatorFactory("a6", "hllsketch_dim1", null, null, ROUND)
+                      )
                   )
-              )
-              .context(queryContext)
-              .build()
-              .withOverriddenContext(ImmutableMap.of("skipEmptyBuckets", true)),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        expectedResults
     );
   }
 
@@ -361,363 +235,421 @@
     // Can't vectorize due to outer query, which runs on an inline datasource.
     cannotVectorize();
 
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-
-    final String sql = "SELECT\n"
-                       + "  AVG(u)\n"
-                       + "FROM ("
-                       + "  SELECT FLOOR(__time TO DAY), APPROX_COUNT_DISTINCT_DS_HLL(cnt) AS u\n"
-                       + "  FROM druid.foo\n"
-                       + "  GROUP BY 1\n"
-                       + ")";
-
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        queryContext,
-        DEFAULT_PARAMETERS,
-        authenticationResult
-    ).toList();
     final List<Object[]> expectedResults = ImmutableList.of(
         new Object[]{
             1L
         }
     );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
 
-    Query expected = GroupByQuery.builder()
-                                 .setDataSource(
-                                     new QueryDataSource(
-                                         Druids.newTimeseriesQueryBuilder()
-                                               .dataSource(CalciteTests.DATASOURCE1)
-                                               .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(
-                                                   Filtration.eternity()
-                                               )))
-                                               .granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC))
-                                               .aggregators(
-                                                   Collections.singletonList(
-                                                       new HllSketchBuildAggregatorFactory(
-                                                           "a0:a",
-                                                           "cnt",
-                                                           null,
-                                                           null,
-                                                           ROUND
-                                                       )
-                                                   )
-                                               )
-                                               .postAggregators(
-                                                   ImmutableList.of(
-                                                       new FinalizingFieldAccessPostAggregator("a0", "a0:a")
-                                                   )
-                                               )
-                                               .context(queryContext)
-                                               .build()
-                                               .withOverriddenContext(
-                                                   BaseCalciteQueryTest.getTimeseriesContextWithFloorTime(
-                                                       ImmutableMap.of("skipEmptyBuckets", true, "sqlQueryId", "dummy"),
-                                                       "d0"
-                                                   )
-                                               )
-                                     )
-                                 )
-                                 .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-                                 .setGranularity(Granularities.ALL)
-                                 .setAggregatorSpecs(
-                                     NullHandling.replaceWithDefault()
-                                     ? Arrays.asList(
-                                       new LongSumAggregatorFactory("_a0:sum", "a0"),
-                                       new CountAggregatorFactory("_a0:count")
-                                     )
-                                     : Arrays.asList(
-                                         new LongSumAggregatorFactory("_a0:sum", "a0"),
-                                         new FilteredAggregatorFactory(
-                                             new CountAggregatorFactory("_a0:count"),
-                                             BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("a0", null, null))
-                                         )
-                                     )
-                                 )
-                                 .setPostAggregatorSpecs(
-                                     ImmutableList.of(
-                                         new ArithmeticPostAggregator(
-                                             "_a0",
-                                             "quotient",
-                                             ImmutableList.of(
-                                                 new FieldAccessPostAggregator(null, "_a0:sum"),
-                                                 new FieldAccessPostAggregator(null, "_a0:count")
-                                             )
-                                         )
-                                     )
-                                 )
-                                 .setContext(queryContext)
-                                 .build();
-
-    Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
-
-    // Verify query
-    Assert.assertEquals(expected, actual);
+    testQuery(
+        "SELECT\n"
+        + "  AVG(u)\n"
+        + "FROM ("
+        + "  SELECT FLOOR(__time TO DAY), APPROX_COUNT_DISTINCT_DS_HLL(cnt) AS u\n"
+        + "  FROM druid.foo\n"
+        + "  GROUP BY 1\n"
+        + ")",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                Druids.newTimeseriesQueryBuilder()
+                                      .dataSource(CalciteTests.DATASOURCE1)
+                                      .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(
+                                          Filtration.eternity()
+                                      )))
+                                      .granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC))
+                                      .aggregators(
+                                          Collections.singletonList(
+                                              new HllSketchBuildAggregatorFactory(
+                                                  "a0:a",
+                                                  "cnt",
+                                                  null,
+                                                  null,
+                                                  ROUND
+                                              )
+                                          )
+                                      )
+                                      .postAggregators(
+                                          ImmutableList.of(
+                                              new FinalizingFieldAccessPostAggregator("a0", "a0:a")
+                                          )
+                                      )
+                                      .context(QUERY_CONTEXT_DEFAULT)
+                                      .build()
+                                      .withOverriddenContext(
+                                          BaseCalciteQueryTest.getTimeseriesContextWithFloorTime(
+                                              ImmutableMap.of(
+                                                  TimeseriesQuery.SKIP_EMPTY_BUCKETS,
+                                                  true,
+                                                  "sqlQueryId",
+                                                  "dummy"
+                                              ),
+                                              "d0"
+                                          )
+                                      )
+                            )
+                        )
+                        .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(
+                            NullHandling.replaceWithDefault()
+                            ? Arrays.asList(
+                                new LongSumAggregatorFactory("_a0:sum", "a0"),
+                                new CountAggregatorFactory("_a0:count")
+                            )
+                            : Arrays.asList(
+                                new LongSumAggregatorFactory("_a0:sum", "a0"),
+                                new FilteredAggregatorFactory(
+                                    new CountAggregatorFactory("_a0:count"),
+                                    BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("a0", null, null))
+                                )
+                            )
+                        )
+                        .setPostAggregatorSpecs(
+                            ImmutableList.of(
+                                new ArithmeticPostAggregator(
+                                    "_a0",
+                                    "quotient",
+                                    ImmutableList.of(
+                                        new FieldAccessPostAggregator(null, "_a0:sum"),
+                                        new FieldAccessPostAggregator(null, "_a0:count")
+                                    )
+                                )
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        expectedResults
+    );
   }
 
   @Test
   public void testApproxCountDistinctHllSketchIsRounded() throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-
-    final String sql = "SELECT"
-                       + "   dim2,"
-                       + "   APPROX_COUNT_DISTINCT_DS_HLL(m1)"
-                       + " FROM druid.foo"
-                       + " GROUP BY dim2"
-                       + " HAVING APPROX_COUNT_DISTINCT_DS_HLL(m1) = 2";
-
-    // Verify results
-    final List<Object[]> results =
-        sqlLifecycle.runSimple(sql, queryContext, DEFAULT_PARAMETERS, authenticationResult).toList();
-    final int expected = NullHandling.replaceWithDefault() ? 1 : 2;
-    Assert.assertEquals(expected, results.size());
+    testQuery(
+        "SELECT"
+        + "   dim2,"
+        + "   APPROX_COUNT_DISTINCT_DS_HLL(m1)"
+        + " FROM druid.foo"
+        + " GROUP BY dim2"
+        + " HAVING APPROX_COUNT_DISTINCT_DS_HLL(m1) = 2",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setDataSource(CalciteTests.DATASOURCE1)
+                        .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(
+                            aggregators(
+                                new HllSketchBuildAggregatorFactory("a0", "m1", null, null, true)
+                            )
+                        )
+                        .setHavingSpec(having(selector("a0", "2", null)))
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        NullHandling.sqlCompatible()
+        ? ImmutableList.of(
+            new Object[]{null, 2L},
+            new Object[]{"a", 2L}
+        )
+        : ImmutableList.of(
+            new Object[]{"a", 2L}
+        )
+    );
   }
 
   @Test
   public void testHllSketchPostAggs() throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
+    final String sketchSummary = "### HLL SKETCH SUMMARY: \n"
+                                 + "  Log Config K   : 12\n"
+                                 + "  Hll Target     : HLL_4\n"
+                                 + "  Current Mode   : LIST\n"
+                                 + "  Memory         : false\n"
+                                 + "  LB             : 2.0\n"
+                                 + "  Estimate       : 2.000000004967054\n"
+                                 + "  UB             : 2.000099863468538\n"
+                                 + "  OutOfOrder Flag: false\n"
+                                 + "  Coupon Count   : 2\n";
 
-    final String sql = "SELECT\n"
-                       + "  DS_HLL(dim2),\n"
-                       + "  DS_HLL(m1),\n"
-                       + "  HLL_SKETCH_ESTIMATE(DS_HLL(dim2)),\n"
-                       + "  HLL_SKETCH_ESTIMATE(DS_HLL(dim2)) + 1,\n"
-                       + "  HLL_SKETCH_ESTIMATE(DS_HLL(CONCAT(dim2, 'hello'))),\n"
-                       + "  ABS(HLL_SKETCH_ESTIMATE(DS_HLL(dim2))),\n"
-                       + "  HLL_SKETCH_ESTIMATE_WITH_ERROR_BOUNDS(DS_HLL(dim2), 2),\n"
-                       + "  HLL_SKETCH_ESTIMATE_WITH_ERROR_BOUNDS(DS_HLL(dim2)),\n"
-                       + "  DS_HLL(POWER(ABS(m1 + 100), 2)),\n"
-                       + "  APPROX_COUNT_DISTINCT_DS_HLL(dim2),\n"
-                       + "  HLL_SKETCH_TO_STRING(DS_HLL(dim2)),\n"
-                       + "  UPPER(HLL_SKETCH_TO_STRING(DS_HLL(dim2))),\n"
-                       + "  HLL_SKETCH_ESTIMATE(DS_HLL(dim2), true)\n"
-                       + "FROM druid.foo";
-
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        queryContext,
-        DEFAULT_PARAMETERS,
-        authenticationResult
-    ).toList();
-    final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{
-            "\"AgEHDAMIAgDhUv8P63iABQ==\"",
-            "\"AgEHDAMIBgALpZ0PjpTfBY5ElQo+C7UE4jA+DKfcYQQ=\"",
-            2.000000004967054d,
-            3.000000004967054d,
-            3.000000014901161d,
-            2.000000004967054d,
-            "[2.000000004967054,2.0,2.0001997319422404]",
-            "[2.000000004967054,2.0,2.000099863468538]",
-            "\"AgEHDAMIBgC1EYgH1mlHBwsKPwu5SK8MIiUxB7iZVwU=\"",
-            2L,
-            "### HLL SKETCH SUMMARY: \n"
-              + "  Log Config K   : 12\n"
-              + "  Hll Target     : HLL_4\n"
-              + "  Current Mode   : LIST\n"
-              + "  Memory         : false\n"
-              + "  LB             : 2.0\n"
-              + "  Estimate       : 2.000000004967054\n"
-              + "  UB             : 2.000099863468538\n"
-              + "  OutOfOrder Flag: false\n"
-              + "  Coupon Count   : 2\n",
-            "### HLL SKETCH SUMMARY: \n"
-              + "  LOG CONFIG K   : 12\n"
-              + "  HLL TARGET     : HLL_4\n"
-              + "  CURRENT MODE   : LIST\n"
-              + "  MEMORY         : FALSE\n"
-              + "  LB             : 2.0\n"
-              + "  ESTIMATE       : 2.000000004967054\n"
-              + "  UB             : 2.000099863468538\n"
-              + "  OUTOFORDER FLAG: FALSE\n"
-              + "  COUPON COUNT   : 2\n",
-            2.0
-        }
-    );
-
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
-
-    Query actualQuery = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
-
-    Query expectedQuery =
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .virtualColumns(
-                  new ExpressionVirtualColumn(
-                      "v0",
-                      "concat(\"dim2\",'hello')",
-                      ValueType.STRING,
-                      TestExprMacroTable.INSTANCE
-                  ),
-                  new ExpressionVirtualColumn(
-                      "v1",
-                      "pow(abs((\"m1\" + 100)),2)",
-                      ValueType.DOUBLE,
-                      TestExprMacroTable.INSTANCE
-                  )
-              )
-              .aggregators(
-                  ImmutableList.of(
-                      new HllSketchBuildAggregatorFactory(
-                          "a0",
-                          "dim2",
-                          null,
-                          null,
-                          true
-                      ),
-                      new HllSketchBuildAggregatorFactory(
-                          "a1",
-                          "m1",
-                          null,
-                          null,
-                          true
-                      ),
-                      new HllSketchBuildAggregatorFactory(
-                          "a2",
+    final String otherSketchSummary = "### HLL SKETCH SUMMARY: \n"
+                                      + "  LOG CONFIG K   : 12\n"
+                                      + "  HLL TARGET     : HLL_4\n"
+                                      + "  CURRENT MODE   : LIST\n"
+                                      + "  MEMORY         : FALSE\n"
+                                      + "  LB             : 2.0\n"
+                                      + "  ESTIMATE       : 2.000000004967054\n"
+                                      + "  UB             : 2.000099863468538\n"
+                                      + "  OUTOFORDER FLAG: FALSE\n"
+                                      + "  COUPON COUNT   : 2\n";
+    testQuery(
+        "SELECT\n"
+        + "  DS_HLL(dim2),\n"
+        + "  DS_HLL(m1),\n"
+        + "  HLL_SKETCH_ESTIMATE(DS_HLL(dim2)),\n"
+        + "  HLL_SKETCH_ESTIMATE(DS_HLL(dim2)) + 1,\n"
+        + "  HLL_SKETCH_ESTIMATE(DS_HLL(CONCAT(dim2, 'hello'))),\n"
+        + "  ABS(HLL_SKETCH_ESTIMATE(DS_HLL(dim2))),\n"
+        + "  HLL_SKETCH_ESTIMATE_WITH_ERROR_BOUNDS(DS_HLL(dim2), 2),\n"
+        + "  HLL_SKETCH_ESTIMATE_WITH_ERROR_BOUNDS(DS_HLL(dim2)),\n"
+        + "  DS_HLL(POWER(ABS(m1 + 100), 2)),\n"
+        + "  APPROX_COUNT_DISTINCT_DS_HLL(dim2),\n"
+        + "  HLL_SKETCH_TO_STRING(DS_HLL(dim2)),\n"
+        + "  UPPER(HLL_SKETCH_TO_STRING(DS_HLL(dim2))),\n"
+        + "  HLL_SKETCH_ESTIMATE(DS_HLL(dim2), true)\n"
+        + "FROM druid.foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .virtualColumns(
+                      new ExpressionVirtualColumn(
                           "v0",
-                          null,
-                          null,
-                          true
+                          "concat(\"dim2\",'hello')",
+                          ValueType.STRING,
+                          TestExprMacroTable.INSTANCE
                       ),
-                      new HllSketchBuildAggregatorFactory(
-                          "a3",
+                      new ExpressionVirtualColumn(
                           "v1",
-                          null,
-                          null,
-                          true
-                      ),
-                      new HllSketchBuildAggregatorFactory(
-                          "a4",
-                          "dim2",
-                          null,
-                          null,
-                          true
+                          "pow(abs((\"m1\" + 100)),2)",
+                          ValueType.DOUBLE,
+                          TestExprMacroTable.INSTANCE
                       )
                   )
-              )
-              .postAggregators(
-                  ImmutableList.of(
-                      new FieldAccessPostAggregator("p0", "a0"),
-                      new FieldAccessPostAggregator("p1", "a1"),
-                      new HllSketchToEstimatePostAggregator("p3", new FieldAccessPostAggregator("p2", "a0"), false),
-                      new HllSketchToEstimatePostAggregator("p5", new FieldAccessPostAggregator("p4", "a0"), false),
-                      new ExpressionPostAggregator("p6", "(p5 + 1)", null, TestExprMacroTable.INSTANCE),
-                      new HllSketchToEstimatePostAggregator("p8", new FieldAccessPostAggregator("p7", "a2"), false),
-                      new HllSketchToEstimatePostAggregator("p10", new FieldAccessPostAggregator("p9", "a0"), false),
-                      new ExpressionPostAggregator("p11", "abs(p10)", null, TestExprMacroTable.INSTANCE),
-                      new HllSketchToEstimateWithBoundsPostAggregator(
-                          "p13",
-                          new FieldAccessPostAggregator("p12", "a0"),
-                          2
-                      ),
-                      new HllSketchToEstimateWithBoundsPostAggregator(
-                          "p15",
-                          new FieldAccessPostAggregator("p14", "a0"),
-                          1
-                      ),
-                      new FieldAccessPostAggregator("p16", "a3"),
-                      new HllSketchToStringPostAggregator("p18", new FieldAccessPostAggregator("p17", "a0")),
-                      new HllSketchToStringPostAggregator("p20", new FieldAccessPostAggregator("p19", "a0")),
-                      new ExpressionPostAggregator("p21", "upper(p20)", null, TestExprMacroTable.INSTANCE),
-                      new HllSketchToEstimatePostAggregator("p23", new FieldAccessPostAggregator("p22", "a0"), true)
+                  .aggregators(
+                      ImmutableList.of(
+                          new HllSketchBuildAggregatorFactory(
+                              "a0",
+                              "dim2",
+                              null,
+                              null,
+                              true
+                          ),
+                          new HllSketchBuildAggregatorFactory(
+                              "a1",
+                              "m1",
+                              null,
+                              null,
+                              true
+                          ),
+                          new HllSketchBuildAggregatorFactory(
+                              "a2",
+                              "v0",
+                              null,
+                              null,
+                              true
+                          ),
+                          new HllSketchBuildAggregatorFactory(
+                              "a3",
+                              "v1",
+                              null,
+                              null,
+                              true
+                          ),
+                          new HllSketchBuildAggregatorFactory(
+                              "a4",
+                              "dim2",
+                              null,
+                              null,
+                              true
+                          )
+                      )
                   )
-              )
-              .context(queryContext)
-              .build()
-              .withOverriddenContext(ImmutableMap.of("skipEmptyBuckets", true));
-
-    // Verify query
-    Assert.assertEquals(expectedQuery, actualQuery);
+                  .postAggregators(
+                      ImmutableList.of(
+                          new FieldAccessPostAggregator("p0", "a0"),
+                          new FieldAccessPostAggregator("p1", "a1"),
+                          new HllSketchToEstimatePostAggregator("p3", new FieldAccessPostAggregator("p2", "a0"), false),
+                          new HllSketchToEstimatePostAggregator("p5", new FieldAccessPostAggregator("p4", "a0"), false),
+                          new ExpressionPostAggregator("p6", "(p5 + 1)", null, TestExprMacroTable.INSTANCE),
+                          new HllSketchToEstimatePostAggregator("p8", new FieldAccessPostAggregator("p7", "a2"), false),
+                          new HllSketchToEstimatePostAggregator(
+                              "p10",
+                              new FieldAccessPostAggregator("p9", "a0"),
+                              false
+                          ),
+                          new ExpressionPostAggregator("p11", "abs(p10)", null, TestExprMacroTable.INSTANCE),
+                          new HllSketchToEstimateWithBoundsPostAggregator(
+                              "p13",
+                              new FieldAccessPostAggregator("p12", "a0"),
+                              2
+                          ),
+                          new HllSketchToEstimateWithBoundsPostAggregator(
+                              "p15",
+                              new FieldAccessPostAggregator("p14", "a0"),
+                              1
+                          ),
+                          new FieldAccessPostAggregator("p16", "a3"),
+                          new HllSketchToStringPostAggregator("p18", new FieldAccessPostAggregator("p17", "a0")),
+                          new HllSketchToStringPostAggregator("p20", new FieldAccessPostAggregator("p19", "a0")),
+                          new ExpressionPostAggregator("p21", "upper(p20)", null, TestExprMacroTable.INSTANCE),
+                          new HllSketchToEstimatePostAggregator("p23", new FieldAccessPostAggregator("p22", "a0"), true)
+                      )
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{
+                "\"AgEHDAMIAgDhUv8P63iABQ==\"",
+                "\"AgEHDAMIBgALpZ0PjpTfBY5ElQo+C7UE4jA+DKfcYQQ=\"",
+                2.000000004967054d,
+                3.000000004967054d,
+                3.000000014901161d,
+                2.000000004967054d,
+                "[2.000000004967054,2.0,2.0001997319422404]",
+                "[2.000000004967054,2.0,2.000099863468538]",
+                "\"AgEHDAMIBgC1EYgH1mlHBwsKPwu5SK8MIiUxB7iZVwU=\"",
+                2L,
+                sketchSummary,
+                otherSketchSummary,
+                2.0
+            }
+        )
+    );
   }
 
   @Test
   public void testtHllSketchPostAggsPostSort() throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
+    final String sketchSummary = "### HLL SKETCH SUMMARY: \n"
+                                 + "  Log Config K   : 12\n"
+                                 + "  Hll Target     : HLL_4\n"
+                                 + "  Current Mode   : LIST\n"
+                                 + "  Memory         : false\n"
+                                 + "  LB             : 2.0\n"
+                                 + "  Estimate       : 2.000000004967054\n"
+                                 + "  UB             : 2.000099863468538\n"
+                                 + "  OutOfOrder Flag: false\n"
+                                 + "  Coupon Count   : 2\n";
 
     final String sql = "SELECT DS_HLL(dim2) as y FROM druid.foo ORDER BY HLL_SKETCH_ESTIMATE(DS_HLL(dim2)) DESC LIMIT 10";
-    final String sql2 = StringUtils.format("SELECT HLL_SKETCH_ESTIMATE(y), HLL_SKETCH_TO_STRING(y) from (%s)", sql);
 
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql2,
-        queryContext,
-        DEFAULT_PARAMETERS,
-        authenticationResult
-    ).toList();
-    final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{
-            2.000000004967054d,
-            "### HLL SKETCH SUMMARY: \n"
-              + "  Log Config K   : 12\n"
-              + "  Hll Target     : HLL_4\n"
-              + "  Current Mode   : LIST\n"
-              + "  Memory         : false\n"
-              + "  LB             : 2.0\n"
-              + "  Estimate       : 2.000000004967054\n"
-              + "  UB             : 2.000099863468538\n"
-              + "  OutOfOrder Flag: false\n"
-              + "  Coupon Count   : 2\n"
-        }
-    );
-
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
-
-    Query actualQuery = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
-
-    Query expectedQuery =
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(
-                  ImmutableList.of(
-                      new HllSketchBuildAggregatorFactory(
-                          "a0",
-                          "dim2",
-                          null,
-                          null,
-                          true
+    testQuery(
+        StringUtils.format("SELECT HLL_SKETCH_ESTIMATE(y), HLL_SKETCH_TO_STRING(y) from (%s)", sql),
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(
+                      ImmutableList.of(
+                          new HllSketchBuildAggregatorFactory(
+                              "a0",
+                              "dim2",
+                              null,
+                              null,
+                              true
+                          )
                       )
                   )
-              )
-              .postAggregators(
-                  ImmutableList.of(
-                      new FieldAccessPostAggregator("p0", "a0"),
-                      new HllSketchToEstimatePostAggregator("p2", new FieldAccessPostAggregator("p1", "a0"), false),
-                      new HllSketchToEstimatePostAggregator("s1", new FieldAccessPostAggregator("s0", "p0"), false),
-                      new HllSketchToStringPostAggregator("s3", new FieldAccessPostAggregator("s2", "p0"))
+                  .postAggregators(
+                      ImmutableList.of(
+                          new FieldAccessPostAggregator("p0", "a0"),
+                          new HllSketchToEstimatePostAggregator("p2", new FieldAccessPostAggregator("p1", "a0"), false),
+                          new HllSketchToEstimatePostAggregator("s1", new FieldAccessPostAggregator("s0", "p0"), false),
+                          new HllSketchToStringPostAggregator("s3", new FieldAccessPostAggregator("s2", "p0"))
+                      )
                   )
-              )
-              .context(queryContext)
-              .build()
-              .withOverriddenContext(ImmutableMap.of("skipEmptyBuckets", true));
-
-    // Verify query
-    Assert.assertEquals(expectedQuery, actualQuery);
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{
+                2.000000004967054d,
+                sketchSummary
+            }
+        )
+    );
   }
 
-  private void cannotVectorize()
+  @Test
+  public void testEmptyTimeseriesResults() throws Exception
   {
-    if (QueryContexts.Vectorize.fromString((String) queryContext.get(QueryContexts.VECTORIZE_KEY))
-        == QueryContexts.Vectorize.FORCE) {
-      expectedException.expectMessage("Cannot vectorize");
-    }
+    // timeseries with all granularity have a single group, so should return default results for given aggregators
+    testQuery(
+        "SELECT\n"
+        + " APPROX_COUNT_DISTINCT_DS_HLL(dim2),\n"
+        + " DS_HLL(dim2)\n"
+        + "FROM druid.foo WHERE dim2 = 0",
+        ImmutableList.of(Druids.newTimeseriesQueryBuilder()
+                               .dataSource(CalciteTests.DATASOURCE1)
+                               .intervals(querySegmentSpec(Filtration.eternity()))
+                               .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
+                               .granularity(Granularities.ALL)
+                               .aggregators(
+                                   aggregators(
+                                       new HllSketchBuildAggregatorFactory(
+                                           "a0",
+                                           "dim2",
+                                           null,
+                                           null,
+                                           true
+                                       ),
+                                       new HllSketchBuildAggregatorFactory(
+                                           "a1",
+                                           "dim2",
+                                           null,
+                                           null,
+                                           true
+                                       )
+                                   )
+                               )
+                               .context(QUERY_CONTEXT_DEFAULT)
+                               .build()),
+        ImmutableList.of(new Object[]{0L, "0"})
+    );
+  }
+
+  @Test
+  public void testGroupByAggregatorDefaultValues() throws Exception
+  {
+    testQuery(
+        "SELECT\n"
+        + "dim2,\n"
+        + "APPROX_COUNT_DISTINCT_DS_HLL(dim2) FILTER(WHERE dim1 = 'nonexistent'),"
+        + "DS_HLL(dim2) FILTER(WHERE dim1 = 'nonexistent')"
+        + "FROM foo WHERE dim2 = 'a' GROUP BY dim2",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE1)
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setDimFilter(selector("dim2", "a", null))
+                        .setGranularity(Granularities.ALL)
+                        .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING))
+                        .setDimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING))
+                        .setAggregatorSpecs(
+                            aggregators(
+                                new FilteredAggregatorFactory(
+                                    new HllSketchBuildAggregatorFactory(
+                                        "a0",
+                                        "v0",
+                                        null,
+                                        null,
+                                        true
+                                    ),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new HllSketchBuildAggregatorFactory(
+                                        "a1",
+                                        "v0",
+                                        null,
+                                        null,
+                                        true
+                                    ),
+                                    selector("dim1", "nonexistent", null)
+                                )
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(new Object[]{"a", 0L, "0"})
+    );
   }
 }
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java
index 647f300..4ebd7c9 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java
@@ -21,7 +21,6 @@
 
 import com.fasterxml.jackson.databind.Module;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.granularity.Granularities;
@@ -49,6 +48,7 @@
 import org.apache.druid.query.filter.NotDimFilter;
 import org.apache.druid.query.filter.SelectorDimFilter;
 import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
 import org.apache.druid.segment.IndexBuilder;
 import org.apache.druid.segment.QueryableIndex;
@@ -56,16 +56,11 @@
 import org.apache.druid.segment.incremental.IncrementalIndexSchema;
 import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
 import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
-import org.apache.druid.server.security.AuthenticationResult;
-import org.apache.druid.sql.SqlLifecycle;
 import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
 import org.apache.druid.sql.calcite.filtration.Filtration;
 import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
-import org.apache.druid.sql.calcite.planner.PlannerConfig;
-import org.apache.druid.sql.calcite.planner.PlannerContext;
 import org.apache.druid.sql.calcite.util.CalciteTests;
 import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
-import org.apache.druid.sql.http.SqlParameter;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.partition.LinearShardSpec;
 import org.junit.Test;
@@ -73,7 +68,6 @@
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 
 public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
 {
@@ -134,36 +128,9 @@
   }
 
   @Override
-  public List<Object[]> getResults(
-      final PlannerConfig plannerConfig,
-      final Map<String, Object> queryContext,
-      final List<SqlParameter> parameters,
-      final String sql,
-      final AuthenticationResult authenticationResult
-  ) throws Exception
+  public DruidOperatorTable createOperatorTable()
   {
-    return getResults(
-        plannerConfig,
-        queryContext,
-        parameters,
-        sql,
-        authenticationResult,
-        OPERATOR_TABLE,
-        CalciteTests.createExprMacroTable(),
-        CalciteTests.TEST_AUTHORIZER_MAPPER,
-        CalciteTests.getJsonMapper()
-    );
-  }
-
-  private SqlLifecycle getSqlLifecycle()
-  {
-    return getSqlLifecycleFactory(
-        BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT,
-        OPERATOR_TABLE,
-        CalciteTests.createExprMacroTable(),
-        CalciteTests.TEST_AUTHORIZER_MAPPER,
-        CalciteTests.getJsonMapper()
-    ).factorize();
+    return OPERATOR_TABLE;
   }
 
   @Test
@@ -220,7 +187,7 @@
                       new DoublesSketchToQuantilePostAggregator("a7", makeFieldAccessPostAgg("a5:agg"), 0.999f),
                       new DoublesSketchToQuantilePostAggregator("a8", makeFieldAccessPostAgg("a8:agg"), 0.50f)
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -279,7 +246,7 @@
                       new DoublesSketchToQuantilePostAggregator("a5", makeFieldAccessPostAgg("a5:agg"), 0.999f),
                       new DoublesSketchToQuantilePostAggregator("a6", makeFieldAccessPostAgg("a4:agg"), 0.999f)
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -382,7 +349,7 @@
                       new DoublesSketchToQuantilePostAggregator("a6", makeFieldAccessPostAgg("a6:agg"), 0.999f),
                       new DoublesSketchToQuantilePostAggregator("a7", makeFieldAccessPostAgg("a5:agg"), 0.999f)
                   )
-                  .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         expectedResults
@@ -667,7 +634,7 @@
                           ExprMacroTable.nil()
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -742,7 +709,7 @@
                           )
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -754,6 +721,106 @@
     );
   }
 
+  @Test
+  public void testEmptyTimeseriesResults() throws Exception
+  {
+    testQuery(
+        "SELECT\n"
+        + "APPROX_QUANTILE_DS(m1, 0.01),\n"
+        + "APPROX_QUANTILE_DS(qsketch_m1, 0.01),\n"
+        + "DS_QUANTILES_SKETCH(m1),\n"
+        + "DS_QUANTILES_SKETCH(qsketch_m1)\n"
+        + "FROM foo WHERE dim2 = 0",
+        Collections.singletonList(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
+                  .aggregators(ImmutableList.of(
+                      new DoublesSketchAggregatorFactory("a0:agg", "m1", null),
+                      new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", null),
+                      new DoublesSketchAggregatorFactory("a2:agg", "m1", null),
+                      new DoublesSketchAggregatorFactory("a3:agg", "qsketch_m1", null)
+                  ))
+                  .postAggregators(
+                      new DoublesSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.01f),
+                      new DoublesSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a1:agg"), 0.01f)
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{
+                Double.NaN,
+                Double.NaN,
+                "0",
+                "0"
+            }
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByAggregatorDefaultValues() throws Exception
+  {
+    testQuery(
+        "SELECT\n"
+        + "dim2,\n"
+        + "APPROX_QUANTILE_DS(m1, 0.01) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "APPROX_QUANTILE_DS(qsketch_m1, 0.01) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "DS_QUANTILES_SKETCH(m1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "DS_QUANTILES_SKETCH(qsketch_m1) FILTER(WHERE dim1 = 'nonexistent')\n"
+        + "FROM foo WHERE dim2 = 'a' GROUP BY dim2",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE1)
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setDimFilter(selector("dim2", "a", null))
+                        .setGranularity(Granularities.ALL)
+                        .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING))
+                        .setDimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING))
+                        .setAggregatorSpecs(
+                            aggregators(
+                                new FilteredAggregatorFactory(
+                                    new DoublesSketchAggregatorFactory("a0:agg", "m1", null),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", null),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new DoublesSketchAggregatorFactory("a2:agg", "m1", null),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new DoublesSketchAggregatorFactory("a3:agg", "qsketch_m1", null),
+                                    selector("dim1", "nonexistent", null)
+                                )
+                            )
+                        )
+                        .setPostAggregatorSpecs(
+                            ImmutableList.of(
+                                new DoublesSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.01f),
+                                new DoublesSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a1:agg"), 0.01f)
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{
+                "a",
+                Double.NaN,
+                Double.NaN,
+                "0",
+                "0"
+            }
+        )
+    );
+  }
+
   private static PostAggregator makeFieldAccessPostAgg(String name)
   {
     return new FieldAccessPostAggregator(name, name);
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
index 0aac2b0..963a69c 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
@@ -21,20 +21,13 @@
 
 import com.fasterxml.jackson.databind.Module;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import org.apache.calcite.schema.SchemaPlus;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.granularity.PeriodGranularity;
-import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.query.Druids;
-import org.apache.druid.query.Query;
-import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.QueryDataSource;
-import org.apache.druid.query.QueryRunnerFactoryConglomerate;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
 import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
@@ -46,8 +39,10 @@
 import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
 import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
 import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.expression.TestExprMacroTable;
 import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
 import org.apache.druid.segment.IndexBuilder;
 import org.apache.druid.segment.QueryableIndex;
@@ -56,101 +51,28 @@
 import org.apache.druid.segment.incremental.IncrementalIndexSchema;
 import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
 import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
-import org.apache.druid.server.QueryStackTests;
-import org.apache.druid.server.security.AuthTestUtils;
-import org.apache.druid.server.security.AuthenticationResult;
-import org.apache.druid.sql.SqlLifecycle;
-import org.apache.druid.sql.SqlLifecycleFactory;
 import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
 import org.apache.druid.sql.calcite.filtration.Filtration;
 import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
-import org.apache.druid.sql.calcite.planner.PlannerConfig;
-import org.apache.druid.sql.calcite.planner.PlannerContext;
-import org.apache.druid.sql.calcite.planner.PlannerFactory;
-import org.apache.druid.sql.calcite.util.CalciteTestBase;
 import org.apache.druid.sql.calcite.util.CalciteTests;
-import org.apache.druid.sql.calcite.util.QueryLogHook;
 import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.partition.LinearShardSpec;
 import org.joda.time.DateTimeZone;
 import org.joda.time.Period;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 
-@RunWith(Parameterized.class)
-public class ThetaSketchSqlAggregatorTest extends CalciteTestBase
+public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
 {
   private static final String DATA_SOURCE = "foo";
 
-  private static QueryRunnerFactoryConglomerate conglomerate;
-  private static Closer resourceCloser;
-  private static AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
-
-  @BeforeClass
-  public static void setUpClass()
-  {
-    resourceCloser = Closer.create();
-    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(resourceCloser);
-  }
-
-  @AfterClass
-  public static void tearDownClass() throws IOException
-  {
-    resourceCloser.close();
-  }
-
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-  @Rule
-  public QueryLogHook queryLogHook = QueryLogHook.create();
-
-  private final Map<String, Object> queryContext;
-  private SpecificSegmentsQuerySegmentWalker walker;
-  private SqlLifecycleFactory sqlLifecycleFactory;
-
-  public ThetaSketchSqlAggregatorTest(final String vectorize)
-  {
-    this.queryContext = ImmutableMap.of(
-        PlannerContext.CTX_SQL_QUERY_ID, "dummy",
-        QueryContexts.VECTORIZE_KEY, vectorize,
-        QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize
-    );
-  }
-
-  @Parameterized.Parameters(name = "vectorize = {0}")
-  public static Collection<?> constructorFeeder()
-  {
-    final List<Object[]> constructors = new ArrayList<>();
-    for (String vectorize : new String[]{"false", "true", "force"}) {
-      constructors.add(new Object[]{vectorize});
-    }
-    return constructors;
-  }
-
-  @Before
-  public void setUp() throws Exception
+  @Override
+  public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker() throws IOException
   {
     SketchModule.registerSerde();
     for (Module mod : new SketchModule().getJacksonModules()) {
@@ -181,7 +103,7 @@
                                              .rows(CalciteTests.ROWS1)
                                              .buildMMappedIndex();
 
-    walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
+    return new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
         DataSegment.builder()
                    .dataSource(DATA_SOURCE)
                    .interval(index.getDataInterval())
@@ -191,9 +113,12 @@
                    .build(),
         index
     );
+  }
 
-    final PlannerConfig plannerConfig = new PlannerConfig();
-    final DruidOperatorTable operatorTable = new DruidOperatorTable(
+  @Override
+  public DruidOperatorTable createOperatorTable()
+  {
+    return new DruidOperatorTable(
         ImmutableSet.of(
             new ThetaSketchApproxCountDistinctSqlAggregator(),
             new ThetaSketchObjectSqlAggregator()
@@ -206,29 +131,8 @@
             new ThetaSketchSetNotOperatorConversion()
         )
     );
-    SchemaPlus rootSchema =
-        CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
-
-    sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
-        new PlannerFactory(
-            rootSchema,
-            CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
-            operatorTable,
-            CalciteTests.createExprMacroTable(),
-            plannerConfig,
-            AuthTestUtils.TEST_AUTHORIZER_MAPPER,
-            CalciteTests.getJsonMapper(),
-            CalciteTests.DRUID_SCHEMA_NAME
-        )
-    );
   }
 
-  @After
-  public void tearDown() throws Exception
-  {
-    walker.close();
-    walker = null;
-  }
 
   @Test
   public void testApproxCountDistinctThetaSketch() throws Exception
@@ -236,30 +140,6 @@
     // Cannot vectorize due to SUBSTRING.
     cannotVectorize();
 
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-    final String sql = "SELECT\n"
-                       + "  SUM(cnt),\n"
-                       + "  APPROX_COUNT_DISTINCT_DS_THETA(dim2),\n"
-                       // uppercase
-                       + "  APPROX_COUNT_DISTINCT_DS_THETA(dim2) FILTER(WHERE dim2 <> ''),\n"
-                       // lowercase; also, filtered
-                       + "  APPROX_COUNT_DISTINCT_DS_THETA(SUBSTRING(dim2, 1, 1)),\n"
-                       // on extractionFn
-                       + "  APPROX_COUNT_DISTINCT_DS_THETA(SUBSTRING(dim2, 1, 1) || 'x'),\n"
-                       // on expression
-                       + "  APPROX_COUNT_DISTINCT_DS_THETA(thetasketch_dim1, 32768),\n"
-                       // on native theta sketch column
-                       + "  APPROX_COUNT_DISTINCT_DS_THETA(thetasketch_dim1)\n"
-                       // on native theta sketch column
-                       + "FROM druid.foo";
-
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        queryContext,
-        DEFAULT_PARAMETERS,
-        authenticationResult
-    ).toList();
     final List<Object[]> expectedResults;
 
     if (NullHandling.replaceWithDefault()) {
@@ -288,77 +168,87 @@
       );
     }
 
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
-
-    // Verify query
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .virtualColumns(
-                  new ExpressionVirtualColumn(
-                      "v0",
-                      "substring(\"dim2\", 0, 1)",
-                      ValueType.STRING,
-                      TestExprMacroTable.INSTANCE
-                  ),
-                  new ExpressionVirtualColumn(
-                      "v1",
-                      "concat(substring(\"dim2\", 0, 1),'x')",
-                      ValueType.STRING,
-                      TestExprMacroTable.INSTANCE
-                  )
-              )
-              .aggregators(
-                  ImmutableList.of(
-                      new LongSumAggregatorFactory("a0", "cnt"),
-                      new SketchMergeAggregatorFactory(
-                          "a1",
-                          "dim2",
-                          null,
-                          null,
-                          null,
-                          null
+    testQuery(
+        "SELECT\n"
+        + "  SUM(cnt),\n"
+        + "  APPROX_COUNT_DISTINCT_DS_THETA(dim2),\n"
+        // uppercase
+        + "  APPROX_COUNT_DISTINCT_DS_THETA(dim2) FILTER(WHERE dim2 <> ''),\n"
+        // lowercase; also, filtered
+        + "  APPROX_COUNT_DISTINCT_DS_THETA(SUBSTRING(dim2, 1, 1)),\n"
+        // on extractionFn
+        + "  APPROX_COUNT_DISTINCT_DS_THETA(SUBSTRING(dim2, 1, 1) || 'x'),\n"
+        // on expression
+        + "  APPROX_COUNT_DISTINCT_DS_THETA(thetasketch_dim1, 32768),\n"
+        // on native theta sketch column
+        + "  APPROX_COUNT_DISTINCT_DS_THETA(thetasketch_dim1)\n"
+        // on native theta sketch column
+        + "FROM druid.foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .virtualColumns(
+                      new ExpressionVirtualColumn(
+                          "v0",
+                          "substring(\"dim2\", 0, 1)",
+                          ValueType.STRING,
+                          TestExprMacroTable.INSTANCE
                       ),
-                      new FilteredAggregatorFactory(
+                      new ExpressionVirtualColumn(
+                          "v1",
+                          "concat(substring(\"dim2\", 0, 1),'x')",
+                          ValueType.STRING,
+                          TestExprMacroTable.INSTANCE
+                      )
+                  )
+                  .aggregators(
+                      ImmutableList.of(
+                          new LongSumAggregatorFactory("a0", "cnt"),
                           new SketchMergeAggregatorFactory(
-                              "a2",
+                              "a1",
                               "dim2",
                               null,
                               null,
                               null,
                               null
                           ),
-                          BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null))
-                      ),
-                      new SketchMergeAggregatorFactory(
-                          "a3",
-                          "v0",
-                          null,
-                          null,
-                          null,
-                          null
-                      ),
-                      new SketchMergeAggregatorFactory(
-                          "a4",
-                          "v1",
-                          null,
-                          null,
-                          null,
-                          null
-                      ),
-                      new SketchMergeAggregatorFactory("a5", "thetasketch_dim1", 32768, null, null, null),
-                      new SketchMergeAggregatorFactory("a6", "thetasketch_dim1", null, null, null, null)
+                          new FilteredAggregatorFactory(
+                              new SketchMergeAggregatorFactory(
+                                  "a2",
+                                  "dim2",
+                                  null,
+                                  null,
+                                  null,
+                                  null
+                              ),
+                              BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null))
+                          ),
+                          new SketchMergeAggregatorFactory(
+                              "a3",
+                              "v0",
+                              null,
+                              null,
+                              null,
+                              null
+                          ),
+                          new SketchMergeAggregatorFactory(
+                              "a4",
+                              "v1",
+                              null,
+                              null,
+                              null,
+                              null
+                          ),
+                          new SketchMergeAggregatorFactory("a5", "thetasketch_dim1", 32768, null, null, null),
+                          new SketchMergeAggregatorFactory("a6", "thetasketch_dim1", null, null, null, null)
+                      )
                   )
-              )
-              .context(queryContext)
-              .build()
-              .withOverriddenContext(ImmutableMap.of("skipEmptyBuckets", true)),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        expectedResults
     );
   }
 
@@ -368,136 +258,97 @@
     // Can't vectorize due to outer query (it operates on an inlined data source, which cannot be vectorized).
     cannotVectorize();
 
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-
-    final String sql = "SELECT\n"
-                       + "  AVG(u)\n"
-                       + "FROM (SELECT FLOOR(__time TO DAY), APPROX_COUNT_DISTINCT_DS_THETA(cnt) AS u FROM druid.foo GROUP BY 1)";
-
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        queryContext,
-        DEFAULT_PARAMETERS,
-        authenticationResult
-    ).toList();
     final List<Object[]> expectedResults = ImmutableList.of(
         new Object[]{
             1L
         }
     );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
 
-    Query expected = GroupByQuery.builder()
-                                 .setDataSource(
-                                     new QueryDataSource(Druids.newTimeseriesQueryBuilder()
-                                                               .dataSource(CalciteTests.DATASOURCE1)
-                                                               .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(
-                                                                   Filtration.eternity()
-                                                               )))
-                                                               .granularity(new PeriodGranularity(
-                                                                   Period.days(1),
-                                                                   null,
-                                                                   DateTimeZone.UTC
-                                                               ))
-                                                               .aggregators(
-                                                                   Collections.singletonList(
-                                                                       new SketchMergeAggregatorFactory(
-                                                                           "a0:a",
-                                                                           "cnt",
-                                                                           null,
-                                                                           null,
-                                                                           null,
-                                                                           null
-                                                                       )
-                                                                   )
-                                                               )
-                                                               .postAggregators(
-                                                                   ImmutableList.of(
-                                                                       new FinalizingFieldAccessPostAggregator(
-                                                                           "a0",
-                                                                           "a0:a"
-                                                                       )
-                                                                   )
-                                                               )
-                                                               .context(queryContext)
-                                                               .build()
-                                                               .withOverriddenContext(
-                                                                   BaseCalciteQueryTest.getTimeseriesContextWithFloorTime(
-                                                                       ImmutableMap.of(
-                                                                           "skipEmptyBuckets",
-                                                                           true,
-                                                                           "sqlQueryId",
-                                                                           "dummy"
-                                                                       ),
-                                                                       "d0"
-                                                                   )
-                                                               )
-                                     )
-                                 )
-                                 .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-                                 .setGranularity(Granularities.ALL)
-                                 .setAggregatorSpecs(
-                                     NullHandling.replaceWithDefault()
-                                     ? Arrays.asList(
-                                         new LongSumAggregatorFactory("_a0:sum", "a0"),
-                                         new CountAggregatorFactory("_a0:count")
-                                     )
-                                     : Arrays.asList(
-                                         new LongSumAggregatorFactory("_a0:sum", "a0"),
-                                         new FilteredAggregatorFactory(
-                                             new CountAggregatorFactory("_a0:count"),
-                                             BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("a0", null, null))
-                                         )
-                                     )
-                                 )
-                                 .setPostAggregatorSpecs(
-                                     ImmutableList.of(
-                                         new ArithmeticPostAggregator(
-                                             "_a0",
-                                             "quotient",
-                                             ImmutableList.of(
-                                                 new FieldAccessPostAggregator(null, "_a0:sum"),
-                                                 new FieldAccessPostAggregator(null, "_a0:count")
-                                             )
-                                         )
-                                     )
-                                 )
-                                 .setContext(queryContext)
-                                 .build();
-
-    Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
-
-    // Verify query
-    Assert.assertEquals(expected, actual);
+    testQuery(
+        "SELECT\n"
+        + "  AVG(u)\n"
+        + "FROM (SELECT FLOOR(__time TO DAY), APPROX_COUNT_DISTINCT_DS_THETA(cnt) AS u FROM druid.foo GROUP BY 1)",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(Druids.newTimeseriesQueryBuilder()
+                                                      .dataSource(CalciteTests.DATASOURCE1)
+                                                      .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(
+                                                          Filtration.eternity()
+                                                      )))
+                                                      .granularity(new PeriodGranularity(
+                                                          Period.days(1),
+                                                          null,
+                                                          DateTimeZone.UTC
+                                                      ))
+                                                      .aggregators(
+                                                          Collections.singletonList(
+                                                              new SketchMergeAggregatorFactory(
+                                                                  "a0:a",
+                                                                  "cnt",
+                                                                  null,
+                                                                  null,
+                                                                  null,
+                                                                  null
+                                                              )
+                                                          )
+                                                      )
+                                                      .postAggregators(
+                                                          ImmutableList.of(
+                                                              new FinalizingFieldAccessPostAggregator(
+                                                                  "a0",
+                                                                  "a0:a"
+                                                              )
+                                                          )
+                                                      )
+                                                      .context(TIMESERIES_CONTEXT_BY_GRAN)
+                                                      .build()
+                                                      .withOverriddenContext(
+                                                          BaseCalciteQueryTest.getTimeseriesContextWithFloorTime(
+                                                              TIMESERIES_CONTEXT_BY_GRAN,
+                                                              "d0"
+                                                          )
+                                                      )
+                            )
+                        )
+                        .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(
+                            NullHandling.replaceWithDefault()
+                            ? Arrays.asList(
+                                new LongSumAggregatorFactory("_a0:sum", "a0"),
+                                new CountAggregatorFactory("_a0:count")
+                            )
+                            : Arrays.asList(
+                                new LongSumAggregatorFactory("_a0:sum", "a0"),
+                                new FilteredAggregatorFactory(
+                                    new CountAggregatorFactory("_a0:count"),
+                                    BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("a0", null, null))
+                                )
+                            )
+                        )
+                        .setPostAggregatorSpecs(
+                            ImmutableList.of(
+                                new ArithmeticPostAggregator(
+                                    "_a0",
+                                    "quotient",
+                                    ImmutableList.of(
+                                        new FieldAccessPostAggregator(null, "_a0:sum"),
+                                        new FieldAccessPostAggregator(null, "_a0:count")
+                                    )
+                                )
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        expectedResults
+    );
   }
 
   @Test
   public void testThetaSketchPostAggs() throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-    final String sql = "SELECT\n"
-                       + "  SUM(cnt),\n"
-                       + "  theta_sketch_estimate(DS_THETA(dim2)),\n"
-                       + "  theta_sketch_estimate(DS_THETA(CONCAT(dim2, 'hello'))),\n"
-                       + "  theta_sketch_estimate_with_error_bounds(DS_THETA(dim2), 10),\n"
-                       + "  THETA_SKETCH_INTERSECT(DS_THETA(dim2), DS_THETA(dim1)),\n"
-                       + "  THETA_SKETCH_UNION(DS_THETA(dim2), DS_THETA(dim1)),\n"
-                       + "  THETA_SKETCH_NOT(DS_THETA(dim2), DS_THETA(dim1)),\n"
-                       + "  THETA_SKETCH_INTERSECT(32768, DS_THETA(dim2), DS_THETA(dim1)),\n"
-                       + "  theta_sketch_estimate(THETA_SKETCH_INTERSECT(THETA_SKETCH_INTERSECT(DS_THETA(dim2), DS_THETA(dim1)), DS_THETA(dim2)))\n"
-                       + "FROM druid.foo";
-
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        queryContext,
-        DEFAULT_PARAMETERS,
-        authenticationResult
-    ).toList();
     final List<Object[]> expectedResults;
 
     if (NullHandling.replaceWithDefault()) {
@@ -530,209 +381,321 @@
       );
     }
 
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
-
-    Query actualQuery = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
-
-    Query expectedQuery =
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .virtualColumns(
-                  new ExpressionVirtualColumn(
-                      "v0",
-                      "concat(\"dim2\",'hello')",
-                      ValueType.STRING,
-                      TestExprMacroTable.INSTANCE
-                  )
-              )
-              .aggregators(
-                  ImmutableList.of(
-                      new LongSumAggregatorFactory("a0", "cnt"),
-                      new SketchMergeAggregatorFactory(
-                          "a1",
-                          "dim2",
-                          null,
-                          null,
-                          null,
-                          null
-                      ),
-                      new SketchMergeAggregatorFactory(
-                          "a2",
+    testQuery(
+        "SELECT\n"
+        + "  SUM(cnt),\n"
+        + "  theta_sketch_estimate(DS_THETA(dim2)),\n"
+        + "  theta_sketch_estimate(DS_THETA(CONCAT(dim2, 'hello'))),\n"
+        + "  theta_sketch_estimate_with_error_bounds(DS_THETA(dim2), 10),\n"
+        + "  THETA_SKETCH_INTERSECT(DS_THETA(dim2), DS_THETA(dim1)),\n"
+        + "  THETA_SKETCH_UNION(DS_THETA(dim2), DS_THETA(dim1)),\n"
+        + "  THETA_SKETCH_NOT(DS_THETA(dim2), DS_THETA(dim1)),\n"
+        + "  THETA_SKETCH_INTERSECT(32768, DS_THETA(dim2), DS_THETA(dim1)),\n"
+        + "  theta_sketch_estimate(THETA_SKETCH_INTERSECT(THETA_SKETCH_INTERSECT(DS_THETA(dim2), DS_THETA(dim1)), DS_THETA(dim2)))\n"
+        + "FROM druid.foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .virtualColumns(
+                      new ExpressionVirtualColumn(
                           "v0",
-                          null,
-                          null,
-                          null,
-                          null
-                      ),
-                      new SketchMergeAggregatorFactory(
-                          "a3",
-                          "dim1",
-                          null,
-                          null,
-                          null,
-                          null
+                          "concat(\"dim2\",'hello')",
+                          ValueType.STRING,
+                          TestExprMacroTable.INSTANCE
                       )
                   )
-              )
-              .postAggregators(
-                  new SketchEstimatePostAggregator(
-                      "p1",
-                      new FieldAccessPostAggregator("p0", "a1"),
-                      null
-                  ),
-                  new SketchEstimatePostAggregator(
-                      "p3",
-                      new FieldAccessPostAggregator("p2", "a2"),
-                      null
-                  ),
-                  new SketchEstimatePostAggregator(
-                      "p5",
-                      new FieldAccessPostAggregator("p4", "a1"),
-                      10
-                  ),
-                  new SketchSetPostAggregator(
-                      "p8",
-                      "INTERSECT",
-                      null,
+                  .aggregators(
                       ImmutableList.of(
-                          new FieldAccessPostAggregator("p6", "a1"),
-                          new FieldAccessPostAggregator("p7", "a3")
+                          new LongSumAggregatorFactory("a0", "cnt"),
+                          new SketchMergeAggregatorFactory(
+                              "a1",
+                              "dim2",
+                              null,
+                              null,
+                              null,
+                              null
+                          ),
+                          new SketchMergeAggregatorFactory(
+                              "a2",
+                              "v0",
+                              null,
+                              null,
+                              null,
+                              null
+                          ),
+                          new SketchMergeAggregatorFactory(
+                              "a3",
+                              "dim1",
+                              null,
+                              null,
+                              null,
+                              null
+                          )
                       )
-                  ),
-                  new SketchSetPostAggregator(
-                      "p11",
-                      "UNION",
-                      null,
-                      ImmutableList.of(
-                          new FieldAccessPostAggregator("p9", "a1"),
-                          new FieldAccessPostAggregator("p10", "a3")
-                      )
-                  ),
-                  new SketchSetPostAggregator(
-                      "p14",
-                      "NOT",
-                      null,
-                      ImmutableList.of(
-                          new FieldAccessPostAggregator("p12", "a1"),
-                          new FieldAccessPostAggregator("p13", "a3")
-                      )
-                  ),
-                  new SketchSetPostAggregator(
-                      "p17",
-                      "INTERSECT",
-                      32768,
-                      ImmutableList.of(
-                          new FieldAccessPostAggregator("p15", "a1"),
-                          new FieldAccessPostAggregator("p16", "a3")
-                      )
-                  ),
-                  new SketchEstimatePostAggregator(
-                      "p23",
+                  )
+                  .postAggregators(
+                      new SketchEstimatePostAggregator(
+                          "p1",
+                          new FieldAccessPostAggregator("p0", "a1"),
+                          null
+                      ),
+                      new SketchEstimatePostAggregator(
+                          "p3",
+                          new FieldAccessPostAggregator("p2", "a2"),
+                          null
+                      ),
+                      new SketchEstimatePostAggregator(
+                          "p5",
+                          new FieldAccessPostAggregator("p4", "a1"),
+                          10
+                      ),
                       new SketchSetPostAggregator(
-                          "p22",
+                          "p8",
                           "INTERSECT",
                           null,
                           ImmutableList.of(
-                              new SketchSetPostAggregator(
-                                  "p20",
-                                  "INTERSECT",
-                                  null,
-                                  ImmutableList.of(
-                                      new FieldAccessPostAggregator("p18", "a1"),
-                                      new FieldAccessPostAggregator("p19", "a3")
-                                  )
-                              ),
-                              new FieldAccessPostAggregator("p21", "a1")
+                              new FieldAccessPostAggregator("p6", "a1"),
+                              new FieldAccessPostAggregator("p7", "a3")
                           )
                       ),
-                      null
+                      new SketchSetPostAggregator(
+                          "p11",
+                          "UNION",
+                          null,
+                          ImmutableList.of(
+                              new FieldAccessPostAggregator("p9", "a1"),
+                              new FieldAccessPostAggregator("p10", "a3")
+                          )
+                      ),
+                      new SketchSetPostAggregator(
+                          "p14",
+                          "NOT",
+                          null,
+                          ImmutableList.of(
+                              new FieldAccessPostAggregator("p12", "a1"),
+                              new FieldAccessPostAggregator("p13", "a3")
+                          )
+                      ),
+                      new SketchSetPostAggregator(
+                          "p17",
+                          "INTERSECT",
+                          32768,
+                          ImmutableList.of(
+                              new FieldAccessPostAggregator("p15", "a1"),
+                              new FieldAccessPostAggregator("p16", "a3")
+                          )
+                      ),
+                      new SketchEstimatePostAggregator(
+                          "p23",
+                          new SketchSetPostAggregator(
+                              "p22",
+                              "INTERSECT",
+                              null,
+                              ImmutableList.of(
+                                  new SketchSetPostAggregator(
+                                      "p20",
+                                      "INTERSECT",
+                                      null,
+                                      ImmutableList.of(
+                                          new FieldAccessPostAggregator("p18", "a1"),
+                                          new FieldAccessPostAggregator("p19", "a3")
+                                      )
+                                  ),
+                                  new FieldAccessPostAggregator("p21", "a1")
+                              )
+                          ),
+                          null
+                      )
                   )
-              )
-              .context(queryContext)
-              .build()
-              .withOverriddenContext(ImmutableMap.of("skipEmptyBuckets", true));
-
-
-    // Verify query
-    Assert.assertEquals(expectedQuery, actualQuery);
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        expectedResults
+    );
   }
 
   @Test
   public void testThetaSketchPostAggsPostSort() throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-
     final String sql = "SELECT DS_THETA(dim2) as y FROM druid.foo ORDER BY THETA_SKETCH_ESTIMATE(DS_THETA(dim2)) DESC LIMIT 10";
-    final String sql2 = StringUtils.format("SELECT THETA_SKETCH_ESTIMATE(y) from (%s)", sql);
 
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql2,
-        queryContext,
-        DEFAULT_PARAMETERS,
-        authenticationResult
-    ).toList();
     final List<Object[]> expectedResults = ImmutableList.of(
         new Object[]{
             2.0d
         }
     );
 
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
+    testQuery(
+        StringUtils.format("SELECT THETA_SKETCH_ESTIMATE(y) from (%s)", sql),
+        ImmutableList.of(Druids.newTimeseriesQueryBuilder()
+                               .dataSource(CalciteTests.DATASOURCE1)
+                               .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                               .granularity(Granularities.ALL)
+                               .aggregators(
+                                   ImmutableList.of(
+                                       new SketchMergeAggregatorFactory(
+                                           "a0",
+                                           "dim2",
+                                           null,
+                                           null,
+                                           null,
+                                           null
+                                       )
+                                   )
+                               )
+                               .postAggregators(
+                                   new FieldAccessPostAggregator("p0", "a0"),
+                                   new SketchEstimatePostAggregator(
+                                       "p2",
+                                       new FieldAccessPostAggregator("p1", "a0"),
+                                       null
+                                   ),
+                                   new SketchEstimatePostAggregator(
+                                       "s1",
+                                       new FieldAccessPostAggregator("s0", "p0"),
+                                       null
+                                   )
+                               )
+                               .context(QUERY_CONTEXT_DEFAULT)
+                               .build()
 
-    Query actualQuery = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
-
-    Query expectedQuery =
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(
-                  ImmutableList.of(
-                      new SketchMergeAggregatorFactory(
-                          "a0",
-                          "dim2",
-                          null,
-                          null,
-                          null,
-                          null
-                      )
-                  )
-              )
-              .postAggregators(
-                  new FieldAccessPostAggregator("p0", "a0"),
-                  new SketchEstimatePostAggregator(
-                      "p2",
-                      new FieldAccessPostAggregator("p1", "a0"),
-                      null
-                  ),
-                  new SketchEstimatePostAggregator(
-                      "s1",
-                      new FieldAccessPostAggregator("s0", "p0"),
-                      null
-                  )
-              )
-              .context(queryContext)
-              .build()
-              .withOverriddenContext(ImmutableMap.of("skipEmptyBuckets", true));
-
-    // Verify query
-    Assert.assertEquals(expectedQuery, actualQuery);
+        ),
+        expectedResults
+    );
   }
 
-  private void cannotVectorize()
+  @Test
+  public void testEmptyTimeseriesResults() throws Exception
   {
-    if (QueryContexts.Vectorize.fromString((String) queryContext.get(QueryContexts.VECTORIZE_KEY))
-        == QueryContexts.Vectorize.FORCE) {
-      expectedException.expectMessage("Cannot vectorize");
-    }
+    testQuery(
+        "SELECT\n"
+        + "  APPROX_COUNT_DISTINCT_DS_THETA(dim2),\n"
+        + "  APPROX_COUNT_DISTINCT_DS_THETA(thetasketch_dim1),\n"
+        + "  DS_THETA(dim2, 1024),\n"
+        + "  DS_THETA(thetasketch_dim1, 1024)\n"
+        + "FROM druid.foo WHERE dim2 = 0",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
+                  .aggregators(
+                      ImmutableList.of(
+                          new SketchMergeAggregatorFactory(
+                              "a0",
+                              "dim2",
+                              null,
+                              null,
+                              null,
+                              null
+                          ),
+                          new SketchMergeAggregatorFactory(
+                              "a1",
+                              "thetasketch_dim1",
+                              null,
+                              null,
+                              null,
+                              null
+                          ),
+                          new SketchMergeAggregatorFactory(
+                              "a2",
+                              "dim2",
+                              1024,
+                              null,
+                              null,
+                              null
+                          ),
+                          new SketchMergeAggregatorFactory(
+                              "a3",
+                              "thetasketch_dim1",
+                              1024,
+                              null,
+                              null,
+                              null
+                          )
+                      )
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(new Object[]{0L, 0L, "0.0", "0.0"})
+    );
+  }
+
+  @Test
+  public void testGroupByAggregatorDefaultValues() throws Exception
+  {
+    testQuery(
+        "SELECT\n"
+        + "dim2,\n"
+        + "  APPROX_COUNT_DISTINCT_DS_THETA(dim2) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "  APPROX_COUNT_DISTINCT_DS_THETA(thetasketch_dim1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "  DS_THETA(dim2, 1024) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "  DS_THETA(thetasketch_dim1, 1024) FILTER(WHERE dim1 = 'nonexistent')\n"
+        + "FROM foo WHERE dim2 = 'a' GROUP BY dim2",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE1)
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setDimFilter(selector("dim2", "a", null))
+                        .setGranularity(Granularities.ALL)
+                        .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING))
+                        .setDimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING))
+                        .setAggregatorSpecs(
+                            aggregators(
+                                new FilteredAggregatorFactory(
+                                    new SketchMergeAggregatorFactory(
+                                        "a0",
+                                        "v0",
+                                        null,
+                                        null,
+                                        null,
+                                        null
+                                    ),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new SketchMergeAggregatorFactory(
+                                        "a1",
+                                        "thetasketch_dim1",
+                                        null,
+                                        null,
+                                        null,
+                                        null
+                                    ),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new SketchMergeAggregatorFactory(
+                                        "a2",
+                                        "v0",
+                                        1024,
+                                        null,
+                                        null,
+                                        null
+                                    ),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new SketchMergeAggregatorFactory(
+                                        "a3",
+                                        "thetasketch_dim1",
+                                        1024,
+                                        null,
+                                        null,
+                                        null
+                                    ),
+                                    selector("dim1", "nonexistent", null)
+                                )
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(new Object[]{"a", 0L, 0L, "0.0", "0.0"})
+    );
   }
 }
diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java
index 40726e5..effb87d 100644
--- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java
+++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java
@@ -19,15 +19,8 @@
 
 package org.apache.druid.query.aggregation.bloom.sql;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.Key;
-import org.apache.calcite.schema.SchemaPlus;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.data.input.InputRow;
 import org.apache.druid.data.input.impl.DimensionSchema;
@@ -40,122 +33,63 @@
 import org.apache.druid.data.input.impl.TimeAndDimsParseSpec;
 import org.apache.druid.data.input.impl.TimestampSpec;
 import org.apache.druid.guice.BloomFilterSerializersModule;
-import org.apache.druid.guice.annotations.Json;
 import org.apache.druid.java.util.common.granularity.Granularities;
-import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.query.Druids;
-import org.apache.druid.query.QueryRunnerFactoryConglomerate;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
+import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
 import org.apache.druid.query.aggregation.bloom.BloomFilterAggregatorFactory;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.dimension.ExtractionDimensionSpec;
-import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable;
 import org.apache.druid.query.expression.TestExprMacroTable;
 import org.apache.druid.query.extraction.SubstringDimExtractionFn;
 import org.apache.druid.query.filter.BloomKFilter;
-import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
 import org.apache.druid.segment.IndexBuilder;
 import org.apache.druid.segment.QueryableIndex;
-import org.apache.druid.segment.TestHelper;
 import org.apache.druid.segment.column.ValueType;
 import org.apache.druid.segment.incremental.IncrementalIndexSchema;
 import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
 import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
-import org.apache.druid.server.QueryStackTests;
-import org.apache.druid.server.security.AuthTestUtils;
-import org.apache.druid.server.security.AuthenticationResult;
-import org.apache.druid.sql.SqlLifecycle;
-import org.apache.druid.sql.SqlLifecycleFactory;
 import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
 import org.apache.druid.sql.calcite.filtration.Filtration;
 import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
-import org.apache.druid.sql.calcite.planner.PlannerConfig;
-import org.apache.druid.sql.calcite.planner.PlannerFactory;
-import org.apache.druid.sql.calcite.util.CalciteTestBase;
 import org.apache.druid.sql.calcite.util.CalciteTests;
-import org.apache.druid.sql.calcite.util.QueryLogHook;
 import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
-import org.apache.druid.testing.InitializedNullHandlingTest;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.partition.LinearShardSpec;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.List;
 
-public class BloomFilterSqlAggregatorTest extends InitializedNullHandlingTest
+public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
 {
   private static final int TEST_NUM_ENTRIES = 1000;
-  private static AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
-  private static final Injector INJECTOR = Guice.createInjector(
-      binder -> {
-        binder.bind(Key.get(ObjectMapper.class, Json.class)).toInstance(TestHelper.makeJsonMapper());
-        binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(
-            LookupEnabledTestExprMacroTable.createTestLookupProvider(
-                ImmutableMap.of(
-                    "a", "xa",
-                    "abc", "xabc"
-                )
-            )
-        );
-      }
-  );
-
-  private static ObjectMapper jsonMapper =
-      INJECTOR
-          .getInstance(Key.get(ObjectMapper.class, Json.class))
-          .registerModules(Collections.singletonList(new BloomFilterSerializersModule()));
 
   private static final String DATA_SOURCE = "numfoo";
 
-  private static QueryRunnerFactoryConglomerate conglomerate;
-  private static Closer resourceCloser;
+  private static final DruidOperatorTable OPERATOR_TABLE = new DruidOperatorTable(
+      ImmutableSet.of(new BloomFilterSqlAggregator()),
+      ImmutableSet.of()
+  );
 
-  @BeforeClass
-  public static void setUpClass()
+  @Override
+  public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker() throws IOException
   {
-    resourceCloser = Closer.create();
-    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(resourceCloser);
-  }
-
-  @AfterClass
-  public static void tearDownClass() throws IOException
-  {
-    resourceCloser.close();
-  }
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-  @Rule
-  public QueryLogHook queryLogHook = QueryLogHook.create(jsonMapper);
-
-  private SpecificSegmentsQuerySegmentWalker walker;
-  private SqlLifecycleFactory sqlLifecycleFactory;
-
-  @Before
-  public void setUp() throws Exception
-  {
+    CalciteTests.getJsonMapper().registerModule(new BloomFilterSerializersModule());
     InputRowParser parser = new MapInputRowParser(
         new TimeAndDimsParseSpec(
             new TimestampSpec("t", "iso", null),
             new DimensionsSpec(
                 ImmutableList.<DimensionSchema>builder()
-                    .addAll(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3")))
-                    .add(new DoubleDimensionSchema("d1"))
-                    .add(new FloatDimensionSchema("f1"))
-                    .add(new LongDimensionSchema("l1"))
-                    .build(),
+                             .addAll(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3")))
+                             .add(new DoubleDimensionSchema("d1"))
+                             .add(new FloatDimensionSchema("f1"))
+                             .add(new LongDimensionSchema("l1"))
+                             .build(),
                 null,
                 null
             )
@@ -178,7 +112,7 @@
                     .rows(CalciteTests.ROWS1_WITH_NUMERIC_DIMS)
                     .buildMMappedIndex();
 
-    walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
+    return new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
         DataSegment.builder()
                    .dataSource(DATA_SOURCE)
                    .interval(index.getDataInterval())
@@ -188,52 +122,18 @@
                    .build(),
         index
     );
-
-    final PlannerConfig plannerConfig = new PlannerConfig();
-
-    final DruidOperatorTable operatorTable = new DruidOperatorTable(
-        ImmutableSet.of(new BloomFilterSqlAggregator()),
-        ImmutableSet.of()
-    );
-
-    SchemaPlus rootSchema =
-        CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
-    sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
-        new PlannerFactory(
-            rootSchema,
-            CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
-            operatorTable,
-            CalciteTests.createExprMacroTable(),
-            plannerConfig,
-            AuthTestUtils.TEST_AUTHORIZER_MAPPER,
-            jsonMapper,
-            CalciteTests.DRUID_SCHEMA_NAME
-        )
-    );
   }
 
-  @After
-  public void tearDown() throws Exception
+  @Override
+  public DruidOperatorTable createOperatorTable()
   {
-    walker.close();
-    walker = null;
+    return OPERATOR_TABLE;
   }
 
   @Test
   public void testBloomFilterAgg() throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-    final String sql = "SELECT\n"
-                       + "BLOOM_FILTER(dim1, 1000)\n"
-                       + "FROM numfoo";
-
-    final List<Object[]> results =
-        sqlLifecycle.runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            authenticationResult
-        ).toList();
+    cannotVectorize();
 
     BloomKFilter expected1 = new BloomKFilter(TEST_NUM_ENTRIES);
     for (InputRow row : CalciteTests.ROWS1_WITH_NUMERIC_DIMS) {
@@ -245,52 +145,37 @@
       }
     }
 
-    final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{
-            jsonMapper.writeValueAsString(expected1)
-        }
-    );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
-
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE3)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(
-                  ImmutableList.of(
-                      new BloomFilterAggregatorFactory(
-                          "a0:agg",
-                          new DefaultDimensionSpec("dim1", "a0:dim1"),
-                          TEST_NUM_ENTRIES
+    testQuery(
+        "SELECT\n"
+        + "BLOOM_FILTER(dim1, 1000)\n"
+        + "FROM numfoo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(
+                      ImmutableList.of(
+                          new BloomFilterAggregatorFactory(
+                              "a0:agg",
+                              new DefaultDimensionSpec("dim1", "a0:dim1"),
+                              TEST_NUM_ENTRIES
+                          )
                       )
                   )
-              )
-              .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{CalciteTests.getJsonMapper().writeValueAsString(expected1)}
+        )
     );
   }
 
   @Test
   public void testBloomFilterTwoAggs() throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-    final String sql = "SELECT\n"
-                       + "BLOOM_FILTER(dim1, 1000),\n"
-                       + "BLOOM_FILTER(dim2, 1000)\n"
-                       + "FROM numfoo";
-
-    final List<Object[]> results =
-        sqlLifecycle.runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            authenticationResult
-        ).toList();
+    cannotVectorize();
 
     BloomKFilter expected1 = new BloomKFilter(TEST_NUM_ENTRIES);
     BloomKFilter expected2 = new BloomKFilter(TEST_NUM_ENTRIES);
@@ -315,19 +200,12 @@
       }
     }
 
-    final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{
-            jsonMapper.writeValueAsString(expected1),
-            jsonMapper.writeValueAsString(expected2)
-        }
-    );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
-
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
+    testQuery(
+        "SELECT\n"
+        + "BLOOM_FILTER(dim1, 1000),\n"
+        + "BLOOM_FILTER(dim2, 1000)\n"
+        + "FROM numfoo",
+        ImmutableList.of(Druids.newTimeseriesQueryBuilder()
               .dataSource(CalciteTests.DATASOURCE3)
               .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
               .granularity(Granularities.ALL)
@@ -345,27 +223,22 @@
                       )
                   )
               )
-              .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+              .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+              .build()
+        ),
+        ImmutableList.of(
+            new Object[] {
+                CalciteTests.getJsonMapper().writeValueAsString(expected1),
+                CalciteTests.getJsonMapper().writeValueAsString(expected2)
+            }
+        )
     );
   }
 
   @Test
   public void testBloomFilterAggExtractionFn() throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-    final String sql = "SELECT\n"
-                       + "BLOOM_FILTER(SUBSTRING(dim1, 1, 1), 1000)\n"
-                       + "FROM numfoo";
-
-    final List<Object[]> results =
-        sqlLifecycle.runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            authenticationResult
-        ).toList();
+    cannotVectorize();
 
     BloomKFilter expected1 = new BloomKFilter(TEST_NUM_ENTRIES);
     for (InputRow row : CalciteTests.ROWS1_WITH_NUMERIC_DIMS) {
@@ -377,56 +250,43 @@
         expected1.addString(raw.substring(0, 1));
       }
     }
-    final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{
-            jsonMapper.writeValueAsString(expected1)
-        }
-    );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
 
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE3)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(
-                  ImmutableList.of(
-                      new BloomFilterAggregatorFactory(
-                          "a0:agg",
-                          new ExtractionDimensionSpec(
-                              "dim1",
-                              "a0:dim1",
-                              new SubstringDimExtractionFn(0, 1)
-                          ),
-                          TEST_NUM_ENTRIES
+    testQuery(
+        "SELECT\n"
+        + "BLOOM_FILTER(SUBSTRING(dim1, 1, 1), 1000)\n"
+        + "FROM numfoo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(
+                      ImmutableList.of(
+                          new BloomFilterAggregatorFactory(
+                              "a0:agg",
+                              new ExtractionDimensionSpec(
+                                  "dim1",
+                                  "a0:dim1",
+                                  new SubstringDimExtractionFn(0, 1)
+                              ),
+                              TEST_NUM_ENTRIES
+                          )
                       )
                   )
-              )
-              .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{CalciteTests.getJsonMapper().writeValueAsString(expected1)}
+        )
     );
+
   }
 
   @Test
   public void testBloomFilterAggLong() throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-
-    final String sql = "SELECT\n"
-                       + "BLOOM_FILTER(l1, 1000)\n"
-                       + "FROM numfoo";
-
-    final List<Object[]> results =
-        sqlLifecycle.runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            authenticationResult
-        ).toList();
+    cannotVectorize();
 
     BloomKFilter expected3 = new BloomKFilter(TEST_NUM_ENTRIES);
     for (InputRow row : CalciteTests.ROWS1_WITH_NUMERIC_DIMS) {
@@ -441,52 +301,38 @@
         expected3.addLong(((Number) raw).longValue());
       }
     }
-    final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{
-            jsonMapper.writeValueAsString(expected3)
-        }
-    );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
 
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE3)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(
-                  ImmutableList.of(
-                      new BloomFilterAggregatorFactory(
-                          "a0:agg",
-                          new DefaultDimensionSpec("l1", "a0:l1", ValueType.LONG),
-                          TEST_NUM_ENTRIES
+    testQuery(
+        "SELECT\n"
+        + "BLOOM_FILTER(l1, 1000)\n"
+        + "FROM numfoo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(
+                      ImmutableList.of(
+                          new BloomFilterAggregatorFactory(
+                              "a0:agg",
+                              new DefaultDimensionSpec("l1", "a0:l1", ValueType.LONG),
+                              TEST_NUM_ENTRIES
+                          )
                       )
                   )
-              )
-              .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{CalciteTests.getJsonMapper().writeValueAsString(expected3)}
+        )
     );
   }
 
   @Test
   public void testBloomFilterAggLongVirtualColumn() throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-    final String sql = "SELECT\n"
-                       + "BLOOM_FILTER(l1 * 2, 1000)\n"
-                       + "FROM numfoo";
-
-    final List<Object[]> results =
-        sqlLifecycle.runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            authenticationResult
-        ).toList();
-
+    cannotVectorize();
     BloomKFilter expected1 = new BloomKFilter(TEST_NUM_ENTRIES);
     for (InputRow row : CalciteTests.ROWS1_WITH_NUMERIC_DIMS) {
       Object raw = row.getRaw("l1");
@@ -500,59 +346,46 @@
         expected1.addLong(2 * ((Number) raw).longValue());
       }
     }
-    final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{
-            jsonMapper.writeValueAsString(expected1)
-        }
-    );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
 
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE3)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .virtualColumns(
-                  new ExpressionVirtualColumn(
-                      "v0",
-                      "(\"l1\" * 2)",
-                      ValueType.LONG,
-                      TestExprMacroTable.INSTANCE
-                  )
-              )
-              .aggregators(
-                  ImmutableList.of(
-                      new BloomFilterAggregatorFactory(
-                          "a0:agg",
-                          new DefaultDimensionSpec("v0", "a0:v0"),
-                          TEST_NUM_ENTRIES
+    testQuery(
+        "SELECT\n"
+        + "BLOOM_FILTER(l1 * 2, 1000)\n"
+        + "FROM numfoo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .virtualColumns(
+                      new ExpressionVirtualColumn(
+                          "v0",
+                          "(\"l1\" * 2)",
+                          ValueType.LONG,
+                          TestExprMacroTable.INSTANCE
                       )
                   )
-              )
-              .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .aggregators(
+                      ImmutableList.of(
+                          new BloomFilterAggregatorFactory(
+                              "a0:agg",
+                              new DefaultDimensionSpec("v0", "a0:v0"),
+                              TEST_NUM_ENTRIES
+                          )
+                      )
+                  )
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{CalciteTests.getJsonMapper().writeValueAsString(expected1)}
+        )
     );
   }
 
   @Test
   public void testBloomFilterAggFloatVirtualColumn() throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-    final String sql = "SELECT\n"
-                       + "BLOOM_FILTER(f1 * 2, 1000)\n"
-                       + "FROM numfoo";
-
-    final List<Object[]> results =
-        sqlLifecycle.runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            authenticationResult
-        ).toList();
+    cannotVectorize();
 
     BloomKFilter expected1 = new BloomKFilter(TEST_NUM_ENTRIES);
     for (InputRow row : CalciteTests.ROWS1_WITH_NUMERIC_DIMS) {
@@ -567,60 +400,46 @@
         expected1.addFloat(2 * ((Number) raw).floatValue());
       }
     }
-    final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{
-            jsonMapper.writeValueAsString(expected1)
-        }
-    );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
 
-    // Verify query
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE3)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .virtualColumns(
-                  new ExpressionVirtualColumn(
-                      "v0",
-                      "(\"f1\" * 2)",
-                      ValueType.FLOAT,
-                      TestExprMacroTable.INSTANCE
-                  )
-              )
-              .aggregators(
-                  ImmutableList.of(
-                      new BloomFilterAggregatorFactory(
-                          "a0:agg",
-                          new DefaultDimensionSpec("v0", "a0:v0"),
-                          TEST_NUM_ENTRIES
+    testQuery(
+        "SELECT\n"
+        + "BLOOM_FILTER(f1 * 2, 1000)\n"
+        + "FROM numfoo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .virtualColumns(
+                      new ExpressionVirtualColumn(
+                          "v0",
+                          "(\"f1\" * 2)",
+                          ValueType.FLOAT,
+                          TestExprMacroTable.INSTANCE
                       )
                   )
-              )
-              .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .aggregators(
+                      ImmutableList.of(
+                          new BloomFilterAggregatorFactory(
+                              "a0:agg",
+                              new DefaultDimensionSpec("v0", "a0:v0"),
+                              TEST_NUM_ENTRIES
+                          )
+                      )
+                  )
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{CalciteTests.getJsonMapper().writeValueAsString(expected1)}
+        )
     );
   }
 
   @Test
   public void testBloomFilterAggDoubleVirtualColumn() throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-    final String sql = "SELECT\n"
-                       + "BLOOM_FILTER(d1 * 2, 1000)\n"
-                       + "FROM numfoo";
-
-    final List<Object[]> results =
-        sqlLifecycle.runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            authenticationResult
-        ).toList();
+    cannotVectorize();
 
     BloomKFilter expected1 = new BloomKFilter(TEST_NUM_ENTRIES);
     for (InputRow row : CalciteTests.ROWS1_WITH_NUMERIC_DIMS) {
@@ -635,42 +454,141 @@
         expected1.addDouble(2 * ((Number) raw).doubleValue());
       }
     }
-    final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{
-            jsonMapper.writeValueAsString(expected1)
-        }
-    );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
 
-    // Verify query
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE3)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .virtualColumns(
-                  new ExpressionVirtualColumn(
-                      "v0",
-                      "(\"d1\" * 2)",
-                      ValueType.DOUBLE,
-                      TestExprMacroTable.INSTANCE
-                  )
-              )
-              .aggregators(
-                  ImmutableList.of(
-                      new BloomFilterAggregatorFactory(
-                          "a0:agg",
-                          new DefaultDimensionSpec("v0", "a0:v0"),
-                          TEST_NUM_ENTRIES
+    testQuery(
+        "SELECT\n"
+        + "BLOOM_FILTER(d1 * 2, 1000)\n"
+        + "FROM numfoo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .virtualColumns(
+                      new ExpressionVirtualColumn(
+                          "v0",
+                          "(\"d1\" * 2)",
+                          ValueType.DOUBLE,
+                          TestExprMacroTable.INSTANCE
                       )
                   )
-              )
-              .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .aggregators(
+                      ImmutableList.of(
+                          new BloomFilterAggregatorFactory(
+                              "a0:agg",
+                              new DefaultDimensionSpec("v0", "a0:v0"),
+                              TEST_NUM_ENTRIES
+                          )
+                      )
+                  )
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{CalciteTests.getJsonMapper().writeValueAsString(expected1)}
+        )
+    );
+  }
+
+  @Test
+  public void testEmptyTimeseriesResults() throws Exception
+  {
+    // makes empty bloom filters
+    cannotVectorize();
+
+    BloomKFilter expected1 = new BloomKFilter(TEST_NUM_ENTRIES);
+    BloomKFilter expected2 = new BloomKFilter(TEST_NUM_ENTRIES);
+
+    testQuery(
+        "SELECT\n"
+        + "BLOOM_FILTER(dim1, 1000),\n"
+        + "BLOOM_FILTER(l1, 1000)\n"
+        + "FROM numfoo where dim2 = 0",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .filters(BaseCalciteQueryTest.bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
+                  .aggregators(
+                      ImmutableList.of(
+                          new BloomFilterAggregatorFactory(
+                              "a0:agg",
+                              new DefaultDimensionSpec("dim1", "a0:dim1"),
+                              TEST_NUM_ENTRIES
+                          ),
+                          new BloomFilterAggregatorFactory(
+                              "a1:agg",
+                              new DefaultDimensionSpec("l1", "a1:l1", ValueType.LONG),
+                              TEST_NUM_ENTRIES
+                          )
+                      )
+                  )
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[] {
+                CalciteTests.getJsonMapper().writeValueAsString(expected1),
+                CalciteTests.getJsonMapper().writeValueAsString(expected2)
+            }
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByAggregatorDefaultValues() throws Exception
+  {
+    // makes empty bloom filters
+    cannotVectorize();
+
+    BloomKFilter expected1 = new BloomKFilter(TEST_NUM_ENTRIES);
+    BloomKFilter expected2 = new BloomKFilter(TEST_NUM_ENTRIES);
+
+    testQuery(
+        "SELECT\n"
+        + "dim2,\n"
+        + "BLOOM_FILTER(dim1, 1000) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "BLOOM_FILTER(l1, 1000) FILTER(WHERE dim1 = 'nonexistent')\n"
+        + "FROM numfoo WHERE dim2 = 'a' GROUP BY dim2",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE3)
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setDimFilter(selector("dim2", "a", null))
+                        .setGranularity(Granularities.ALL)
+                        .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING))
+                        .setDimensions(new DefaultDimensionSpec("v0", "_d0", ValueType.STRING))
+                        .setAggregatorSpecs(
+                            aggregators(
+                                new FilteredAggregatorFactory(
+                                    new BloomFilterAggregatorFactory(
+                                        "a0:agg",
+                                        new DefaultDimensionSpec("dim1", "a0:dim1"),
+                                        TEST_NUM_ENTRIES
+                                    ),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new BloomFilterAggregatorFactory(
+                                        "a1:agg",
+                                        new DefaultDimensionSpec("l1", "a1:l1", ValueType.LONG),
+                                        TEST_NUM_ENTRIES
+                                    ),
+                                    selector("dim1", "nonexistent", null)
+                                )
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[] {
+                "a",
+                CalciteTests.getJsonMapper().writeValueAsString(expected1),
+                CalciteTests.getJsonMapper().writeValueAsString(expected2)
+            }
+        )
     );
   }
 }
diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java
index 2377dc3..24e3b6b 100644
--- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java
+++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java
@@ -19,25 +19,17 @@
 
 package org.apache.druid.query.filter.sql;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.Key;
 import org.apache.calcite.avatica.SqlType;
 import org.apache.druid.common.config.NullHandling;
-import org.apache.druid.guice.BloomFilterExtensionModule;
 import org.apache.druid.guice.BloomFilterSerializersModule;
 import org.apache.druid.guice.ExpressionModule;
-import org.apache.druid.guice.annotations.Json;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.math.expr.ExprMacroTable;
 import org.apache.druid.query.Druids;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
-import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable;
 import org.apache.druid.query.expression.LookupExprMacro;
 import org.apache.druid.query.expressions.BloomFilterExprMacro;
 import org.apache.druid.query.filter.BloomDimFilter;
@@ -45,66 +37,42 @@
 import org.apache.druid.query.filter.BloomKFilterHolder;
 import org.apache.druid.query.filter.ExpressionDimFilter;
 import org.apache.druid.query.filter.OrDimFilter;
-import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
-import org.apache.druid.segment.TestHelper;
 import org.apache.druid.segment.column.ValueType;
-import org.apache.druid.server.security.AuthenticationResult;
 import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
 import org.apache.druid.sql.calcite.filtration.Filtration;
 import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
-import org.apache.druid.sql.calcite.planner.PlannerConfig;
 import org.apache.druid.sql.calcite.util.CalciteTests;
-import org.apache.druid.sql.calcite.util.QueryLogHook;
 import org.apache.druid.sql.http.SqlParameter;
 import org.junit.Ignore;
-import org.junit.Rule;
 import org.junit.Test;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 
 public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
 {
-  private static final Injector INJECTOR = Guice.createInjector(
-      binder -> {
-        binder.bind(Key.get(ObjectMapper.class, Json.class)).toInstance(TestHelper.makeJsonMapper());
-        binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(
-            LookupEnabledTestExprMacroTable.createTestLookupProvider(
-                ImmutableMap.of(
-                    "a", "xa",
-                    "abc", "xabc"
-                )
-            )
-        );
-      },
-      new BloomFilterExtensionModule()
-  );
+  @Override
+  public DruidOperatorTable createOperatorTable()
+  {
+    CalciteTests.getJsonMapper().registerModule(new BloomFilterSerializersModule());
+    return new DruidOperatorTable(
+        ImmutableSet.of(),
+        ImmutableSet.of(new BloomFilterOperatorConversion())
+    );
+  }
 
-  private static ObjectMapper jsonMapper =
-      INJECTOR
-          .getInstance(Key.get(ObjectMapper.class, Json.class))
-          .registerModules(Collections.singletonList(new BloomFilterSerializersModule()));
-
-  public static ExprMacroTable createExprMacroTable()
+  @Override
+  public ExprMacroTable createMacroTable()
   {
     final List<ExprMacroTable.ExprMacro> exprMacros = new ArrayList<>();
     for (Class<? extends ExprMacroTable.ExprMacro> clazz : ExpressionModule.EXPR_MACROS) {
-      exprMacros.add(INJECTOR.getInstance(clazz));
+      exprMacros.add(CalciteTests.INJECTOR.getInstance(clazz));
     }
-    exprMacros.add(INJECTOR.getInstance(BloomFilterExprMacro.class));
-    exprMacros.add(INJECTOR.getInstance(LookupExprMacro.class));
+    exprMacros.add(CalciteTests.INJECTOR.getInstance(LookupExprMacro.class));
+    exprMacros.add(new BloomFilterExprMacro());
     return new ExprMacroTable(exprMacros);
   }
 
-  @Rule
-  @Override
-  public QueryLogHook getQueryLogHook()
-  {
-    return queryLogHook = QueryLogHook.create(jsonMapper);
-  }
-
   @Test
   public void testBloomFilter() throws Exception
   {
@@ -124,7 +92,7 @@
                       new BloomDimFilter("dim1", BloomKFilterHolder.fromBloomKFilter(filter), null)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -161,11 +129,11 @@
                               base64
                           ),
                           null,
-                          createExprMacroTable()
+                          createMacroTable()
                       )
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -194,7 +162,7 @@
                       new BloomDimFilter("v0", BloomKFilterHolder.fromBloomKFilter(filter), null)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -226,7 +194,7 @@
                       new BloomDimFilter("v0", BloomKFilterHolder.fromBloomKFilter(filter), null)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -261,7 +229,7 @@
                       )
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -289,7 +257,7 @@
                       new BloomDimFilter("dim1", BloomKFilterHolder.fromBloomKFilter(filter), null)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -317,7 +285,7 @@
                       new BloomDimFilter("dim1", BloomKFilterHolder.fromBloomKFilter(filter), null)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -345,7 +313,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -355,30 +323,4 @@
         ImmutableList.of(new SqlParameter(SqlType.VARCHAR, NullHandling.defaultStringValue()), new SqlParameter(SqlType.VARCHAR, base64))
     );
   }
-
-  @Override
-  public List<Object[]> getResults(
-      final PlannerConfig plannerConfig,
-      final Map<String, Object> queryContext,
-      final List<SqlParameter> parameters,
-      final String sql,
-      final AuthenticationResult authenticationResult
-  ) throws Exception
-  {
-    final DruidOperatorTable operatorTable = new DruidOperatorTable(
-        ImmutableSet.of(),
-        ImmutableSet.of(INJECTOR.getInstance(BloomFilterOperatorConversion.class))
-    );
-    return getResults(
-        plannerConfig,
-        queryContext,
-        parameters,
-        sql,
-        authenticationResult,
-        operatorTable,
-        createExprMacroTable(),
-        CalciteTests.TEST_AUTHORIZER_MAPPER,
-        jsonMapper
-    );
-  }
 }
diff --git a/extensions-core/histogram/pom.xml b/extensions-core/histogram/pom.xml
index 28cde2e..b9f4e0f 100644
--- a/extensions-core/histogram/pom.xml
+++ b/extensions-core/histogram/pom.xml
@@ -147,6 +147,11 @@
             <artifactId>equalsverifier</artifactId>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>joda-time</groupId>
+            <artifactId>joda-time</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
 
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java
index dd41d75..948d63d 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java
@@ -23,11 +23,9 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.query.Druids;
-import org.apache.druid.query.Query;
 import org.apache.druid.query.QueryDataSource;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
@@ -43,6 +41,7 @@
 import org.apache.druid.query.filter.NotDimFilter;
 import org.apache.druid.query.filter.SelectorDimFilter;
 import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
 import org.apache.druid.segment.IndexBuilder;
 import org.apache.druid.segment.QueryableIndex;
@@ -50,31 +49,21 @@
 import org.apache.druid.segment.incremental.IncrementalIndexSchema;
 import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
 import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
-import org.apache.druid.server.security.AuthenticationResult;
-import org.apache.druid.sql.SqlLifecycle;
 import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
 import org.apache.druid.sql.calcite.filtration.Filtration;
 import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
-import org.apache.druid.sql.calcite.planner.PlannerConfig;
 import org.apache.druid.sql.calcite.planner.PlannerContext;
 import org.apache.druid.sql.calcite.util.CalciteTests;
 import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
-import org.apache.druid.sql.http.SqlParameter;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.partition.LinearShardSpec;
-import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
 public class FixedBucketsHistogramQuantileSqlAggregatorTest extends BaseCalciteQueryTest
 {
-  private static final AuthenticationResult AUTH_RESULT = CalciteTests.REGULAR_USER_AUTH_RESULT;
-  private static final Map<String, Object> QUERY_CONTEXT_DEFAULT = ImmutableMap.of(
-      PlannerContext.CTX_SQL_QUERY_ID, "dummy"
-  );
   private static final DruidOperatorTable OPERATOR_TABLE = new DruidOperatorTable(
       ImmutableSet.of(new QuantileSqlAggregator(), new FixedBucketsHistogramQuantileSqlAggregator()),
       ImmutableSet.of()
@@ -125,61 +114,17 @@
   }
 
   @Override
-  public List<Object[]> getResults(
-      final PlannerConfig plannerConfig,
-      final Map<String, Object> queryContext,
-      final List<SqlParameter> parameters,
-      final String sql,
-      final AuthenticationResult authenticationResult
-  ) throws Exception
+  public DruidOperatorTable createOperatorTable()
   {
-    return getResults(
-        plannerConfig,
-        queryContext,
-        parameters,
-        sql,
-        authenticationResult,
-        OPERATOR_TABLE,
-        CalciteTests.createExprMacroTable(),
-        CalciteTests.TEST_AUTHORIZER_MAPPER,
-        CalciteTests.getJsonMapper()
-    );
+    return OPERATOR_TABLE;
   }
 
-  private SqlLifecycle getSqlLifecycle()
-  {
-    return getSqlLifecycleFactory(
-        BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT,
-        OPERATOR_TABLE,
-        CalciteTests.createExprMacroTable(),
-        CalciteTests.TEST_AUTHORIZER_MAPPER,
-        CalciteTests.getJsonMapper()
-    ).factorize();
-  }
 
   @Test
   public void testQuantileOnFloatAndLongs() throws Exception
   {
-    SqlLifecycle sqlLifecycle = getSqlLifecycle();
-    final String sql = "SELECT\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.01, 20, 0.0, 10.0),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.5, 20, 0.0, 10.0),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.98, 20, 0.0, 10.0),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.99, 20, 0.0, 10.0),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(m1 * 2, 0.97, 40, 0.0, 20.0),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.99, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc'),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 <> 'abc'),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc'),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(cnt, 0.5, 20, 0.0, 10.0)\n"
-                       + "FROM foo";
+    cannotVectorize();
 
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        TIMESERIES_CONTEXT_DEFAULT,
-        DEFAULT_PARAMETERS,
-        AUTH_RESULT
-    ).toList();
     final List<Object[]> expectedResults = ImmutableList.of(
         new Object[]{
             1.0299999713897705,
@@ -193,100 +138,105 @@
             1.25
         }
     );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
 
-    Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
-    Query expected = Druids.newTimeseriesQueryBuilder()
-                           .dataSource(CalciteTests.DATASOURCE1)
-                           .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-                           .granularity(Granularities.ALL)
-                           .virtualColumns(
-                               new ExpressionVirtualColumn(
-                                   "v0",
-                                   "(\"m1\" * 2)",
-                                   ValueType.FLOAT,
-                                   TestExprMacroTable.INSTANCE
-                               )
-                           )
-                           .aggregators(ImmutableList.of(
-                               new FixedBucketsHistogramAggregatorFactory(
-                                   "a0:agg",
-                                   "m1",
-                                   20,
-                                   0.0d,
-                                   10.0d,
-                                   FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
-                                   false
-                               ),
-                               new FixedBucketsHistogramAggregatorFactory(
-                                   "a4:agg",
-                                   "v0",
-                                   40,
-                                   0.0d,
-                                   20.0d,
-                                   FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
-                                   false
-                               ),
-                               new FilteredAggregatorFactory(
-                                   new FixedBucketsHistogramAggregatorFactory(
-                                       "a5:agg",
-                                       "m1",
-                                       20,
-                                       0.0d,
-                                       10.0d,
-                                       FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
-                                       false
-                                   ),
-                                   new SelectorDimFilter("dim1", "abc", null)
-                               ),
-                               new FilteredAggregatorFactory(
-                                   new FixedBucketsHistogramAggregatorFactory(
-                                       "a6:agg",
-                                       "m1",
-                                       20,
-                                       0.0d,
-                                       10.0d,
-                                       FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
-                                       false
-                                   ),
-                                   new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
-                               ),
-                               new FixedBucketsHistogramAggregatorFactory(
-                                   "a8:agg",
-                                   "cnt",
-                                   20,
-                                   0.0d,
-                                   10.0d,
-                                   FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
-                                   false
-                               )
-                           ))
-                           .postAggregators(
-                               new QuantilePostAggregator("a0", "a0:agg", 0.01f),
-                               new QuantilePostAggregator("a1", "a0:agg", 0.50f),
-                               new QuantilePostAggregator("a2", "a0:agg", 0.98f),
-                               new QuantilePostAggregator("a3", "a0:agg", 0.99f),
-                               new QuantilePostAggregator("a4", "a4:agg", 0.97f),
-                               new QuantilePostAggregator("a5", "a5:agg", 0.99f),
-                               new QuantilePostAggregator("a6", "a6:agg", 0.999f),
-                               new QuantilePostAggregator("a7", "a5:agg", 0.999f),
-                               new QuantilePostAggregator("a8", "a8:agg", 0.50f)
-                           )
-                           .context(TIMESERIES_CONTEXT_DEFAULT)
-                           .build();
-
-    // Verify query
-    Assert.assertEquals(
-        expected,
-        actual
+    testQuery(
+        "SELECT\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.01, 20, 0.0, 10.0),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.5, 20, 0.0, 10.0),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.98, 20, 0.0, 10.0),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.99, 20, 0.0, 10.0),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(m1 * 2, 0.97, 40, 0.0, 20.0),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.99, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc'),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 <> 'abc'),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc'),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(cnt, 0.5, 20, 0.0, 10.0)\n"
+        + "FROM foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .virtualColumns(
+                      new ExpressionVirtualColumn(
+                          "v0",
+                          "(\"m1\" * 2)",
+                          ValueType.FLOAT,
+                          TestExprMacroTable.INSTANCE
+                      )
+                  )
+                  .aggregators(ImmutableList.of(
+                      new FixedBucketsHistogramAggregatorFactory(
+                          "a0:agg",
+                          "m1",
+                          20,
+                          0.0d,
+                          10.0d,
+                          FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                          false
+                      ),
+                      new FixedBucketsHistogramAggregatorFactory(
+                          "a4:agg",
+                          "v0",
+                          40,
+                          0.0d,
+                          20.0d,
+                          FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                          false
+                      ),
+                      new FilteredAggregatorFactory(
+                          new FixedBucketsHistogramAggregatorFactory(
+                              "a5:agg",
+                              "m1",
+                              20,
+                              0.0d,
+                              10.0d,
+                              FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                              false
+                          ),
+                          new SelectorDimFilter("dim1", "abc", null)
+                      ),
+                      new FilteredAggregatorFactory(
+                          new FixedBucketsHistogramAggregatorFactory(
+                              "a6:agg",
+                              "m1",
+                              20,
+                              0.0d,
+                              10.0d,
+                              FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                              false
+                          ),
+                          new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
+                      ),
+                      new FixedBucketsHistogramAggregatorFactory(
+                          "a8:agg",
+                          "cnt",
+                          20,
+                          0.0d,
+                          10.0d,
+                          FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                          false
+                      )
+                  ))
+                  .postAggregators(
+                      new QuantilePostAggregator("a0", "a0:agg", 0.01f),
+                      new QuantilePostAggregator("a1", "a0:agg", 0.50f),
+                      new QuantilePostAggregator("a2", "a0:agg", 0.98f),
+                      new QuantilePostAggregator("a3", "a0:agg", 0.99f),
+                      new QuantilePostAggregator("a4", "a4:agg", 0.97f),
+                      new QuantilePostAggregator("a5", "a5:agg", 0.99f),
+                      new QuantilePostAggregator("a6", "a6:agg", 0.999f),
+                      new QuantilePostAggregator("a7", "a5:agg", 0.999f),
+                      new QuantilePostAggregator("a8", "a8:agg", 0.50f)
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        expectedResults
     );
   }
 
   @Test
-  public void testQuentileOnCastedString() throws Exception
+  public void testQuantileOnCastedString() throws Exception
   {
     cannotVectorize();
 
@@ -374,7 +324,7 @@
                       new QuantilePostAggregator("a6", "a6:agg", 0.999f),
                       new QuantilePostAggregator("a7", "a5:agg", 0.999f)
                   )
-                  .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
+                  .context(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
                   .build()
         ),
         ImmutableList.of(
@@ -406,24 +356,8 @@
   @Test
   public void testQuantileOnComplexColumn() throws Exception
   {
-    SqlLifecycle lifecycle = getSqlLifecycle();
-    final String sql = "SELECT\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.01, 20, 0.0, 10.0),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.5, 20, 0.0, 10.0),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.98, 30, 0.0, 10.0),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.99, 20, 0.0, 10.0),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.99, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc'),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 <> 'abc'),\n"
-                       + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc')\n"
-                       + "FROM foo";
+    cannotVectorize();
 
-    // Verify results
-    final List<Object[]> results = lifecycle.runSimple(
-        sql,
-        TIMESERIES_CONTEXT_DEFAULT,
-        DEFAULT_PARAMETERS,
-        AUTH_RESULT
-    ).toList();
     final List<Object[]> expectedResults = ImmutableList.of(
         new Object[]{
             1.0299999713897705,
@@ -435,152 +369,258 @@
             6.499499797821045
         }
     );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
 
-    Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
-    Query expected = Druids.newTimeseriesQueryBuilder()
-                           .dataSource(CalciteTests.DATASOURCE1)
-                           .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-                           .granularity(Granularities.ALL)
-                           .aggregators(ImmutableList.of(
-                               new FixedBucketsHistogramAggregatorFactory(
-                                   "a0:agg",
-                                   "fbhist_m1",
-                                   20,
-                                   0.0,
-                                   10.0,
-                                   FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
-                                   false
-                               ),
-                               new FixedBucketsHistogramAggregatorFactory(
-                                   "a2:agg",
-                                   "fbhist_m1",
-                                   30,
-                                   0.0,
-                                   10.0,
-                                   FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
-                                   false
-                               ),
-                               new FilteredAggregatorFactory(
-                                   new FixedBucketsHistogramAggregatorFactory(
-                                       "a4:agg",
-                                       "fbhist_m1",
-                                       20,
-                                       0.0,
-                                       10.0,
-                                       FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
-                                       false
-                                   ),
-                                   new SelectorDimFilter("dim1", "abc", null)
-                               ),
-                               new FilteredAggregatorFactory(
-                                   new FixedBucketsHistogramAggregatorFactory(
-                                       "a5:agg",
-                                       "fbhist_m1",
-                                       20,
-                                       0.0,
-                                       10.0,
-                                       FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
-                                       false
-                                   ),
-                                   new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
-                               )
-                           ))
-                           .postAggregators(
-                               new QuantilePostAggregator("a0", "a0:agg", 0.01f),
-                               new QuantilePostAggregator("a1", "a0:agg", 0.50f),
-                               new QuantilePostAggregator("a2", "a2:agg", 0.98f),
-                               new QuantilePostAggregator("a3", "a0:agg", 0.99f),
-                               new QuantilePostAggregator("a4", "a4:agg", 0.99f),
-                               new QuantilePostAggregator("a5", "a5:agg", 0.999f),
-                               new QuantilePostAggregator("a6", "a4:agg", 0.999f)
-                           )
-                           .context(TIMESERIES_CONTEXT_DEFAULT)
-                           .build();
-
-    // Verify query
-    Assert.assertEquals(expected, actual);
+    testQuery(
+        "SELECT\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.01, 20, 0.0, 10.0),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.5, 20, 0.0, 10.0),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.98, 30, 0.0, 10.0),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.99, 20, 0.0, 10.0),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.99, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc'),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 <> 'abc'),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc')\n"
+        + "FROM foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(ImmutableList.of(
+                      new FixedBucketsHistogramAggregatorFactory(
+                          "a0:agg",
+                          "fbhist_m1",
+                          20,
+                          0.0,
+                          10.0,
+                          FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                          false
+                      ),
+                      new FixedBucketsHistogramAggregatorFactory(
+                          "a2:agg",
+                          "fbhist_m1",
+                          30,
+                          0.0,
+                          10.0,
+                          FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                          false
+                      ),
+                      new FilteredAggregatorFactory(
+                          new FixedBucketsHistogramAggregatorFactory(
+                              "a4:agg",
+                              "fbhist_m1",
+                              20,
+                              0.0,
+                              10.0,
+                              FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                              false
+                          ),
+                          new SelectorDimFilter("dim1", "abc", null)
+                      ),
+                      new FilteredAggregatorFactory(
+                          new FixedBucketsHistogramAggregatorFactory(
+                              "a5:agg",
+                              "fbhist_m1",
+                              20,
+                              0.0,
+                              10.0,
+                              FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                              false
+                          ),
+                          new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
+                      )
+                  ))
+                  .postAggregators(
+                      new QuantilePostAggregator("a0", "a0:agg", 0.01f),
+                      new QuantilePostAggregator("a1", "a0:agg", 0.50f),
+                      new QuantilePostAggregator("a2", "a2:agg", 0.98f),
+                      new QuantilePostAggregator("a3", "a0:agg", 0.99f),
+                      new QuantilePostAggregator("a4", "a4:agg", 0.99f),
+                      new QuantilePostAggregator("a5", "a5:agg", 0.999f),
+                      new QuantilePostAggregator("a6", "a4:agg", 0.999f)
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        expectedResults
+    );
   }
 
   @Test
   public void testQuantileOnInnerQuery() throws Exception
   {
-    SqlLifecycle sqlLifecycle = getSqlLifecycle();
-    final String sql = "SELECT AVG(x), APPROX_QUANTILE_FIXED_BUCKETS(x, 0.98, 100, 0.0, 100.0)\n"
-                       + "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)";
-
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        QUERY_CONTEXT_DEFAULT,
-        DEFAULT_PARAMETERS,
-        AUTH_RESULT
-    ).toList();
     final List<Object[]> expectedResults;
     if (NullHandling.replaceWithDefault()) {
       expectedResults = ImmutableList.of(new Object[]{7.0, 11.940000534057617});
     } else {
       expectedResults = ImmutableList.of(new Object[]{5.25, 8.920000076293945});
     }
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
 
-    Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
-    Query expected = GroupByQuery.builder()
-                                 .setDataSource(
-                                     new QueryDataSource(
-                                         GroupByQuery.builder()
-                                                     .setDataSource(CalciteTests.DATASOURCE1)
-                                                     .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(
-                                                         Filtration.eternity())))
-                                                     .setGranularity(Granularities.ALL)
-                                                     .setDimensions(new DefaultDimensionSpec("dim2", "d0"))
-                                                     .setAggregatorSpecs(
-                                                         ImmutableList.of(
-                                                             new DoubleSumAggregatorFactory("a0", "m1")
-                                                         )
-                                                     )
-                                                     .setContext(QUERY_CONTEXT_DEFAULT)
-                                                     .build()
-                                     )
-                                 )
-                                 .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-                                 .setGranularity(Granularities.ALL)
-                                 .setAggregatorSpecs(
-                                     new DoubleSumAggregatorFactory("_a0:sum", "a0"),
-                                     new CountAggregatorFactory("_a0:count"),
-                                     new FixedBucketsHistogramAggregatorFactory(
-                                         "_a1:agg",
-                                         "a0",
-                                         100,
-                                         0,
-                                         100.0d,
-                                         FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
-                                         false
-                                     )
-                                 )
-                                 .setPostAggregatorSpecs(
-                                     ImmutableList.of(
-                                         new ArithmeticPostAggregator(
-                                             "_a0",
-                                             "quotient",
-                                             ImmutableList.of(
-                                                 new FieldAccessPostAggregator(null, "_a0:sum"),
-                                                 new FieldAccessPostAggregator(null, "_a0:count")
-                                             )
-                                         ),
-                                         new QuantilePostAggregator("_a1", "_a1:agg", 0.98f)
-                                     )
-                                 )
-                                 .setContext(QUERY_CONTEXT_DEFAULT)
-                                 .build();
+    testQuery(
+        "SELECT AVG(x), APPROX_QUANTILE_FIXED_BUCKETS(x, 0.98, 100, 0.0, 100.0)\n"
+        + "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                GroupByQuery.builder()
+                                            .setDataSource(CalciteTests.DATASOURCE1)
+                                            .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(
+                                                Filtration.eternity())))
+                                            .setGranularity(Granularities.ALL)
+                                            .setDimensions(new DefaultDimensionSpec("dim2", "d0"))
+                                            .setAggregatorSpecs(
+                                                ImmutableList.of(
+                                                    new DoubleSumAggregatorFactory("a0", "m1")
+                                                )
+                                            )
+                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                            .build()
+                            )
+                        )
+                        .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(
+                            new DoubleSumAggregatorFactory("_a0:sum", "a0"),
+                            new CountAggregatorFactory("_a0:count"),
+                            new FixedBucketsHistogramAggregatorFactory(
+                                "_a1:agg",
+                                "a0",
+                                100,
+                                0,
+                                100.0d,
+                                FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                                false
+                            )
+                        )
+                        .setPostAggregatorSpecs(
+                            ImmutableList.of(
+                                new ArithmeticPostAggregator(
+                                    "_a0",
+                                    "quotient",
+                                    ImmutableList.of(
+                                        new FieldAccessPostAggregator(null, "_a0:sum"),
+                                        new FieldAccessPostAggregator(null, "_a0:count")
+                                    )
+                                ),
+                                new QuantilePostAggregator("_a1", "_a1:agg", 0.98f)
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        expectedResults
+    );
+  }
 
-    // Verify query
-    Assert.assertEquals(expected, actual);
+  @Test
+  public void testEmptyTimeseriesResults() throws Exception
+  {
+    cannotVectorize();
+
+    testQuery(
+        "SELECT\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.01, 20, 0.0, 10.0),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.01, 20, 0.0, 10.0)\n"
+        + "FROM foo WHERE dim2 = 0",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
+                  .aggregators(ImmutableList.of(
+                      new FixedBucketsHistogramAggregatorFactory(
+                          "a0:agg",
+                          "fbhist_m1",
+                          20,
+                          0.0,
+                          10.0,
+                          FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                          false
+                      ),
+                      new FixedBucketsHistogramAggregatorFactory(
+                          "a1:agg",
+                          "m1",
+                          20,
+                          0.0,
+                          10.0,
+                          FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                          false
+                      )
+
+                  ))
+                  .postAggregators(
+                      new QuantilePostAggregator("a0", "a0:agg", 0.01f),
+                      new QuantilePostAggregator("a1", "a1:agg", 0.01f)
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{0.0, 0.0}
+        )
+    );
+  }
+
+
+  @Test
+  public void testGroupByAggregatorDefaultValues() throws Exception
+  {
+    cannotVectorize();
+    testQuery(
+        "SELECT\n"
+        + "dim2,\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.01, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.01, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'nonexistent')\n"
+        + "FROM foo WHERE dim2 = 'a' GROUP BY dim2",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE1)
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setDimFilter(selector("dim2", "a", null))
+                        .setGranularity(Granularities.ALL)
+                        .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING))
+                        .setDimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING))
+                        .setAggregatorSpecs(
+                            aggregators(
+                                new FilteredAggregatorFactory(
+                                    new FixedBucketsHistogramAggregatorFactory(
+                                        "a0:agg",
+                                        "fbhist_m1",
+                                        20,
+                                        0.0,
+                                        10.0,
+                                        FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                                        false
+                                    ),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new FixedBucketsHistogramAggregatorFactory(
+                                        "a1:agg",
+                                        "m1",
+                                        20,
+                                        0.0,
+                                        10.0,
+                                        FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
+                                        false
+                                    ),
+                                    selector("dim1", "nonexistent", null)
+                                )
+                            )
+                        )
+                        .setPostAggregatorSpecs(
+                            ImmutableList.of(
+                                new QuantilePostAggregator("a0", "a0:agg", 0.01f),
+                                new QuantilePostAggregator("a1", "a1:agg", 0.01f)
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{"a", 0.0, 0.0}
+        )
+    );
   }
 }
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java
index d5e51c5..48993ca 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java
@@ -21,9 +21,7 @@
 
 import com.fasterxml.jackson.databind.Module;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.math.expr.ExprMacroTable;
@@ -43,6 +41,7 @@
 import org.apache.druid.query.filter.NotDimFilter;
 import org.apache.druid.query.filter.SelectorDimFilter;
 import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
 import org.apache.druid.segment.IndexBuilder;
 import org.apache.druid.segment.QueryableIndex;
@@ -50,33 +49,20 @@
 import org.apache.druid.segment.incremental.IncrementalIndexSchema;
 import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
 import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
-import org.apache.druid.server.security.AuthenticationResult;
-import org.apache.druid.sql.SqlLifecycle;
 import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
 import org.apache.druid.sql.calcite.filtration.Filtration;
 import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
-import org.apache.druid.sql.calcite.planner.PlannerConfig;
-import org.apache.druid.sql.calcite.planner.PlannerContext;
 import org.apache.druid.sql.calcite.util.CalciteTests;
 import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
-import org.apache.druid.sql.http.SqlParameter;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.partition.LinearShardSpec;
-import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
 public class QuantileSqlAggregatorTest extends BaseCalciteQueryTest
 {
-  private static final AuthenticationResult AUTH_RESULT = CalciteTests.REGULAR_USER_AUTH_RESULT;
-  private static final Map<String, Object> QUERY_CONTEXT_DEFAULT = ImmutableMap.of(
-      PlannerContext.CTX_SQL_QUERY_ID,
-      "dummy"
-  );
-
   private static final DruidOperatorTable OPERATOR_TABLE = new DruidOperatorTable(
       ImmutableSet.of(new QuantileSqlAggregator()),
       ImmutableSet.of()
@@ -127,262 +113,195 @@
   }
 
   @Override
-  public List<Object[]> getResults(
-      final PlannerConfig plannerConfig,
-      final Map<String, Object> queryContext,
-      final List<SqlParameter> parameters,
-      final String sql,
-      final AuthenticationResult authenticationResult
-  ) throws Exception
+  public DruidOperatorTable createOperatorTable()
   {
-    return getResults(
-        plannerConfig,
-        queryContext,
-        parameters,
-        sql,
-        authenticationResult,
-        OPERATOR_TABLE,
-        CalciteTests.createExprMacroTable(),
-        CalciteTests.TEST_AUTHORIZER_MAPPER,
-        CalciteTests.getJsonMapper()
-    );
-  }
-
-  private SqlLifecycle getSqlLifecycle()
-  {
-    return getSqlLifecycleFactory(
-        BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT,
-        OPERATOR_TABLE,
-        CalciteTests.createExprMacroTable(),
-        CalciteTests.TEST_AUTHORIZER_MAPPER,
-        CalciteTests.getJsonMapper()
-    ).factorize();
+    return OPERATOR_TABLE;
   }
 
   @Test
   public void testQuantileOnFloatAndLongs() throws Exception
   {
-    SqlLifecycle sqlLifecycle = getSqlLifecycle();
-
-    final String sql = "SELECT\n"
-                       + "APPROX_QUANTILE(m1, 0.01),\n"
-                       + "APPROX_QUANTILE(m1, 0.5, 50),\n"
-                       + "APPROX_QUANTILE(m1, 0.98, 200),\n"
-                       + "APPROX_QUANTILE(m1, 0.99),\n"
-                       + "APPROX_QUANTILE(m1 * 2, 0.97),\n"
-                       + "APPROX_QUANTILE(m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n"
-                       + "APPROX_QUANTILE(m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n"
-                       + "APPROX_QUANTILE(m1, 0.999) FILTER(WHERE dim1 = 'abc'),\n"
-                       + "APPROX_QUANTILE(cnt, 0.5)\n"
-                       + "FROM foo";
-
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        TIMESERIES_CONTEXT_DEFAULT,
-        DEFAULT_PARAMETERS,
-        AUTH_RESULT
-    ).toList();
-    final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{
-            1.0,
-            3.0,
-            5.880000114440918,
-            5.940000057220459,
-            11.640000343322754,
-            6.0,
-            4.994999885559082,
-            6.0,
-            1.0
-        }
-    );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
-
-    // Verify query
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .virtualColumns(
-                  new ExpressionVirtualColumn(
-                      "v0",
-                      "(\"m1\" * 2)",
-                      ValueType.FLOAT,
-                      TestExprMacroTable.INSTANCE
+    cannotVectorize();
+    testQuery(
+        "SELECT\n"
+        + "APPROX_QUANTILE(m1, 0.01),\n"
+        + "APPROX_QUANTILE(m1, 0.5, 50),\n"
+        + "APPROX_QUANTILE(m1, 0.98, 200),\n"
+        + "APPROX_QUANTILE(m1, 0.99),\n"
+        + "APPROX_QUANTILE(m1 * 2, 0.97),\n"
+        + "APPROX_QUANTILE(m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n"
+        + "APPROX_QUANTILE(m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n"
+        + "APPROX_QUANTILE(m1, 0.999) FILTER(WHERE dim1 = 'abc'),\n"
+        + "APPROX_QUANTILE(cnt, 0.5)\n"
+        + "FROM foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .virtualColumns(
+                      new ExpressionVirtualColumn(
+                          "v0",
+                          "(\"m1\" * 2)",
+                          ValueType.FLOAT,
+                          TestExprMacroTable.INSTANCE
+                      )
                   )
-              )
-              .aggregators(ImmutableList.of(
-                  new ApproximateHistogramAggregatorFactory("a0:agg", "m1", null, null, null, null, false),
-                  new ApproximateHistogramAggregatorFactory("a2:agg", "m1", 200, null, null, null, false),
-                  new ApproximateHistogramAggregatorFactory("a4:agg", "v0", null, null, null, null, false),
-                  new FilteredAggregatorFactory(
-                      new ApproximateHistogramAggregatorFactory("a5:agg", "m1", null, null, null, null, false),
-                      new SelectorDimFilter("dim1", "abc", null)
-                  ),
-                  new FilteredAggregatorFactory(
-                      new ApproximateHistogramAggregatorFactory("a6:agg", "m1", null, null, null, null, false),
-                      new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
-                  ),
-                  new ApproximateHistogramAggregatorFactory("a8:agg", "cnt", null, null, null, null, false)
-              ))
-              .postAggregators(
-                  new QuantilePostAggregator("a0", "a0:agg", 0.01f),
-                  new QuantilePostAggregator("a1", "a0:agg", 0.50f),
-                  new QuantilePostAggregator("a2", "a2:agg", 0.98f),
-                  new QuantilePostAggregator("a3", "a0:agg", 0.99f),
-                  new QuantilePostAggregator("a4", "a4:agg", 0.97f),
-                  new QuantilePostAggregator("a5", "a5:agg", 0.99f),
-                  new QuantilePostAggregator("a6", "a6:agg", 0.999f),
-                  new QuantilePostAggregator("a7", "a5:agg", 0.999f),
-                  new QuantilePostAggregator("a8", "a8:agg", 0.50f)
-              )
-              .context(TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .aggregators(ImmutableList.of(
+                      new ApproximateHistogramAggregatorFactory("a0:agg", "m1", null, null, null, null, false),
+                      new ApproximateHistogramAggregatorFactory("a2:agg", "m1", 200, null, null, null, false),
+                      new ApproximateHistogramAggregatorFactory("a4:agg", "v0", null, null, null, null, false),
+                      new FilteredAggregatorFactory(
+                          new ApproximateHistogramAggregatorFactory("a5:agg", "m1", null, null, null, null, false),
+                          new SelectorDimFilter("dim1", "abc", null)
+                      ),
+                      new FilteredAggregatorFactory(
+                          new ApproximateHistogramAggregatorFactory("a6:agg", "m1", null, null, null, null, false),
+                          new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
+                      ),
+                      new ApproximateHistogramAggregatorFactory("a8:agg", "cnt", null, null, null, null, false)
+                  ))
+                  .postAggregators(
+                      new QuantilePostAggregator("a0", "a0:agg", 0.01f),
+                      new QuantilePostAggregator("a1", "a0:agg", 0.50f),
+                      new QuantilePostAggregator("a2", "a2:agg", 0.98f),
+                      new QuantilePostAggregator("a3", "a0:agg", 0.99f),
+                      new QuantilePostAggregator("a4", "a4:agg", 0.97f),
+                      new QuantilePostAggregator("a5", "a5:agg", 0.99f),
+                      new QuantilePostAggregator("a6", "a6:agg", 0.999f),
+                      new QuantilePostAggregator("a7", "a5:agg", 0.999f),
+                      new QuantilePostAggregator("a8", "a8:agg", 0.50f)
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{
+                1.0,
+                3.0,
+                5.880000114440918,
+                5.940000057220459,
+                11.640000343322754,
+                6.0,
+                4.994999885559082,
+                6.0,
+                1.0
+            }
+        )
     );
   }
 
   @Test
   public void testQuantileOnComplexColumn() throws Exception
   {
-    SqlLifecycle lifecycle = getSqlLifecycle();
-    final String sql = "SELECT\n"
-                       + "APPROX_QUANTILE(hist_m1, 0.01),\n"
-                       + "APPROX_QUANTILE(hist_m1, 0.5, 50),\n"
-                       + "APPROX_QUANTILE(hist_m1, 0.98, 200),\n"
-                       + "APPROX_QUANTILE(hist_m1, 0.99),\n"
-                       + "APPROX_QUANTILE(hist_m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n"
-                       + "APPROX_QUANTILE(hist_m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n"
-                       + "APPROX_QUANTILE(hist_m1, 0.999) FILTER(WHERE dim1 = 'abc')\n"
-                       + "FROM foo";
-
-    // Verify results
-    final List<Object[]> results = lifecycle.runSimple(
-        sql,
-        TIMESERIES_CONTEXT_DEFAULT,
-        DEFAULT_PARAMETERS,
-        AUTH_RESULT
-    ).toList();
-    final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{1.0, 3.0, 5.880000114440918, 5.940000057220459, 6.0, 4.994999885559082, 6.0}
-    );
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
-
-    // Verify query
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE1)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(ImmutableList.of(
-                  new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null, false),
-                  new ApproximateHistogramFoldingAggregatorFactory("a2:agg", "hist_m1", 200, null, null, null, false),
-                  new FilteredAggregatorFactory(
-                      new ApproximateHistogramFoldingAggregatorFactory("a4:agg", "hist_m1", null, null, null, null, false),
-                      new SelectorDimFilter("dim1", "abc", null)
-                  ),
-                  new FilteredAggregatorFactory(
-                      new ApproximateHistogramFoldingAggregatorFactory("a5:agg", "hist_m1", null, null, null, null, false),
-                      new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
+    testQuery(
+        "SELECT\n"
+        + "APPROX_QUANTILE(hist_m1, 0.01),\n"
+        + "APPROX_QUANTILE(hist_m1, 0.5, 50),\n"
+        + "APPROX_QUANTILE(hist_m1, 0.98, 200),\n"
+        + "APPROX_QUANTILE(hist_m1, 0.99),\n"
+        + "APPROX_QUANTILE(hist_m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n"
+        + "APPROX_QUANTILE(hist_m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n"
+        + "APPROX_QUANTILE(hist_m1, 0.999) FILTER(WHERE dim1 = 'abc')\n"
+        + "FROM foo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(ImmutableList.of(
+                      new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null, false),
+                      new ApproximateHistogramFoldingAggregatorFactory("a2:agg", "hist_m1", 200, null, null, null, false),
+                      new FilteredAggregatorFactory(
+                          new ApproximateHistogramFoldingAggregatorFactory("a4:agg", "hist_m1", null, null, null, null, false),
+                          new SelectorDimFilter("dim1", "abc", null)
+                      ),
+                      new FilteredAggregatorFactory(
+                          new ApproximateHistogramFoldingAggregatorFactory("a5:agg", "hist_m1", null, null, null, null, false),
+                          new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
+                      )
+                  ))
+                  .postAggregators(
+                      new QuantilePostAggregator("a0", "a0:agg", 0.01f),
+                      new QuantilePostAggregator("a1", "a0:agg", 0.50f),
+                      new QuantilePostAggregator("a2", "a2:agg", 0.98f),
+                      new QuantilePostAggregator("a3", "a0:agg", 0.99f),
+                      new QuantilePostAggregator("a4", "a4:agg", 0.99f),
+                      new QuantilePostAggregator("a5", "a5:agg", 0.999f),
+                      new QuantilePostAggregator("a6", "a4:agg", 0.999f)
                   )
-              ))
-              .postAggregators(
-                  new QuantilePostAggregator("a0", "a0:agg", 0.01f),
-                  new QuantilePostAggregator("a1", "a0:agg", 0.50f),
-                  new QuantilePostAggregator("a2", "a2:agg", 0.98f),
-                  new QuantilePostAggregator("a3", "a0:agg", 0.99f),
-                  new QuantilePostAggregator("a4", "a4:agg", 0.99f),
-                  new QuantilePostAggregator("a5", "a5:agg", 0.999f),
-                  new QuantilePostAggregator("a6", "a4:agg", 0.999f)
-              )
-              .context(TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{1.0, 3.0, 5.880000114440918, 5.940000057220459, 6.0, 4.994999885559082, 6.0}
+        )
     );
   }
 
   @Test
   public void testQuantileOnInnerQuery() throws Exception
   {
-    SqlLifecycle sqlLifecycle = getSqlLifecycle();
-    final String sql = "SELECT AVG(x), APPROX_QUANTILE(x, 0.98)\n"
-                       + "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)";
-
-    // Verify results
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sql,
-        QUERY_CONTEXT_DEFAULT,
-        DEFAULT_PARAMETERS,
-        AUTH_RESULT
-    ).toList();
     final List<Object[]> expectedResults;
     if (NullHandling.replaceWithDefault()) {
       expectedResults = ImmutableList.of(new Object[]{7.0, 8.26386833190918});
     } else {
       expectedResults = ImmutableList.of(new Object[]{5.25, 6.59091854095459});
     }
-    Assert.assertEquals(expectedResults.size(), results.size());
-    for (int i = 0; i < expectedResults.size(); i++) {
-      Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
-    }
 
-    // Verify query
-    Assert.assertEquals(
-        GroupByQuery.builder()
-                    .setDataSource(
-                        new QueryDataSource(
-                            GroupByQuery.builder()
-                                        .setDataSource(CalciteTests.DATASOURCE1)
-                                        .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-                                        .setGranularity(Granularities.ALL)
-                                        .setDimensions(new DefaultDimensionSpec("dim2", "d0"))
-                                        .setAggregatorSpecs(
-                                            ImmutableList.of(
-                                                new DoubleSumAggregatorFactory("a0", "m1")
+    testQuery(
+        "SELECT AVG(x), APPROX_QUANTILE(x, 0.98)\n"
+        + "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                GroupByQuery.builder()
+                                            .setDataSource(CalciteTests.DATASOURCE1)
+                                            .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                                            .setGranularity(Granularities.ALL)
+                                            .setDimensions(new DefaultDimensionSpec("dim2", "d0"))
+                                            .setAggregatorSpecs(
+                                                ImmutableList.of(
+                                                    new DoubleSumAggregatorFactory("a0", "m1")
+                                                )
                                             )
-                                        )
-                                        .setContext(QUERY_CONTEXT_DEFAULT)
-                                        .build()
+                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                            .build()
+                            )
                         )
-                    )
-                    .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-                    .setGranularity(Granularities.ALL)
-                    .setAggregatorSpecs(
-                        new DoubleSumAggregatorFactory("_a0:sum", "a0"),
-                        new CountAggregatorFactory("_a0:count"),
-                        new ApproximateHistogramAggregatorFactory(
-                            "_a1:agg",
-                            "a0",
-                            null,
-                            null,
-                            null,
-                            null,
-                            false
+                        .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(
+                            new DoubleSumAggregatorFactory("_a0:sum", "a0"),
+                            new CountAggregatorFactory("_a0:count"),
+                            new ApproximateHistogramAggregatorFactory(
+                                "_a1:agg",
+                                "a0",
+                                null,
+                                null,
+                                null,
+                                null,
+                                false
+                            )
                         )
-                    )
-                    .setPostAggregatorSpecs(
-                        ImmutableList.of(
-                            new ArithmeticPostAggregator(
-                                "_a0",
-                                "quotient",
-                                ImmutableList.of(
-                                    new FieldAccessPostAggregator(null, "_a0:sum"),
-                                    new FieldAccessPostAggregator(null, "_a0:count")
-                                )
-                            ),
-                            new QuantilePostAggregator("_a1", "_a1:agg", 0.98f)
+                        .setPostAggregatorSpecs(
+                            ImmutableList.of(
+                                new ArithmeticPostAggregator(
+                                    "_a0",
+                                    "quotient",
+                                    ImmutableList.of(
+                                        new FieldAccessPostAggregator(null, "_a0:sum"),
+                                        new FieldAccessPostAggregator(null, "_a0:count")
+                                    )
+                                ),
+                                new QuantilePostAggregator("_a1", "_a1:agg", 0.98f)
+                            )
                         )
-                    )
-                    .setContext(QUERY_CONTEXT_DEFAULT)
-                    .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        expectedResults
     );
   }
 
@@ -448,4 +367,80 @@
         expectedResults
     );
   }
+
+  @Test
+  public void testEmptyTimeseriesResults() throws Exception
+  {
+    testQuery(
+        "SELECT\n"
+        + "APPROX_QUANTILE(hist_m1, 0.01),\n"
+        + "APPROX_QUANTILE(m1, 0.01)\n"
+        + "FROM foo WHERE dim2 = 0",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
+                  .aggregators(ImmutableList.of(
+                      new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null, false),
+                      new ApproximateHistogramAggregatorFactory("a1:agg", "m1", null, null, null, null, false)
+
+                  ))
+                  .postAggregators(
+                      new QuantilePostAggregator("a0", "a0:agg", 0.01f),
+                      new QuantilePostAggregator("a1", "a1:agg", 0.01f)
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{Double.NaN, Double.NaN}
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByAggregatorDefaultValues() throws Exception
+  {
+    testQuery(
+        "SELECT\n"
+        + "dim2,\n"
+        + "APPROX_QUANTILE(hist_m1, 0.01) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "APPROX_QUANTILE(m1, 0.01) FILTER(WHERE dim1 = 'nonexistent')\n"
+        + "FROM foo WHERE dim2 = 'a' GROUP BY dim2",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE1)
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setDimFilter(selector("dim2", "a", null))
+                        .setGranularity(Granularities.ALL)
+                        .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING))
+                        .setDimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING))
+                        .setAggregatorSpecs(
+                            aggregators(
+                                new FilteredAggregatorFactory(
+                                    new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null, false),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new ApproximateHistogramAggregatorFactory("a1:agg", "m1", null, null, null, null, false),
+                                    selector("dim1", "nonexistent", null)
+                                )
+                            )
+                        )
+                        .setPostAggregatorSpecs(
+                            ImmutableList.of(
+                                new QuantilePostAggregator("a0", "a0:agg", 0.01f),
+                                new QuantilePostAggregator("a1", "a1:agg", 0.01f)
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{"a", Double.NaN, Double.NaN}
+        )
+    );
+  }
 }
diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java
index 5e3b2b2..e67f09f 100644
--- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java
+++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java
@@ -21,7 +21,6 @@
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.data.input.InputRow;
 import org.apache.druid.data.input.impl.DimensionSchema;
@@ -34,6 +33,7 @@
 import org.apache.druid.query.Druids;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
+import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
 import org.apache.druid.query.aggregation.variance.StandardDeviationPostAggregator;
 import org.apache.druid.query.aggregation.variance.VarianceAggregatorCollector;
 import org.apache.druid.query.aggregation.variance.VarianceAggregatorFactory;
@@ -49,16 +49,11 @@
 import org.apache.druid.segment.incremental.IncrementalIndexSchema;
 import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
 import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
-import org.apache.druid.server.security.AuthenticationResult;
-import org.apache.druid.sql.SqlLifecycle;
 import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
 import org.apache.druid.sql.calcite.filtration.Filtration;
 import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
-import org.apache.druid.sql.calcite.planner.PlannerConfig;
-import org.apache.druid.sql.calcite.util.CalciteTestBase;
 import org.apache.druid.sql.calcite.util.CalciteTests;
 import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
-import org.apache.druid.sql.http.SqlParameter;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.partition.LinearShardSpec;
 import org.junit.Assert;
@@ -66,12 +61,9 @@
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
 public class VarianceSqlAggregatorTest extends BaseCalciteQueryTest
 {
-  private static final AuthenticationResult AUTH_RESULT = CalciteTests.REGULAR_USER_AUTH_RESULT;
-  private static final String DATA_SOURCE = "numfoo";
   private static final DruidOperatorTable OPERATOR_TABLE = new DruidOperatorTable(
       ImmutableSet.of(
           new BaseVarianceSqlAggregator.VarPopSqlAggregator(),
@@ -117,7 +109,7 @@
 
     return new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
         DataSegment.builder()
-                   .dataSource(DATA_SOURCE)
+                   .dataSource(CalciteTests.DATASOURCE3)
                    .interval(index.getDataInterval())
                    .version("1")
                    .shardSpec(new LinearShardSpec(0))
@@ -128,36 +120,9 @@
   }
 
   @Override
-  public List<Object[]> getResults(
-      final PlannerConfig plannerConfig,
-      final Map<String, Object> queryContext,
-      final List<SqlParameter> parameters,
-      final String sql,
-      final AuthenticationResult authenticationResult
-  ) throws Exception
+  public DruidOperatorTable createOperatorTable()
   {
-    return getResults(
-        plannerConfig,
-        queryContext,
-        parameters,
-        sql,
-        authenticationResult,
-        OPERATOR_TABLE,
-        CalciteTests.createExprMacroTable(),
-        CalciteTests.TEST_AUTHORIZER_MAPPER,
-        CalciteTests.getJsonMapper()
-    );
-  }
-
-  private SqlLifecycle getSqlLifecycle()
-  {
-    return getSqlLifecycleFactory(
-        BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT,
-        OPERATOR_TABLE,
-        CalciteTests.createExprMacroTable(),
-        CalciteTests.TEST_AUTHORIZER_MAPPER,
-        CalciteTests.getJsonMapper()
-    ).factorize();
+    return OPERATOR_TABLE;
   }
 
   public void addToHolder(VarianceAggregatorCollector holder, Object raw)
@@ -191,20 +156,6 @@
   @Test
   public void testVarPop() throws Exception
   {
-    final String sql = "SELECT\n"
-                       + "VAR_POP(d1),\n"
-                       + "VAR_POP(f1),\n"
-                       + "VAR_POP(l1)\n"
-                       + "FROM numfoo";
-
-    final List<Object[]> results =
-        getSqlLifecycle().runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            AUTH_RESULT
-        ).toList();
-
     VarianceAggregatorCollector holder1 = new VarianceAggregatorCollector();
     VarianceAggregatorCollector holder2 = new VarianceAggregatorCollector();
     VarianceAggregatorCollector holder3 = new VarianceAggregatorCollector();
@@ -224,43 +175,34 @@
             holder3.getVariance(true).longValue()
         }
     );
-    assertResultsEquals(sql, expectedResults, results);
-
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE3)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(
-                  ImmutableList.of(
-                      new VarianceAggregatorFactory("a0:agg", "d1", "population", "double"),
-                      new VarianceAggregatorFactory("a1:agg", "f1", "population", "float"),
-                      new VarianceAggregatorFactory("a2:agg", "l1", "population", "long")
+    testQuery(
+        "SELECT\n"
+        + "VAR_POP(d1),\n"
+        + "VAR_POP(f1),\n"
+        + "VAR_POP(l1)\n"
+        + "FROM numfoo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(
+                      ImmutableList.of(
+                          new VarianceAggregatorFactory("a0:agg", "d1", "population", "double"),
+                          new VarianceAggregatorFactory("a1:agg", "f1", "population", "float"),
+                          new VarianceAggregatorFactory("a2:agg", "l1", "population", "long")
+                      )
                   )
-              )
-              .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        expectedResults
     );
   }
 
   @Test
   public void testVarSamp() throws Exception
   {
-    final String sql = "SELECT\n"
-                       + "VAR_SAMP(d1),\n"
-                       + "VAR_SAMP(f1),\n"
-                       + "VAR_SAMP(l1)\n"
-                       + "FROM numfoo";
-
-    final List<Object[]> results =
-        getSqlLifecycle().runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            AUTH_RESULT
-        ).toList();
-
     VarianceAggregatorCollector holder1 = new VarianceAggregatorCollector();
     VarianceAggregatorCollector holder2 = new VarianceAggregatorCollector();
     VarianceAggregatorCollector holder3 = new VarianceAggregatorCollector();
@@ -274,49 +216,40 @@
     }
 
     final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{
+        new Object[] {
             holder1.getVariance(false),
             holder2.getVariance(false).floatValue(),
             holder3.getVariance(false).longValue(),
-            }
+        }
     );
-    assertResultsEquals(sql, expectedResults, results);
-
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE3)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(
-                  ImmutableList.of(
-                      new VarianceAggregatorFactory("a0:agg", "d1", "sample", "double"),
-                      new VarianceAggregatorFactory("a1:agg", "f1", "sample", "float"),
-                      new VarianceAggregatorFactory("a2:agg", "l1", "sample", "long")
+    testQuery(
+        "SELECT\n"
+        + "VAR_SAMP(d1),\n"
+        + "VAR_SAMP(f1),\n"
+        + "VAR_SAMP(l1)\n"
+        + "FROM numfoo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(
+                      ImmutableList.of(
+                          new VarianceAggregatorFactory("a0:agg", "d1", "sample", "double"),
+                          new VarianceAggregatorFactory("a1:agg", "f1", "sample", "float"),
+                          new VarianceAggregatorFactory("a2:agg", "l1", "sample", "long")
+                      )
                   )
-              )
-              .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        expectedResults
     );
   }
 
   @Test
   public void testStdDevPop() throws Exception
   {
-    final String sql = "SELECT\n"
-                       + "STDDEV_POP(d1),\n"
-                       + "STDDEV_POP(f1),\n"
-                       + "STDDEV_POP(l1)\n"
-                       + "FROM numfoo";
-
-    final List<Object[]> results =
-        getSqlLifecycle().runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            AUTH_RESULT
-        ).toList();
-
     VarianceAggregatorCollector holder1 = new VarianceAggregatorCollector();
     VarianceAggregatorCollector holder2 = new VarianceAggregatorCollector();
     VarianceAggregatorCollector holder3 = new VarianceAggregatorCollector();
@@ -330,56 +263,48 @@
     }
 
     final List<Object[]> expectedResults = ImmutableList.of(
-        new Object[]{
+        new Object[] {
             Math.sqrt(holder1.getVariance(true)),
             (float) Math.sqrt(holder2.getVariance(true)),
             (long) Math.sqrt(holder3.getVariance(true)),
-            }
+        }
     );
-    assertResultsEquals(sql, expectedResults, results);
 
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE3)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(
-                  ImmutableList.of(
-                      new VarianceAggregatorFactory("a0:agg", "d1", "population", "double"),
-                      new VarianceAggregatorFactory("a1:agg", "f1", "population", "float"),
-                      new VarianceAggregatorFactory("a2:agg", "l1", "population", "long")
+    testQuery(
+        "SELECT\n"
+        + "STDDEV_POP(d1),\n"
+        + "STDDEV_POP(f1),\n"
+        + "STDDEV_POP(l1)\n"
+        + "FROM numfoo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(
+                      ImmutableList.of(
+                          new VarianceAggregatorFactory("a0:agg", "d1", "population", "double"),
+                          new VarianceAggregatorFactory("a1:agg", "f1", "population", "float"),
+                          new VarianceAggregatorFactory("a2:agg", "l1", "population", "long")
+                      )
                   )
-              )
-              .postAggregators(
-                  ImmutableList.of(
-                      new StandardDeviationPostAggregator("a0", "a0:agg", "population"),
-                      new StandardDeviationPostAggregator("a1", "a1:agg", "population"),
-                      new StandardDeviationPostAggregator("a2", "a2:agg", "population")
+                  .postAggregators(
+                      ImmutableList.of(
+                          new StandardDeviationPostAggregator("a0", "a0:agg", "population"),
+                          new StandardDeviationPostAggregator("a1", "a1:agg", "population"),
+                          new StandardDeviationPostAggregator("a2", "a2:agg", "population")
+                      )
                   )
-              )
-              .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        expectedResults
     );
   }
 
   @Test
   public void testStdDevSamp() throws Exception
   {
-    final String sql = "SELECT\n"
-                       + "STDDEV_SAMP(d1),\n"
-                       + "STDDEV_SAMP(f1),\n"
-                       + "STDDEV_SAMP(l1)\n"
-                       + "FROM numfoo";
-
-    final List<Object[]> results =
-        getSqlLifecycle().runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            AUTH_RESULT
-        ).toList();
-
     VarianceAggregatorCollector holder1 = new VarianceAggregatorCollector();
     VarianceAggregatorCollector holder2 = new VarianceAggregatorCollector();
     VarianceAggregatorCollector holder3 = new VarianceAggregatorCollector();
@@ -397,50 +322,42 @@
             Math.sqrt(holder1.getVariance(false)),
             (float) Math.sqrt(holder2.getVariance(false)),
             (long) Math.sqrt(holder3.getVariance(false)),
-            }
+        }
     );
-    assertResultsEquals(sql, expectedResults, results);
 
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE3)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .aggregators(
-                  ImmutableList.of(
-                      new VarianceAggregatorFactory("a0:agg", "d1", "sample", "double"),
-                      new VarianceAggregatorFactory("a1:agg", "f1", "sample", "float"),
-                      new VarianceAggregatorFactory("a2:agg", "l1", "sample", "long")
+    testQuery(
+        "SELECT\n"
+        + "STDDEV_SAMP(d1),\n"
+        + "STDDEV_SAMP(f1),\n"
+        + "STDDEV_SAMP(l1)\n"
+        + "FROM numfoo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .aggregators(
+                      ImmutableList.of(
+                          new VarianceAggregatorFactory("a0:agg", "d1", "sample", "double"),
+                          new VarianceAggregatorFactory("a1:agg", "f1", "sample", "float"),
+                          new VarianceAggregatorFactory("a2:agg", "l1", "sample", "long")
+                      )
                   )
-              )
-              .postAggregators(
-                  new StandardDeviationPostAggregator("a0", "a0:agg", "sample"),
-                  new StandardDeviationPostAggregator("a1", "a1:agg", "sample"),
-                  new StandardDeviationPostAggregator("a2", "a2:agg", "sample")
-              )
-              .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .postAggregators(
+                      new StandardDeviationPostAggregator("a0", "a0:agg", "sample"),
+                      new StandardDeviationPostAggregator("a1", "a1:agg", "sample"),
+                      new StandardDeviationPostAggregator("a2", "a2:agg", "sample")
+                  )
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        expectedResults
     );
   }
 
   @Test
   public void testStdDevWithVirtualColumns() throws Exception
   {
-    final String sql = "SELECT\n"
-                       + "STDDEV(d1*7),\n"
-                       + "STDDEV(f1*7),\n"
-                       + "STDDEV(l1*7)\n"
-                       + "FROM numfoo";
-
-    final List<Object[]> results =
-        getSqlLifecycle().runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            AUTH_RESULT
-        ).toList();
-
     VarianceAggregatorCollector holder1 = new VarianceAggregatorCollector();
     VarianceAggregatorCollector holder2 = new VarianceAggregatorCollector();
     VarianceAggregatorCollector holder3 = new VarianceAggregatorCollector();
@@ -458,35 +375,41 @@
             Math.sqrt(holder1.getVariance(false)),
             (float) Math.sqrt(holder2.getVariance(false)),
             (long) Math.sqrt(holder3.getVariance(false)),
-            }
+        }
     );
-    assertResultsEquals(sql, expectedResults, results);
 
-    Assert.assertEquals(
-        Druids.newTimeseriesQueryBuilder()
-              .dataSource(CalciteTests.DATASOURCE3)
-              .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-              .granularity(Granularities.ALL)
-              .virtualColumns(
-                  BaseCalciteQueryTest.expressionVirtualColumn("v0", "(\"d1\" * 7)", ValueType.DOUBLE),
-                  BaseCalciteQueryTest.expressionVirtualColumn("v1", "(\"f1\" * 7)", ValueType.FLOAT),
-                  BaseCalciteQueryTest.expressionVirtualColumn("v2", "(\"l1\" * 7)", ValueType.LONG)
-              )
-              .aggregators(
-                  ImmutableList.of(
-                      new VarianceAggregatorFactory("a0:agg", "v0", "sample", "double"),
-                      new VarianceAggregatorFactory("a1:agg", "v1", "sample", "float"),
-                      new VarianceAggregatorFactory("a2:agg", "v2", "sample", "long")
+    testQuery(
+        "SELECT\n"
+        + "STDDEV(d1*7),\n"
+        + "STDDEV(f1*7),\n"
+        + "STDDEV(l1*7)\n"
+        + "FROM numfoo",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                  .granularity(Granularities.ALL)
+                  .virtualColumns(
+                      BaseCalciteQueryTest.expressionVirtualColumn("v0", "(\"d1\" * 7)", ValueType.DOUBLE),
+                      BaseCalciteQueryTest.expressionVirtualColumn("v1", "(\"f1\" * 7)", ValueType.FLOAT),
+                      BaseCalciteQueryTest.expressionVirtualColumn("v2", "(\"l1\" * 7)", ValueType.LONG)
                   )
-              )
-              .postAggregators(
-                  new StandardDeviationPostAggregator("a0", "a0:agg", "sample"),
-                  new StandardDeviationPostAggregator("a1", "a1:agg", "sample"),
-                  new StandardDeviationPostAggregator("a2", "a2:agg", "sample")
-              )
-              .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
-              .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                  .aggregators(
+                      ImmutableList.of(
+                          new VarianceAggregatorFactory("a0:agg", "v0", "sample", "double"),
+                          new VarianceAggregatorFactory("a1:agg", "v1", "sample", "float"),
+                          new VarianceAggregatorFactory("a2:agg", "v2", "sample", "long")
+                      )
+                  )
+                  .postAggregators(
+                      new StandardDeviationPostAggregator("a0", "a0:agg", "sample"),
+                      new StandardDeviationPostAggregator("a1", "a1:agg", "sample"),
+                      new StandardDeviationPostAggregator("a2", "a2:agg", "sample")
+                  )
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        expectedResults
     );
   }
 
@@ -494,15 +417,6 @@
   @Test
   public void testVarianceOrderBy() throws Exception
   {
-    queryLogHook.clearRecordedQueries();
-    final String sql = "select dim2, VARIANCE(f1) from druid.numfoo group by 1 order by 2 desc";
-    final List<Object[]> results =
-        getSqlLifecycle().runSimple(
-            sql,
-            BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT,
-            CalciteTestBase.DEFAULT_PARAMETERS,
-            AUTH_RESULT
-        ).toList();
     List<Object[]> expectedResults = NullHandling.sqlCompatible()
                                      ? ImmutableList.of(
         new Object[]{"a", 0f},
@@ -514,32 +428,34 @@
         new Object[]{"", 0.0033333334f},
         new Object[]{"abc", 0f}
     );
-    assertResultsEquals(sql, expectedResults, results);
 
-    Assert.assertEquals(
-        GroupByQuery.builder()
-                    .setDataSource(CalciteTests.DATASOURCE3)
-                    .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
-                    .setGranularity(Granularities.ALL)
-                    .setDimensions(new DefaultDimensionSpec("dim2", "_d0"))
-                    .setAggregatorSpecs(
-                        new VarianceAggregatorFactory("a0:agg", "f1", "sample", "float")
-                    )
-                    .setLimitSpec(
-                        DefaultLimitSpec
-                            .builder()
-                            .orderBy(
-                                new OrderByColumnSpec(
-                                    "a0:agg",
-                                    OrderByColumnSpec.Direction.DESCENDING,
-                                    StringComparators.NUMERIC
+    testQuery(
+        "select dim2, VARIANCE(f1) from druid.numfoo group by 1 order by 2 desc",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE3)
+                        .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                        .setGranularity(Granularities.ALL)
+                        .setDimensions(new DefaultDimensionSpec("dim2", "_d0"))
+                        .setAggregatorSpecs(
+                            new VarianceAggregatorFactory("a0:agg", "f1", "sample", "float")
+                        )
+                        .setLimitSpec(
+                            DefaultLimitSpec
+                                .builder()
+                                .orderBy(
+                                    new OrderByColumnSpec(
+                                        "a0:agg",
+                                        OrderByColumnSpec.Direction.DESCENDING,
+                                        StringComparators.NUMERIC
+                                    )
                                 )
-                            )
-                            .build()
-                    )
-                    .setContext(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
-                    .build(),
-        Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+                                .build()
+                        )
+                        .setContext(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        expectedResults
     );
   }
 
@@ -555,7 +471,7 @@
         + "FROM numfoo",
         ImmutableList.of(
             Druids.newTimeseriesQueryBuilder()
-            .dataSource(DATA_SOURCE)
+            .dataSource(CalciteTests.DATASOURCE3)
             .intervals(querySegmentSpec(Filtration.eternity()))
             .virtualColumns(
                 new ExpressionVirtualColumn("v0", "CAST(\"dim1\", 'DOUBLE')", ValueType.DOUBLE, ExprMacroTable.nil())
@@ -572,7 +488,7 @@
                 new StandardDeviationPostAggregator("a1", "a1:agg", "sample"),
                 new StandardDeviationPostAggregator("a2", "a2:agg", "sample")
             )
-            .context(BaseCalciteQueryTest.TIMESERIES_CONTEXT_DEFAULT)
+            .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
             .build()
         ),
         ImmutableList.of(
@@ -583,6 +499,136 @@
     );
   }
 
+  @Test
+  public void testEmptyTimeseriesResults() throws Exception
+  {
+    testQuery(
+        "SELECT\n"
+        + "STDDEV_POP(d1),\n"
+        + "STDDEV_SAMP(d1),\n"
+        + "STDDEV(d1),\n"
+        + "VARIANCE(d1),\n"
+        + "STDDEV_POP(l1),\n"
+        + "STDDEV_SAMP(l1),\n"
+        + "STDDEV(l1),\n"
+        + "VARIANCE(l1)\n"
+        + "FROM numfoo WHERE dim2 = 0",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(querySegmentSpec(Filtration.eternity()))
+                  .granularity(Granularities.ALL)
+                  .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
+                  .aggregators(
+                      new VarianceAggregatorFactory("a0:agg", "d1", "population", "double"),
+                      new VarianceAggregatorFactory("a1:agg", "d1", "sample", "double"),
+                      new VarianceAggregatorFactory("a2:agg", "d1", "sample", "double"),
+                      new VarianceAggregatorFactory("a3:agg", "d1", "sample", "double"),
+                      new VarianceAggregatorFactory("a4:agg", "l1", "population", "long"),
+                      new VarianceAggregatorFactory("a5:agg", "l1", "sample", "long"),
+                      new VarianceAggregatorFactory("a6:agg", "l1", "sample", "long"),
+                      new VarianceAggregatorFactory("a7:agg", "l1", "sample", "long")
+
+                  )
+                  .postAggregators(
+                      new StandardDeviationPostAggregator("a0", "a0:agg", "population"),
+                      new StandardDeviationPostAggregator("a1", "a1:agg", "sample"),
+                      new StandardDeviationPostAggregator("a2", "a2:agg", "sample"),
+                      new StandardDeviationPostAggregator("a4", "a4:agg", "population"),
+                      new StandardDeviationPostAggregator("a5", "a5:agg", "sample"),
+                      new StandardDeviationPostAggregator("a6", "a6:agg", "sample")
+                  )
+                  .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            NullHandling.replaceWithDefault()
+            ? new Object[]{0.0, 0.0, 0.0, 0.0, 0L, 0L, 0L, 0L}
+            : new Object[]{null, null, null, null, null, null, null, null}
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByAggregatorDefaultValues() throws Exception
+  {
+    testQuery(
+        "SELECT\n"
+        + "dim2,\n"
+        + "STDDEV_POP(d1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "STDDEV_SAMP(d1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "STDDEV(d1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "VARIANCE(d1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "STDDEV_POP(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "STDDEV_SAMP(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "STDDEV(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + "VARIANCE(l1) FILTER(WHERE dim1 = 'nonexistent')\n"
+        + "FROM numfoo WHERE dim2 = 'a' GROUP BY dim2",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE3)
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setDimFilter(selector("dim2", "a", null))
+                        .setGranularity(Granularities.ALL)
+                        .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING))
+                        .setDimensions(new DefaultDimensionSpec("v0", "_d0", ValueType.STRING))
+                        .setAggregatorSpecs(
+                            aggregators(
+                                new FilteredAggregatorFactory(
+                                    new VarianceAggregatorFactory("a0:agg", "d1", "population", "double"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new VarianceAggregatorFactory("a1:agg", "d1", "sample", "double"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new VarianceAggregatorFactory("a2:agg", "d1", "sample", "double"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new VarianceAggregatorFactory("a3:agg", "d1", "sample", "double"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new VarianceAggregatorFactory("a4:agg", "l1", "population", "long"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new VarianceAggregatorFactory("a5:agg", "l1", "sample", "long"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new VarianceAggregatorFactory("a6:agg", "l1", "sample", "long"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new VarianceAggregatorFactory("a7:agg", "l1", "sample", "long"),
+                                    selector("dim1", "nonexistent", null)
+                                )
+                            )
+                        )
+                        .setPostAggregatorSpecs(
+                            ImmutableList.of(
+                                new StandardDeviationPostAggregator("a0", "a0:agg", "population"),
+                                new StandardDeviationPostAggregator("a1", "a1:agg", "sample"),
+                                new StandardDeviationPostAggregator("a2", "a2:agg", "sample"),
+                                new StandardDeviationPostAggregator("a4", "a4:agg", "population"),
+                                new StandardDeviationPostAggregator("a5", "a5:agg", "sample"),
+                                new StandardDeviationPostAggregator("a6", "a6:agg", "sample")
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            NullHandling.replaceWithDefault()
+            ? new Object[]{"a", 0.0, 0.0, 0.0, 0.0, 0L, 0L, 0L, 0L}
+            : new Object[]{"a", null, null, null, null, null, null, null, null}
+        )
+    );
+  }
+
   @Override
   public void assertResultsEquals(String sql, List<Object[]> expectedResults, List<Object[]> results)
   {
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 89aff39..bc4173a 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
@@ -1687,7 +1687,7 @@
                                   .aggregators(aggregatorFactoryList)
                                   .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT)
                                   .descending(descending)
-                                  .context(makeContext(ImmutableMap.of("skipEmptyBuckets", "true")))
+                                  .context(makeContext(ImmutableMap.of(TimeseriesQuery.SKIP_EMPTY_BUCKETS, "true")))
                                   .build();
 
     List<Result<TimeseriesResultValue>> expectedResults = Collections.emptyList();
@@ -2559,7 +2559,7 @@
                                       makeContext(
                                           ImmutableMap.of(
                                               TimeseriesQuery.CTX_TIMESTAMP_RESULT_FIELD, TIMESTAMP_RESULT_FIELD_NAME,
-                                              "skipEmptyBuckets", true
+                                              TimeseriesQuery.SKIP_EMPTY_BUCKETS, true
                                           )
                                       )
                                   )
@@ -2690,7 +2690,7 @@
                                       makeContext(
                                           ImmutableMap.of(
                                               TimeseriesQuery.CTX_TIMESTAMP_RESULT_FIELD, TIMESTAMP_RESULT_FIELD_NAME,
-                                              "skipEmptyBuckets", true
+                                              TimeseriesQuery.SKIP_EMPTY_BUCKETS, true
                                           )
                                       )
                                   )
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java
index a12ba34..c503f96 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java
@@ -784,8 +784,6 @@
     final boolean descending;
     int timeseriesLimit = 0;
     final Map<String, Object> theContext = new HashMap<>();
-    theContext.put("skipEmptyBuckets", true);
-    theContext.putAll(plannerContext.getQueryContext());
     if (grouping.getDimensions().isEmpty()) {
       queryGranularity = Granularities.ALL;
       descending = false;
@@ -842,6 +840,14 @@
       return null;
     }
 
+    // An aggregation query should return one row per group, with no grouping (e.g. ALL granularity), the entire table
+    // is the group, so we should not skip empty buckets. When there are no results, this means we return the
+    // initialized state for given aggregators instead of nothing.
+    if (!Granularities.ALL.equals(queryGranularity)) {
+      theContext.put(TimeseriesQuery.SKIP_EMPTY_BUCKETS, true);
+    }
+    theContext.putAll(plannerContext.getQueryContext());
+
     final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration(
         dataSource,
         filter,
diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java
index 666fa15..7a3a034 100644
--- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java
@@ -337,7 +337,7 @@
         ImmutableList.of(
             ImmutableMap.of(
                 "PLAN",
-                StringUtils.format("DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"skipEmptyBuckets\":true,\"sqlQueryId\":\"%s\",\"sqlTimeZone\":\"America/Los_Angeles\"}}], signature=[{a0:LONG}])\n",
+                StringUtils.format("DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"sqlQueryId\":\"%s\",\"sqlTimeZone\":\"America/Los_Angeles\"}}], signature=[{a0:LONG}])\n",
                                    DUMMY_SQL_QUERY_ID
                 ),
                 "RESOURCES",
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
index 19e5860..645f9b9 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
@@ -178,7 +178,15 @@
   public static final Map<String, Object> QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS = ImmutableMap.of(
       PlannerContext.CTX_SQL_QUERY_ID, DUMMY_SQL_ID,
       PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z",
-      "skipEmptyBuckets", false,
+      TimeseriesQuery.SKIP_EMPTY_BUCKETS, false,
+      QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS,
+      QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE
+  );
+
+  public static final Map<String, Object> QUERY_CONTEXT_DO_SKIP_EMPTY_BUCKETS = ImmutableMap.of(
+      PlannerContext.CTX_SQL_QUERY_ID, DUMMY_SQL_ID,
+      PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z",
+      TimeseriesQuery.SKIP_EMPTY_BUCKETS, true,
       QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS,
       QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE
   );
@@ -200,10 +208,10 @@
   );
 
   // Matches QUERY_CONTEXT_DEFAULT
-  public static final Map<String, Object> TIMESERIES_CONTEXT_DEFAULT = ImmutableMap.of(
+  public static final Map<String, Object> TIMESERIES_CONTEXT_BY_GRAN = ImmutableMap.of(
       PlannerContext.CTX_SQL_QUERY_ID, DUMMY_SQL_ID,
       PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z",
-      "skipEmptyBuckets", true,
+      TimeseriesQuery.SKIP_EMPTY_BUCKETS, true,
       QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS,
       QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE
   );
@@ -242,7 +250,7 @@
     TIMESERIES_CONTEXT_LOS_ANGELES.put(PlannerContext.CTX_SQL_QUERY_ID, DUMMY_SQL_ID);
     TIMESERIES_CONTEXT_LOS_ANGELES.put(PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z");
     TIMESERIES_CONTEXT_LOS_ANGELES.put(PlannerContext.CTX_SQL_TIME_ZONE, LOS_ANGELES);
-    TIMESERIES_CONTEXT_LOS_ANGELES.put("skipEmptyBuckets", true);
+    TIMESERIES_CONTEXT_LOS_ANGELES.put(TimeseriesQuery.SKIP_EMPTY_BUCKETS, true);
     TIMESERIES_CONTEXT_LOS_ANGELES.put(QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS);
     TIMESERIES_CONTEXT_LOS_ANGELES.put(QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE);
 
@@ -457,6 +465,16 @@
     );
   }
 
+  public DruidOperatorTable createOperatorTable()
+  {
+    return CalciteTests.createOperatorTable();
+  }
+
+  public ExprMacroTable createMacroTable()
+  {
+    return CalciteTests.createExprMacroTable();
+  }
+
   public void assertQueryIsUnplannable(final String sql)
   {
     assertQueryIsUnplannable(PLANNER_CONFIG_DEFAULT, sql);
@@ -681,8 +699,8 @@
         parameters,
         sql,
         authenticationResult,
-        CalciteTests.createOperatorTable(),
-        CalciteTests.createExprMacroTable(),
+        createOperatorTable(),
+        createMacroTable(),
         CalciteTests.TEST_AUTHORIZER_MAPPER,
         CalciteTests.getJsonMapper()
     );
@@ -762,8 +780,8 @@
   {
     SqlLifecycleFactory lifecycleFactory = getSqlLifecycleFactory(
         plannerConfig,
-        CalciteTests.createOperatorTable(),
-        CalciteTests.createExprMacroTable(),
+        createOperatorTable(),
+        createMacroTable(),
         CalciteTests.TEST_AUTHORIZER_MAPPER,
         CalciteTests.getJsonMapper()
     );
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java
index 0308783..e10b6fe 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java
@@ -109,7 +109,7 @@
   {
     testQuery(
         PLANNER_CONFIG_DEFAULT,
-        QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS,
+        QUERY_CONTEXT_DEFAULT,
         ImmutableList.of(
             new SqlParameter(SqlType.INTEGER, 10),
             new SqlParameter(SqlType.INTEGER, 0)
@@ -128,7 +128,7 @@
                                .postAggregators(
                                    expressionPostAgg("p0", "(exp(\"a0\") + 10)")
                                )
-                               .context(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS)
+                               .context(QUERY_CONTEXT_DEFAULT)
                                .build()),
         ImmutableList.of(
             new Object[]{11.0, NullHandling.defaultDoubleValue()}
@@ -175,7 +175,7 @@
                       expressionPostAgg("p9", "' foo'"),
                       expressionPostAgg("p10", "'xfoo'")
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -352,7 +352,7 @@
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new DoubleSumAggregatorFactory("a0", "m1")))
                   .postAggregators(ImmutableList.of(expressionPostAgg("p0", "(\"a0\" / 10)")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{2.1}),
@@ -374,7 +374,7 @@
     // with millis
     testQuery(
         PLANNER_CONFIG_DEFAULT,
-        QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS,
+        QUERY_CONTEXT_DEFAULT,
         ImmutableList.of(
             new SqlParameter(SqlType.INTEGER, 10),
             new SqlParameter(
@@ -396,7 +396,7 @@
                                .postAggregators(
                                    expressionPostAgg("p0", "(exp(\"a0\") + 10)")
                                )
-                               .context(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS)
+                               .context(QUERY_CONTEXT_DEFAULT)
                                .build()),
         ImmutableList.of(
             new Object[]{11.0, NullHandling.defaultDoubleValue()}
@@ -411,7 +411,7 @@
     // with timestampstring
     testQuery(
         PLANNER_CONFIG_DEFAULT,
-        QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS,
+        QUERY_CONTEXT_DEFAULT,
         ImmutableList.of(
             new SqlParameter(SqlType.INTEGER, 10),
             new SqlParameter(
@@ -433,7 +433,7 @@
                                .postAggregators(
                                    expressionPostAgg("p0", "(exp(\"a0\") + 10)")
                                )
-                               .context(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS)
+                               .context(QUERY_CONTEXT_DEFAULT)
                                .build()),
         ImmutableList.of(
             new Object[]{11.0, NullHandling.defaultDoubleValue()}
@@ -448,7 +448,7 @@
 
     testQuery(
         PLANNER_CONFIG_DEFAULT,
-        QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS,
+        QUERY_CONTEXT_DEFAULT,
         ImmutableList.of(
             new SqlParameter(SqlType.INTEGER, 10),
             new SqlParameter(
@@ -470,7 +470,7 @@
                                .postAggregators(
                                    expressionPostAgg("p0", "(exp(\"a0\") + 10)")
                                )
-                               .context(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS)
+                               .context(QUERY_CONTEXT_DEFAULT)
                                .build()),
         ImmutableList.of(
             new Object[]{11.0, NullHandling.defaultDoubleValue()}
@@ -492,10 +492,10 @@
                       bound("cnt", "1.1", "100000001", true, true, null, StringComparators.NUMERIC)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
-        ImmutableList.of(),
+        ImmutableList.of(new Object[]{0L}),
         ImmutableList.of(
             new SqlParameter(SqlType.DOUBLE, 1.1),
             new SqlParameter(SqlType.FLOAT, 100000001.0)
@@ -514,7 +514,7 @@
                       in("cnt", ImmutableList.of("1.0", "100000001"), null)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -541,7 +541,7 @@
                       selector("cnt", "1.0", null)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{6L}),
@@ -563,7 +563,7 @@
                   .granularity(Granularities.ALL)
                   .filters(bound("l1", "3", null, true, false, null, StringComparators.NUMERIC))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{2L}),
@@ -625,7 +625,7 @@
                       )
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         )
         : ImmutableList.of(
@@ -638,10 +638,10 @@
                   )
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
-        ImmutableList.of(),
+        ImmutableList.of(new Object[]{0L}),
         ImmutableList.of(new SqlParameter(SqlType.BIGINT, 3L), new SqlParameter(SqlType.VARCHAR, "wat"))
     );
   }
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 d6d905f..1efd99b 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
@@ -308,7 +308,7 @@
                                   .intervals(querySegmentSpec(Filtration.eternity()))
                                   .granularity(Granularities.HOUR)
                                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                                   .build()
                         )
                         .setInterval(querySegmentSpec(Intervals.ETERNITY))
@@ -614,7 +614,7 @@
                                           .aggregators(aggregators(
                                               new FloatMinAggregatorFactory("a0", "m1")
                                           ))
-                                          .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                                          .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                                           .build()),
                                 "j0.",
                                 "((timestamp_floor(\"__time\",'PT1H',null,'UTC') == \"j0.d0\") && (\"m1\" == \"j0.a0\"))",
@@ -704,11 +704,13 @@
   }
 
   @Test
-  public void testSelectCountStart() throws Exception
+  public void testSelectCountStar() throws Exception
   {
+    // timeseries with all granularity have a single group, so should return default results for given aggregators
+    // which for count is 0 and sum is null in sql compatible mode or 0.0 in default mode.
     testQuery(
         PLANNER_CONFIG_DEFAULT,
-        QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS,
+        QUERY_CONTEXT_DEFAULT,
         "SELECT exp(count(*)) + 10, sum(m2)  FROM druid.foo WHERE  dim2 = 0",
         CalciteTests.REGULAR_USER_AUTH_RESULT,
         ImmutableList.of(Druids.newTimeseriesQueryBuilder()
@@ -723,7 +725,7 @@
                                .postAggregators(
                                    expressionPostAgg("p0", "(exp(\"a0\") + 10)")
                                )
-                               .context(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS)
+                               .context(QUERY_CONTEXT_DEFAULT)
                                .build()),
         ImmutableList.of(
             new Object[]{11.0, NullHandling.defaultDoubleValue()}
@@ -732,7 +734,7 @@
 
     testQuery(
         PLANNER_CONFIG_DEFAULT,
-        QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS,
+        QUERY_CONTEXT_DEFAULT,
         "SELECT exp(count(*)) + 10, sum(m2)  FROM druid.foo WHERE  __time >= TIMESTAMP '2999-01-01 00:00:00'",
         CalciteTests.REGULAR_USER_AUTH_RESULT,
         ImmutableList.of(Druids.newTimeseriesQueryBuilder()
@@ -748,13 +750,34 @@
                                .postAggregators(
                                    expressionPostAgg("p0", "(exp(\"a0\") + 10)")
                                )
-                               .context(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS)
+                               .context(QUERY_CONTEXT_DEFAULT)
                                .build()),
         ImmutableList.of(
             new Object[]{11.0, NullHandling.defaultDoubleValue()}
         )
     );
 
+    // this behavior was not always correct, so make sure legacy behavior can be retained by skipping empty buckets
+    // explicitly in the context which causes these timeseries queries to return no results
+    testQuery(
+        PLANNER_CONFIG_DEFAULT,
+        TIMESERIES_CONTEXT_BY_GRAN,
+        "SELECT COUNT(*) FROM foo WHERE dim1 = 'nonexistent'",
+        CalciteTests.REGULAR_USER_AUTH_RESULT,
+        ImmutableList.of(Druids.newTimeseriesQueryBuilder()
+                               .dataSource(CalciteTests.DATASOURCE1)
+                               .intervals(querySegmentSpec(Filtration.eternity()))
+                               .filters(selector("dim1", "nonexistent", null))
+                               .granularity(Granularities.ALL)
+                               .aggregators(aggregators(
+                                   new CountAggregatorFactory("a0")
+                               ))
+                               .context(TIMESERIES_CONTEXT_BY_GRAN)
+                               .build()),
+        ImmutableList.of()
+    );
+
+    // timeseries with a granularity is grouping by the time expression, so matching nothing returns no results
     testQuery(
         "SELECT COUNT(*) FROM foo WHERE dim1 = 'nonexistent' GROUP BY FLOOR(__time TO DAY)",
         ImmutableList.of(Druids.newTimeseriesQueryBuilder()
@@ -765,7 +788,7 @@
                                .aggregators(aggregators(
                                    new CountAggregatorFactory("a0")
                                ))
-                               .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                               .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                                .build()),
         ImmutableList.of()
     );
@@ -810,7 +833,7 @@
                       expressionPostAgg("p9", "' foo'"),
                       expressionPostAgg("p10", "'xfoo'")
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -846,7 +869,7 @@
                       expressionPostAgg("p4", "'fo'"),
                       expressionPostAgg("p5", "'foo  '")
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -1318,7 +1341,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -2136,7 +2159,7 @@
                           new StringFirstAggregatorFactory("a5", "v2", 10)
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -2176,7 +2199,7 @@
                           new StringLastAggregatorFactory("a5", "v2", 10)
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -2218,7 +2241,7 @@
                           new StringAnyAggregatorFactory("a6", "v2", 10)
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         NullHandling.sqlCompatible() ? ImmutableList.of(new Object[]{1L, 1.0f, 1.0, "", 2L, 2.0f, "1"}) : ImmutableList.of(new Object[]{1L, 1.0f, 1.0, "", 2L, 2.0f, "1"})
@@ -2244,7 +2267,7 @@
                           new FloatAnyAggregatorFactory("a2", "f1")
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -2579,7 +2602,7 @@
                           new FloatFirstAggregatorFactory("a2", "f1")
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -2608,7 +2631,7 @@
                           new FloatLastAggregatorFactory("a2", "f1")
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -2652,7 +2675,7 @@
                           new FloatLastAggregatorFactory("a3", "f1")
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -2697,7 +2720,7 @@
                           new FloatAnyAggregatorFactory("a3", "f2")
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         // first row has null for l2, d2, f2 and dim1 as empty string (which is null in default mode)
@@ -2739,7 +2762,7 @@
                           new FloatAnyAggregatorFactory("a3", "f2")
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -3265,7 +3288,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.MONTH)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -3372,7 +3395,7 @@
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
                   .filters(selector("m1", "1.0", null))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -3393,7 +3416,7 @@
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
                   .filters(selector("m2", "1.0", null))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -3855,21 +3878,21 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build(),
             Druids.newTimeseriesQueryBuilder()
                   .dataSource(CalciteTests.DATASOURCE1)
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build(),
             Druids.newTimeseriesQueryBuilder()
                   .dataSource(CalciteTests.DATASOURCE1)
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{6L}, new Object[]{6L}, new Object[]{6L})
@@ -3889,14 +3912,14 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build(),
             Druids.newTimeseriesQueryBuilder()
                   .dataSource(CalciteTests.DATASOURCE1)
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{6L}, new Object[]{6L})
@@ -4381,7 +4404,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
                   .withOverriddenContext(queryContext),
             Druids.newTimeseriesQueryBuilder()
@@ -4389,7 +4412,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
                   .withOverriddenContext(queryContext)
         ),
@@ -4415,7 +4438,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
                   .withOverriddenContext(queryContext),
             Druids.newTimeseriesQueryBuilder()
@@ -4430,7 +4453,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
                   .withOverriddenContext(queryContext)
         ),
@@ -4463,7 +4486,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build(),
             Druids.newTimeseriesQueryBuilder()
                   .dataSource(
@@ -4477,7 +4500,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{6L}, new Object[]{1L})
@@ -4503,7 +4526,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new DoubleSumAggregatorFactory("a0", "m1")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{21.0})
@@ -4530,7 +4553,7 @@
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new DoubleSumAggregatorFactory("a0", "m1")))
                   .postAggregators(ImmutableList.of(expressionPostAgg("p0", "(\"a0\" / 10)")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{2.1})
@@ -4678,7 +4701,7 @@
                       )
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -4720,7 +4743,7 @@
                   .granularity(Granularities.ALL)
                   .filters(selector("l1", null, null))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -4758,7 +4781,7 @@
                   .granularity(Granularities.ALL)
                   .filters(selector("d1", null, null))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -4797,7 +4820,7 @@
                   .granularity(Granularities.ALL)
                   .filters(selector("f1", null, null))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -4940,7 +4963,7 @@
                   .granularity(Granularities.ALL)
                   .filters(bound("l1", "3", null, true, false, null, StringComparators.NUMERIC))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{2L})
@@ -4961,7 +4984,7 @@
                   .granularity(Granularities.ALL)
                   .filters(bound("d1", "0", null, true, false, null, StringComparators.NUMERIC))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{2L})
@@ -4982,7 +5005,7 @@
                   .granularity(Granularities.ALL)
                   .filters(bound("f1", "0", null, true, false, null, StringComparators.NUMERIC))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{2L})
@@ -5004,7 +5027,7 @@
                     .granularity(Granularities.ALL)
                     .filters(in("dim2", ImmutableList.of("", "a"), null))
                     .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                    .context(TIMESERIES_CONTEXT_DEFAULT)
+                    .context(QUERY_CONTEXT_DEFAULT)
                     .build()
           ),
           ImmutableList.of(
@@ -5024,7 +5047,7 @@
                     .granularity(Granularities.ALL)
                     .filters(selector("dim2", "", null))
                     .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                    .context(TIMESERIES_CONTEXT_DEFAULT)
+                    .context(QUERY_CONTEXT_DEFAULT)
                     .build()
           ),
           ImmutableList.of(
@@ -5038,8 +5061,6 @@
   @Test
   public void testNullStringEquality() throws Exception
   {
-    // In Calcite 1.21, this query is optimized to return 0 without generating a native Druid query, since
-    // null is not equal to null or any other value.
     testQuery(
         "SELECT COUNT(*)\n"
         + "FROM druid.foo\n"
@@ -5121,7 +5142,7 @@
                   .granularity(Granularities.ALL)
                   .filters(selector("dim2", null, null))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5171,7 +5192,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5232,10 +5253,10 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
-        ImmutableList.of()
+        ImmutableList.of(new Object[]{0L})
     );
   }
 
@@ -5408,10 +5429,12 @@
                   .intervals(querySegmentSpec())
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
-        ImmutableList.of()
+        ImmutableList.of(
+            new Object[]{0L}
+        )
     );
   }
 
@@ -5464,9 +5487,6 @@
   @Test
   public void testGroupByWithFilterMatchingNothing() throws Exception
   {
-    // This query should actually return [0, null] rather than an empty result set, but it doesn't.
-    // This test just "documents" the current behavior.
-
     testQuery(
         "SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE dim1 = 'foobar'",
         ImmutableList.of(
@@ -5479,10 +5499,12 @@
                       new CountAggregatorFactory("a0"),
                       new LongMaxAggregatorFactory("a1", "cnt")
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
-        ImmutableList.of()
+        ImmutableList.of(
+            new Object[]{0L, NullHandling.sqlCompatible() ? null : Long.MIN_VALUE}
+        )
     );
   }
 
@@ -5501,7 +5523,7 @@
                       new LongSumAggregatorFactory("a1", "cnt"),
                       new LongMinAggregatorFactory("a2", "cnt")
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{6L, 6L, 1L})
@@ -5523,10 +5545,12 @@
                       new CountAggregatorFactory("a0"),
                       new LongMaxAggregatorFactory("a1", "cnt")
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
-        ImmutableList.of()
+        ImmutableList.of(
+            new Object[]{0L, NullHandling.sqlCompatible() ? null : Long.MIN_VALUE}
+        )
     );
   }
 
@@ -5550,7 +5574,7 @@
                           )
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5575,7 +5599,7 @@
                           not(selector("dim2", null, null))
                       )
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         NullHandling.replaceWithDefault() ?
@@ -5604,7 +5628,7 @@
                           in("dim2", ImmutableList.of("abc", "def"), null)
                       )
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5624,7 +5648,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5649,7 +5673,7 @@
                   ))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5673,7 +5697,7 @@
                   ))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5697,7 +5721,7 @@
                   ))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5748,7 +5772,7 @@
                   .filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(withLeftDirectAccessEnabled(TIMESERIES_CONTEXT_DEFAULT))
+                  .context(withLeftDirectAccessEnabled(QUERY_CONTEXT_DEFAULT))
                   .build()
         ),
         ImmutableList.of(
@@ -5775,7 +5799,7 @@
         + "\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],"
         + "\"postAggregations\":[],"
         + "\"limit\":2147483647,"
-        + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}]"
+        + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}]"
         + ", signature=[{a0:LONG}])\n";
 
     final String resources = "[{\"name\":\"aview\",\"type\":\"VIEW\"}]";
@@ -5806,7 +5830,7 @@
                       )
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5832,7 +5856,7 @@
                       )
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5855,10 +5879,12 @@
                       bound("cnt", "1.1", "100000001.0", true, true, null, StringComparators.NUMERIC)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
-        ImmutableList.of()
+        ImmutableList.of(
+            new Object[]{0L}
+        )
     );
 
     testQuery(
@@ -5872,7 +5898,7 @@
                       selector("cnt", "1.0", null)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5891,10 +5917,12 @@
                       selector("cnt", "100000001.0", null)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
-        ImmutableList.of()
+        ImmutableList.of(
+            new Object[]{0L}
+        )
     );
 
     testQuery(
@@ -5908,7 +5936,7 @@
                       in("cnt", ImmutableList.of("1.0", "100000001.0"), null)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5929,7 +5957,7 @@
                   .granularity(Granularities.ALL)
                   .filters(in("cnt", ImmutableList.of("1", "2"), null))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -5991,7 +6019,7 @@
                                 new LongMinAggregatorFactory("a1", "cnt"),
                                 new LongMaxAggregatorFactory("a2", "l1")
                             ))
-                .context(TIMESERIES_CONTEXT_DEFAULT)
+                .context(QUERY_CONTEXT_DEFAULT)
                 .build()
         ),
         ImmutableList.of(
@@ -6014,7 +6042,7 @@
                       new DoubleMinAggregatorFactory("a0", "d1"),
                       new DoubleMaxAggregatorFactory("a1", "d1")
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6037,7 +6065,7 @@
                       new FloatMinAggregatorFactory("a0", "m1"),
                       new FloatMaxAggregatorFactory("a1", "m1")
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6133,7 +6161,7 @@
                           useDefault ? "((\"a3\" + \"a4\") + \"a5\")" : "((\"a4\" + \"a5\") + \"a6\")"
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         NullHandling.replaceWithDefault() ?
@@ -6354,7 +6382,7 @@
                           and(selector("dim2", "a", null), selector("dim1", "b", null))
                       )
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         NullHandling.replaceWithDefault() ?
@@ -6427,7 +6455,7 @@
                           )
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         NullHandling.replaceWithDefault() ?
@@ -6472,7 +6500,7 @@
                       expressionPostAgg("p0", "log((\"a1\" + \"a2\"))"),
                       expressionPostAgg("p1", "(\"a1\" % 4)")
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6712,7 +6740,7 @@
                       selector("dim2", "a", null)
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6740,10 +6768,12 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
-        ImmutableList.of()
+        ImmutableList.of(
+            new Object[]{0L}
+        )
     );
   }
 
@@ -6759,7 +6789,7 @@
                   .granularity(Granularities.ALL)
                   .filters(bound("m1", "2.5", "3.5", true, true, null, StringComparators.NUMERIC))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6780,7 +6810,7 @@
                   .granularity(Granularities.ALL)
                   .filters(bound("dim1", "a", "b", false, true, null, StringComparators.LEXICOGRAPHIC))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6801,7 +6831,7 @@
                   .granularity(Granularities.ALL)
                   .filters(selector("dim1", "abc", null))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6822,7 +6852,7 @@
                   .granularity(Granularities.ALL)
                   .filters(numericSelector("dim1", "2", null))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6843,7 +6873,7 @@
                   .intervals(querySegmentSpec(Intervals.of("2000-01-01/2001-01-01")))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6871,7 +6901,7 @@
                   .intervals(querySegmentSpec(Intervals.of("2000/2001"), Intervals.of("2010/2011")))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6898,7 +6928,7 @@
                   )
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6929,7 +6959,7 @@
                   )
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6972,7 +7002,7 @@
                   .intervals(querySegmentSpec(Intervals.of("2000-01-01/2000-01-01T00:00:00.001")))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -6998,7 +7028,7 @@
                   )
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7038,7 +7068,7 @@
                       )
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7079,7 +7109,7 @@
                   )
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7109,7 +7139,7 @@
                   .filters(not(selector("dim1", "xxx", null)))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7132,7 +7162,7 @@
                   .filters(not(selector("dim2", "a", null)))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7168,7 +7198,7 @@
                   )
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7201,7 +7231,7 @@
                       )
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7234,7 +7264,7 @@
                       )
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7267,7 +7297,7 @@
                       )
                   )
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7294,7 +7324,7 @@
                           CalciteTests.createExprMacroTable()
                       )
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7324,7 +7354,7 @@
                           CalciteTests.createExprMacroTable()
                       )
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7677,7 +7707,7 @@
                           new HyperUniquesAggregatorFactory("a2", "unique_dim1", false, true)
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7728,7 +7758,7 @@
                           )
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7801,7 +7831,7 @@
                           )
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -7997,7 +8027,7 @@
                           new HyperUniquesAggregatorFactory("a5", "unique_dim1", false, true)
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         NullHandling.replaceWithDefault() ?
@@ -8034,7 +8064,7 @@
                           )
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(new Object[]{6L})
@@ -8332,7 +8362,7 @@
                                       .granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC))
                                       .intervals(querySegmentSpec(Filtration.eternity()))
                                       .aggregators(new CountAggregatorFactory("a0"))
-                                      .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                                      .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                                       .build()
                             )
                         )
@@ -8417,7 +8447,7 @@
                                               new HyperUniqueFinalizingPostAggregator("a0", "a0:a")
                                           )
                                       )
-                                      .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                                      .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                                       .build()
                             )
                         )
@@ -8754,7 +8784,7 @@
     skipVectorize();
 
     final String explanation =
-        "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n"
+        "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n"
         + "  DruidJoinQueryRel(condition=[=(SUBSTRING($3, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false}], signature=[{d0:STRING}])\n"
         + "    DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"order\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"
         + "    DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null,\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false}], signature=[{d0:STRING}])\n";
@@ -9088,7 +9118,7 @@
                       expressionPostAgg("p2", "((\"a0\" / \"a1\") + 3)"),
                       expressionPostAgg("p3", "((CAST(\"a0\", 'DOUBLE') / CAST(\"a1\", 'DOUBLE')) + 3)")
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9128,7 +9158,7 @@
                           )
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9165,7 +9195,7 @@
                           )
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9283,7 +9313,7 @@
                       )
                   )
                   .aggregators(new CountAggregatorFactory("a0"))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9314,7 +9344,7 @@
                       )
                   )
                   .aggregators(new CountAggregatorFactory("a0"))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9473,7 +9503,7 @@
                   .intervals(querySegmentSpec(Intervals.of("2000/P2M")))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9500,7 +9530,7 @@
                   .intervals(querySegmentSpec(Intervals.of("2000-01-01T00-08:00/2000-03-01T00-08:00")))
                   .granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzFromString(LOS_ANGELES)))
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -9529,7 +9559,7 @@
                   .intervals(querySegmentSpec(Intervals.of("2000-01-01T01:02/2002")))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9597,7 +9627,7 @@
                   .intervals(querySegmentSpec(Intervals.of("2000-01-02T00Z/2002-01-01T08Z")))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_LOS_ANGELES)
+                  .context(QUERY_CONTEXT_LOS_ANGELES)
                   .build()
         ),
         ImmutableList.of(
@@ -9617,7 +9647,7 @@
                   .intervals(querySegmentSpec(Intervals.of("2000-01-02/2002")))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9643,7 +9673,7 @@
                   .intervals(querySegmentSpec(Intervals.of("2000-01-02T00Z/2002-01-01T08Z")))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_LOS_ANGELES)
+                  .context(QUERY_CONTEXT_LOS_ANGELES)
                   .build()
         ),
         ImmutableList.of(
@@ -9668,7 +9698,7 @@
                   ))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9690,7 +9720,7 @@
                   .intervals(querySegmentSpec(new Interval(DateTimes.MIN, DateTimes.of("2000-02-01"))))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9712,7 +9742,7 @@
                   .intervals(querySegmentSpec(new Interval(DateTimes.MIN, DateTimes.of("2000-03-01"))))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9747,7 +9777,7 @@
                           selector("v1", "1", null)
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9790,7 +9820,7 @@
                           in("v1", ImmutableList.of("2", "3", "5"), null)
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9815,7 +9845,7 @@
         + "AND EXTRACT(DECADE FROM __time) = 200\n"
         + "AND EXTRACT(CENTURY FROM __time) = 20\n"
         + "AND EXTRACT(MILLENNIUM FROM __time) = 2\n",
-        TIMESERIES_CONTEXT_DEFAULT,
+        QUERY_CONTEXT_DEFAULT,
         ImmutableList.of(
             Druids.newTimeseriesQueryBuilder()
                   .dataSource(CalciteTests.DATASOURCE4)
@@ -9852,7 +9882,7 @@
                           selector("v7", "2", null)
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -9873,10 +9903,10 @@
                   .intervals(querySegmentSpec())
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
-        ImmutableList.of()
+        ImmutableList.of(new Object[]{0L})
     );
   }
 
@@ -10994,7 +11024,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(new CountAggregatorFactory("a0"))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -11050,7 +11080,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(new CountAggregatorFactory("a0"))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -11346,10 +11376,12 @@
                       expressionFilter("(\"cnt\" == CAST(\"j0.k\", 'LONG'))"),
                       expressionFilter("(CAST(\"j0.k\", 'LONG') == \"_j0.cnt\")")
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
-        ImmutableList.of()
+        ImmutableList.of(
+            new Object[]{0L}
+        )
     );
   }
 
@@ -11411,10 +11443,12 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(new CountAggregatorFactory("a0"))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
-        ImmutableList.of()
+        ImmutableList.of(
+            new Object[]{0L}
+        )
     );
   }
 
@@ -11855,7 +11889,7 @@
                           true
                       )
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -11897,7 +11931,7 @@
                           true
                       )
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -11977,7 +12011,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.MONTH)
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -12045,7 +12079,7 @@
                           )
                       )
                   ))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -12107,7 +12141,7 @@
                   .intervals(querySegmentSpec(Intervals.of("1999-12-01T00-08:00/2002-01-01T00-08:00")))
                   .granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzFromString(LOS_ANGELES)))
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -12135,7 +12169,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.MONTH)
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -12266,7 +12300,7 @@
                       )
                   )
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -12294,7 +12328,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzFromString(LOS_ANGELES)))
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -12406,7 +12440,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC))
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -12436,7 +12470,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(new PeriodGranularity(Period.months(3), null, DateTimeZone.UTC))
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -12466,7 +12500,7 @@
                   .granularity(Granularities.MONTH)
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
                   .descending(true)
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -12477,6 +12511,355 @@
   }
 
   @Test
+  public void testTimeseriesEmptyResultsAggregatorDefaultValues() throws Exception
+  {
+    // timeseries with all granularity have a single group, so should return default results for given aggregators
+    testQuery(
+        "SELECT\n"
+        + " count(*),\n"
+        + " COUNT(DISTINCT dim1),\n"
+        + " APPROX_COUNT_DISTINCT(distinct dim1),\n"
+        + " sum(d1),\n"
+        + " max(d1),\n"
+        + " min(d1),\n"
+        + " sum(l1),\n"
+        + " max(l1),\n"
+        + " min(l1),\n"
+        + " avg(l1),\n"
+        + " avg(d1)\n"
+        + "FROM druid.numfoo WHERE dim2 = 0",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(querySegmentSpec(Filtration.eternity()))
+                  .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
+                  .granularity(Granularities.ALL)
+                  .aggregators(
+                      aggregators(
+                          new CountAggregatorFactory("a0"),
+
+                          new CardinalityAggregatorFactory(
+                              "a1",
+                              null,
+                              ImmutableList.of(DefaultDimensionSpec.of("dim1")),
+                              false,
+                              true
+                          ),
+                          new CardinalityAggregatorFactory(
+                              "a2",
+                              null,
+                              ImmutableList.of(DefaultDimensionSpec.of("dim1")),
+                              false,
+                              true
+                          ),
+                          new DoubleSumAggregatorFactory("a3", "d1"),
+                          new DoubleMaxAggregatorFactory("a4", "d1"),
+                          new DoubleMinAggregatorFactory("a5", "d1"),
+                          new LongSumAggregatorFactory("a6", "l1"),
+                          new LongMaxAggregatorFactory("a7", "l1"),
+                          new LongMinAggregatorFactory("a8", "l1"),
+                          new LongSumAggregatorFactory("a9:sum", "l1"),
+                          useDefault
+                          ? new CountAggregatorFactory("a9:count")
+                          : new FilteredAggregatorFactory(
+                              new CountAggregatorFactory("a9:count"),
+                              not(selector("l1", null, null))
+                          ),
+                          new DoubleSumAggregatorFactory("a10:sum", "d1"),
+                          useDefault
+                          ? new CountAggregatorFactory("a10:count")
+                          : new FilteredAggregatorFactory(
+                              new CountAggregatorFactory("a10:count"),
+                              not(selector("d1", null, null))
+                          )
+                      )
+                  )
+                  .postAggregators(
+                      new ArithmeticPostAggregator(
+                          "a9",
+                          "quotient",
+                          ImmutableList.of(
+                              new FieldAccessPostAggregator(null, "a9:sum"),
+                              new FieldAccessPostAggregator(null, "a9:count")
+                          )
+                      ),
+                      new ArithmeticPostAggregator(
+                          "a10",
+                          "quotient",
+                          ImmutableList.of(
+                              new FieldAccessPostAggregator(null, "a10:sum"),
+                              new FieldAccessPostAggregator(null, "a10:count")
+                          )
+                      )
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            useDefault
+            ? new Object[]{0L, 0L, 0L, 0.0, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, 0L, Long.MIN_VALUE, Long.MAX_VALUE, 0L, Double.NaN}
+            : new Object[]{0L, 0L, 0L, null, null, null, null, null, null, null, null}
+        )
+    );
+  }
+
+  @Test
+  public void testTimeseriesEmptyResultsAggregatorDefaultValuesNonVectorized() throws Exception
+  {
+    cannotVectorize();
+    // timeseries with all granularity have a single group, so should return default results for given aggregators
+    testQuery(
+        "SELECT\n"
+        + " ANY_VALUE(dim1, 1024),\n"
+        + " ANY_VALUE(l1),\n"
+        + " EARLIEST(dim1, 1024),\n"
+        + " EARLIEST(l1),\n"
+        + " LATEST(dim1, 1024),\n"
+        + " LATEST(l1),\n"
+        + " ARRAY_AGG(DISTINCT dim3)\n"
+        + "FROM druid.numfoo WHERE dim2 = 0",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE3)
+                  .intervals(querySegmentSpec(Filtration.eternity()))
+                  .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
+                  .granularity(Granularities.ALL)
+                  .aggregators(
+                      aggregators(
+                          new StringAnyAggregatorFactory("a0", "dim1", 1024),
+                          new LongAnyAggregatorFactory("a1", "l1"),
+                          new StringFirstAggregatorFactory("a2", "dim1", 1024),
+                          new LongFirstAggregatorFactory("a3", "l1"),
+                          new StringLastAggregatorFactory("a4", "dim1", 1024),
+                          new LongLastAggregatorFactory("a5", "l1"),
+                          new ExpressionLambdaAggregatorFactory(
+                              "a6",
+                              ImmutableSet.of("dim3"),
+                              "__acc",
+                              "[]",
+                              "[]",
+                              "array_set_add(\"__acc\", \"dim3\")",
+                              "array_set_add_all(\"__acc\", \"a6\")",
+                              null,
+                              "if(array_length(o) == 0, null, o)",
+                              new HumanReadableBytes(1024),
+                              TestExprMacroTable.INSTANCE
+                          )
+                      )
+                  )
+                  .context(QUERY_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of(
+            useDefault
+            ? new Object[]{"", 0L, "", 0L, "", 0L, null}
+            : new Object[]{null, null, null, null, null, null, null}
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByAggregatorDefaultValues() throws Exception
+  {
+    testQuery(
+        "SELECT\n"
+        + " dim2,\n"
+        + " count(*) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " COUNT(DISTINCT dim1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " APPROX_COUNT_DISTINCT(distinct dim1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " sum(d1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " max(d1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " min(d1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " sum(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " max(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " min(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " avg(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " avg(d1) FILTER(WHERE dim1 = 'nonexistent')\n"
+        + "FROM druid.numfoo WHERE dim2 = 'a' GROUP BY dim2",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE3)
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setDimFilter(selector("dim2", "a", null))
+                        .setGranularity(Granularities.ALL)
+                        .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING))
+                        .setDimensions(new DefaultDimensionSpec("v0", "_d0", ValueType.STRING))
+                        .setAggregatorSpecs(
+                            aggregators(
+                                new FilteredAggregatorFactory(
+                                    new CountAggregatorFactory("a0"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new CardinalityAggregatorFactory("a1", null, ImmutableList.of(DefaultDimensionSpec.of("dim1")), false, true),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new CardinalityAggregatorFactory("a2", null, ImmutableList.of(DefaultDimensionSpec.of("dim1")), false, true),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new DoubleSumAggregatorFactory("a3", "d1"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new DoubleMaxAggregatorFactory("a4", "d1"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new DoubleMinAggregatorFactory("a5", "d1"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new LongSumAggregatorFactory("a6", "l1"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new LongMaxAggregatorFactory("a7", "l1"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new LongMinAggregatorFactory("a8", "l1"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new LongSumAggregatorFactory("a9:sum", "l1"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                useDefault
+                                ? new FilteredAggregatorFactory(
+                                    new CountAggregatorFactory("a9:count"),
+                                    selector("dim1", "nonexistent", null)
+                                )
+                                : new FilteredAggregatorFactory(
+                                    new CountAggregatorFactory("a9:count"),
+                                    and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null))
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new DoubleSumAggregatorFactory("a10:sum", "d1"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                useDefault
+                                ? new FilteredAggregatorFactory(
+                                    new CountAggregatorFactory("a10:count"),
+                                    selector("dim1", "nonexistent", null)
+                                )
+                                : new FilteredAggregatorFactory(
+                                    new CountAggregatorFactory("a10:count"),
+                                    and(not(selector("d1", null, null)), selector("dim1", "nonexistent", null))
+                                )
+                            )
+                        )
+                        .setPostAggregatorSpecs(
+                            ImmutableList.of(
+                                new ArithmeticPostAggregator(
+                                    "a9",
+                                    "quotient",
+                                    ImmutableList.of(new FieldAccessPostAggregator(
+                                        null,
+                                        "a9:sum"
+                                    ), new FieldAccessPostAggregator(null, "a9:count"))
+                                ),
+                                new ArithmeticPostAggregator(
+                                    "a10",
+                                    "quotient",
+                                    ImmutableList.of(new FieldAccessPostAggregator(
+                                        null,
+                                        "a10:sum"
+                                    ), new FieldAccessPostAggregator(null, "a10:count"))
+                                )
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            useDefault
+            ? new Object[]{"a", 0L, 0L, 0L, 0.0, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, 0L, Long.MIN_VALUE, Long.MAX_VALUE, 0L, Double.NaN}
+            : new Object[]{"a", 0L, 0L, 0L, null, null, null, null, null, null, null, null}
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByAggregatorDefaultValuesNonVectorized() throws Exception
+  {
+    cannotVectorize();
+    testQuery(
+        "SELECT\n"
+        + " dim2,\n"
+        + " ANY_VALUE(dim1, 1024) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " ANY_VALUE(l1) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " EARLIEST(dim1, 1024) FILTER(WHERE dim1 = 'nonexistent'),\n"
+        + " 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')"
+        + "FROM druid.numfoo WHERE dim2 = 'a' GROUP BY dim2",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE3)
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setDimFilter(selector("dim2", "a", null))
+                        .setGranularity(Granularities.ALL)
+                        .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING))
+                        .setDimensions(new DefaultDimensionSpec("v0", "_d0", ValueType.STRING))
+                        .setAggregatorSpecs(
+                            aggregators(
+                                new FilteredAggregatorFactory(
+                                    new StringAnyAggregatorFactory("a0", "dim1", 1024),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new LongAnyAggregatorFactory("a1", "l1"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new StringFirstAggregatorFactory("a2", "dim1", 1024),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new LongFirstAggregatorFactory("a3", "l1"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new StringLastAggregatorFactory("a4", "dim1", 1024),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new LongLastAggregatorFactory("a5", "l1"),
+                                    selector("dim1", "nonexistent", null)
+                                ),
+                                new FilteredAggregatorFactory(
+                                    new ExpressionLambdaAggregatorFactory(
+                                        "a6",
+                                        ImmutableSet.of("dim3"),
+                                        "__acc",
+                                        "[]",
+                                        "[]",
+                                        "array_set_add(\"__acc\", \"dim3\")",
+                                        "array_set_add_all(\"__acc\", \"a6\")",
+                                        null,
+                                        "if(array_length(o) == 0, null, o)",
+                                        new HumanReadableBytes(1024),
+                                        TestExprMacroTable.INSTANCE
+                                    ),
+                                    selector("dim1", "nonexistent", null)
+                                )
+                            )
+                        )
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            useDefault
+            ? new Object[]{"a", "", 0L, "", 0L, "", 0L, null}
+            : new Object[]{"a", null, null, null, null, null, null, null}
+        )
+    );
+  }
+
+  @Test
   public void testGroupByExtractYear() throws Exception
   {
     // Cannot vectorize due to virtual columns.
@@ -12669,7 +13052,7 @@
                   .granularity(Granularities.MONTH)
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
                   .limit(1)
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -12696,7 +13079,7 @@
                   .granularity(Granularities.MONTH)
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
                   .limit(1)
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -12761,7 +13144,7 @@
                   .granularity(Granularities.MONTH)
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
                   .limit(1)
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -13637,7 +14020,7 @@
                                               .filters(new LikeDimFilter("dim1", "%bc", null, null))
                                               .granularity(Granularities.ALL)
                                               .aggregators(new CountAggregatorFactory("a0"))
-                                              .context(TIMESERIES_CONTEXT_DEFAULT)
+                                              .context(QUERY_CONTEXT_DEFAULT)
                                               .build()
                                     ),
                                     "j0.",
@@ -14052,7 +14435,7 @@
                                               .granularity(Granularities.ALL)
                                               .filters(selector("cnt", "1", null))
                                               .aggregators(new LongMaxAggregatorFactory("a0", "__time"))
-                                              .context(TIMESERIES_CONTEXT_DEFAULT)
+                                              .context(QUERY_CONTEXT_DEFAULT)
                                               .build()
                                     ),
                                     "j0.",
@@ -14066,7 +14449,7 @@
                                           .granularity(Granularities.ALL)
                                           .filters(not(selector("cnt", "2", null)))
                                           .aggregators(new LongMaxAggregatorFactory("a0", "__time"))
-                                          .context(TIMESERIES_CONTEXT_DEFAULT)
+                                          .context(QUERY_CONTEXT_DEFAULT)
                                           .build()
                                 ),
                                 "_j0.",
@@ -14112,7 +14495,7 @@
                                                   .intervals(querySegmentSpec(Filtration.eternity()))
                                                   .granularity(Granularities.ALL)
                                                   .aggregators(new LongMaxAggregatorFactory("a0", "__time"))
-                                                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                                                  .context(QUERY_CONTEXT_DEFAULT)
                                                   .build()
                                         ),
                                         "j0.",
@@ -14130,7 +14513,7 @@
                                                                   .aggregators(
                                                                       new LongMinAggregatorFactory("a0", "__time")
                                                                   )
-                                                                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                                                                  .context(QUERY_CONTEXT_DEFAULT)
                                                                   .build()
                                                         )
                                                     )
@@ -14159,7 +14542,7 @@
                                           .granularity(Granularities.ALL)
                                           .aggregators(new LongMinAggregatorFactory("a0", "__time"))
                                           .postAggregators(expressionPostAgg("p0", "1"))
-                                          .context(TIMESERIES_CONTEXT_DEFAULT)
+                                          .context(QUERY_CONTEXT_DEFAULT)
                                           .build()
                                 ),
                                 "__j0.",
@@ -14213,7 +14596,7 @@
                                               .intervals(querySegmentSpec(Filtration.eternity()))
                                               .granularity(Granularities.ALL)
                                               .aggregators(new LongMaxAggregatorFactory("a0", "__time"))
-                                              .context(TIMESERIES_CONTEXT_DEFAULT)
+                                              .context(QUERY_CONTEXT_DEFAULT)
                                               .build()
                                     ),
                                     "j0.",
@@ -14226,7 +14609,7 @@
                                           .intervals(querySegmentSpec(Filtration.eternity()))
                                           .granularity(Granularities.ALL)
                                           .aggregators(new LongMinAggregatorFactory("a0", "__time"))
-                                          .context(TIMESERIES_CONTEXT_DEFAULT)
+                                          .context(QUERY_CONTEXT_DEFAULT)
                                           .build()
                                 ),
                                 "_j0.",
@@ -14352,7 +14735,7 @@
                                           .intervals(querySegmentSpec(Filtration.eternity()))
                                           .granularity(Granularities.ALL)
                                           .aggregators(new LongMaxAggregatorFactory("a0", "__time"))
-                                          .context(TIMESERIES_CONTEXT_DEFAULT)
+                                          .context(QUERY_CONTEXT_DEFAULT)
                                           .build()
                                           .withOverriddenContext(queryContext)
                                 ),
@@ -14402,7 +14785,7 @@
                                                       .intervals(querySegmentSpec(Filtration.eternity()))
                                                       .granularity(Granularities.ALL)
                                                       .aggregators(new LongMaxAggregatorFactory("a0", "__time"))
-                                                      .context(TIMESERIES_CONTEXT_DEFAULT)
+                                                      .context(QUERY_CONTEXT_DEFAULT)
                                                       .build()
                                             )
                                             .setInterval(querySegmentSpec(Filtration.eternity()))
@@ -14430,7 +14813,7 @@
                                           .granularity(Granularities.ALL)
                                           .aggregators(new LongMaxAggregatorFactory("a0", "__time"))
                                           .postAggregators(expressionPostAgg("p0", "1"))
-                                          .context(TIMESERIES_CONTEXT_DEFAULT)
+                                          .context(QUERY_CONTEXT_DEFAULT)
                                           .build()
                                 ),
                                 "_j0.",
@@ -14632,7 +15015,7 @@
                   .intervals(querySegmentSpec(Filtration.eternity()))
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -14917,7 +15300,7 @@
                       expressionPostAgg("p0", "(\"a0\" + \"a1\")")
                   )
                   .descending(true)
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -15186,7 +15569,7 @@
                   .intervals(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01")))
                   .granularity(Granularities.MONTH)
                   .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0"))
+                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
                   .build()
         ),
         ImmutableList.of(
@@ -15299,7 +15682,7 @@
                   )
                   .granularity(Granularities.ALL)
                   .aggregators(aggregators(new CountAggregatorFactory("a0")))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -15436,7 +15819,7 @@
   {
     testQuery(
         PLANNER_CONFIG_DEFAULT,
-        QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS,
+        QUERY_CONTEXT_DEFAULT,
         "SELECT exp(count(*)) + 10, sin(pi / 6), cos(pi / 6), tan(pi / 6), cot(pi / 6)," +
         "asin(exp(count(*)) / 2), acos(exp(count(*)) / 2), atan(exp(count(*)) / 2), atan2(exp(count(*)), 1) " +
         "FROM druid.foo WHERE  dim2 = 0",
@@ -15462,7 +15845,7 @@
                                    expressionPostAgg("p7", "atan((exp(\"a0\") / 2))"),
                                    expressionPostAgg("p8", "atan2(exp(\"a0\"),1)")
                                )
-                               .context(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS)
+                               .context(QUERY_CONTEXT_DEFAULT)
                                .build()),
         ImmutableList.of(
             new Object[]{
@@ -17981,7 +18364,7 @@
                           )
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -18033,7 +18416,7 @@
                           )
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -18137,7 +18520,7 @@
                           )
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -18192,7 +18575,7 @@
                       )
                   )
                   .postAggregators(expressionPostAgg("p0", "array_to_string(\"a0\",',')"))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -18233,7 +18616,7 @@
                       )
                   )
                   .postAggregators(expressionPostAgg("p0", "array_to_string(\"a0\",',')"))
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -18283,7 +18666,7 @@
                           )
                       )
                   )
-                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .context(QUERY_CONTEXT_DEFAULT)
                   .build()
         ),
         ImmutableList.of(
@@ -18440,7 +18823,7 @@
                                             )
                                         )
                                     )
-                                    .context(TIMESERIES_CONTEXT_DEFAULT)
+                                    .context(QUERY_CONTEXT_DEFAULT)
                                     .build()
                           ),
                           "j0.",
@@ -18516,7 +18899,7 @@
                                                   )
                                               )
                                           )
-                                          .context(TIMESERIES_CONTEXT_DEFAULT)
+                                          .context(QUERY_CONTEXT_DEFAULT)
                                           .build()
                                 ),
                                 "j0.",
diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java
index 4b44bc4..dd0cee1 100644
--- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java
@@ -721,7 +721,7 @@
             ImmutableMap.<String, Object>of(
                 "PLAN",
                 StringUtils.format(
-                    "DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"skipEmptyBuckets\":true,\"sqlQueryId\":\"%s\"}}], signature=[{a0:LONG}])\n",
+                    "DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"sqlQueryId\":\"%s\"}}], signature=[{a0:LONG}])\n",
                     DUMMY_SQL_QUERY_ID
                 ),
                 "RESOURCES",