Add more sketch aggregator support in Druid SQL (#6951)

* Add more sketch aggregator support in Druid SQL

* Add docs

* Tweak module serde register

* Fix tests

* Checkstyle

* Test fix

* PR comment

* PR comment

* PR comments
This commit is contained in:
Jonathan Wei 2019-02-02 22:34:53 -08:00 committed by Clint Wylie
parent 7baa33049c
commit 953b96d0a4
26 changed files with 2908 additions and 259 deletions

View File

@ -119,7 +119,11 @@ Only the COUNT aggregation can accept DISTINCT.
|`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". See also `COUNT(DISTINCT expr)`.|
|`APPROX_QUANTILE(expr, probability, [resolution])`|Computes approximate quantiles on numeric or approxHistogram 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.html) must be loaded to use this function.|
|`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.html) 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-extensions.html) 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.html) 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-extensions.html) must be loaded to use this function.|
|`APPROX_QUANTILE(expr, probability, [resolution])`|Computes approximate quantiles on numeric or [approxHistogram](../development/extensions-core/approximate-histograms.html#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.html) 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.html) 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-extensions.html) 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.html#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.html) 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 positve rate increases. See [bloom filter extension](../development/extensions-core/bloom-filter.html) documentation for additional details.|
### Numeric functions

View File

@ -56,6 +56,12 @@
<version>${project.parent.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-sql</artifactId>
<version>${project.parent.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
@ -131,6 +137,20 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-server</artifactId>
<version>${project.parent.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-sql</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
</dependencies>
<build>

View File

@ -39,11 +39,11 @@ import java.util.Objects;
* Base class for both build and merge factories
* @author Alexander Saydakov
*/
abstract class HllSketchAggregatorFactory extends AggregatorFactory
public abstract class HllSketchAggregatorFactory extends AggregatorFactory
{
static final int DEFAULT_LG_K = 12;
static final TgtHllType DEFAULT_TGT_HLL_TYPE = TgtHllType.HLL_4;
public static final int DEFAULT_LG_K = 12;
public static final TgtHllType DEFAULT_TGT_HLL_TYPE = TgtHllType.HLL_4;
static final Comparator<HllSketch> COMPARATOR =
Comparator.nullsFirst(Comparator.comparingDouble(HllSketch::getEstimate));

View File

@ -22,10 +22,13 @@ package org.apache.druid.query.aggregation.datasketches.hll;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Binder;
import com.yahoo.sketches.hll.HllSketch;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchSqlAggregator;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.sql.guice.SqlBindings;
import java.util.Collections;
import java.util.List;
@ -48,15 +51,8 @@ public class HllSketchModule implements DruidModule
@Override
public void configure(final Binder binder)
{
if (ComplexMetrics.getSerdeForType(TYPE_NAME) == null) {
ComplexMetrics.registerSerde(TYPE_NAME, new HllSketchMergeComplexMetricSerde());
}
if (ComplexMetrics.getSerdeForType(BUILD_TYPE_NAME) == null) {
ComplexMetrics.registerSerde(BUILD_TYPE_NAME, new HllSketchBuildComplexMetricSerde());
}
if (ComplexMetrics.getSerdeForType(MERGE_TYPE_NAME) == null) {
ComplexMetrics.registerSerde(MERGE_TYPE_NAME, new HllSketchMergeComplexMetricSerde());
}
registerSerde();
SqlBindings.addAggregator(binder, HllSketchSqlAggregator.class);
}
@Override
@ -74,4 +70,17 @@ public class HllSketchModule implements DruidModule
);
}
@VisibleForTesting
public static void registerSerde()
{
if (ComplexMetrics.getSerdeForType(TYPE_NAME) == null) {
ComplexMetrics.registerSerde(TYPE_NAME, new HllSketchMergeComplexMetricSerde());
}
if (ComplexMetrics.getSerdeForType(BUILD_TYPE_NAME) == null) {
ComplexMetrics.registerSerde(BUILD_TYPE_NAME, new HllSketchBuildComplexMetricSerde());
}
if (ComplexMetrics.getSerdeForType(MERGE_TYPE_NAME) == null) {
ComplexMetrics.registerSerde(MERGE_TYPE_NAME, new HllSketchMergeComplexMetricSerde());
}
}
}

View File

@ -0,0 +1,202 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation.datasketches.hll.sql;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.InferTypes;
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchMergeAggregatorFactory;
import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.table.RowSignature;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
public class HllSketchSqlAggregator implements SqlAggregator
{
private static final SqlAggFunction FUNCTION_INSTANCE = new HllSketchSqlAggFunction();
private static final String NAME = "APPROX_COUNT_DISTINCT_DS_HLL";
@Override
public SqlAggFunction calciteFunction()
{
return FUNCTION_INSTANCE;
}
@Nullable
@Override
public Aggregation toDruidAggregation(
PlannerContext plannerContext,
RowSignature rowSignature,
RexBuilder rexBuilder,
String name,
AggregateCall aggregateCall,
Project project,
List<Aggregation> existingAggregations,
boolean finalizeAggregations
)
{
// Don't use Aggregations.getArgumentsForSimpleAggregator, since it won't let us use direct column access
// for string columns.
final RexNode columnRexNode = Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(0)
);
final DruidExpression columnArg = Expressions.toDruidExpression(plannerContext, rowSignature, columnRexNode);
if (columnArg == null) {
return null;
}
final int logK;
if (aggregateCall.getArgList().size() >= 2) {
final RexNode logKarg = Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(1)
);
if (!logKarg.isA(SqlKind.LITERAL)) {
// logK must be a literal in order to plan.
return null;
}
logK = ((Number) RexLiteral.value(logKarg)).intValue();
} else {
logK = HllSketchAggregatorFactory.DEFAULT_LG_K;
}
final String tgtHllType;
if (aggregateCall.getArgList().size() >= 3) {
final RexNode tgtHllTypeArg = Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(2)
);
if (!tgtHllTypeArg.isA(SqlKind.LITERAL)) {
// tgtHllType must be a literal in order to plan.
return null;
}
tgtHllType = RexLiteral.stringValue(tgtHllTypeArg);
} else {
tgtHllType = HllSketchAggregatorFactory.DEFAULT_TGT_HLL_TYPE.name();
}
final List<VirtualColumn> virtualColumns = new ArrayList<>();
final AggregatorFactory aggregatorFactory;
final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name;
if (columnArg.isDirectColumnAccess() && rowSignature.getColumnType(columnArg.getDirectColumn()) == ValueType.COMPLEX) {
aggregatorFactory = new HllSketchMergeAggregatorFactory(aggregatorName, columnArg.getDirectColumn(), logK, tgtHllType);
} else {
final SqlTypeName sqlTypeName = columnRexNode.getType().getSqlTypeName();
final ValueType inputType = Calcites.getValueTypeForSqlTypeName(sqlTypeName);
if (inputType == null) {
throw new ISE("Cannot translate sqlTypeName[%s] to Druid type for field[%s]", sqlTypeName, aggregatorName);
}
final DimensionSpec dimensionSpec;
if (columnArg.isDirectColumnAccess()) {
dimensionSpec = columnArg.getSimpleExtraction().toDimensionSpec(null, inputType);
} else {
final ExpressionVirtualColumn virtualColumn = columnArg.toVirtualColumn(
Calcites.makePrefixedName(name, "v"),
inputType,
plannerContext.getExprMacroTable()
);
dimensionSpec = new DefaultDimensionSpec(virtualColumn.getOutputName(), null, inputType);
virtualColumns.add(virtualColumn);
}
aggregatorFactory = new HllSketchBuildAggregatorFactory(
aggregatorName,
dimensionSpec.getDimension(),
logK,
tgtHllType
);
}
return Aggregation.create(
virtualColumns,
Collections.singletonList(aggregatorFactory),
finalizeAggregations ? new FinalizingFieldAccessPostAggregator(
name,
aggregatorFactory.getName()
) : null
);
}
private static class HllSketchSqlAggFunction extends SqlAggFunction
{
private static final String SIGNATURE = "'" + NAME + "(column, lgK, tgtHllType)'\n";
HllSketchSqlAggFunction()
{
super(
NAME,
null,
SqlKind.OTHER_FUNCTION,
ReturnTypes.explicit(SqlTypeName.BIGINT),
InferTypes.VARCHAR_1024,
OperandTypes.or(
OperandTypes.ANY,
OperandTypes.and(
OperandTypes.sequence(SIGNATURE, OperandTypes.ANY, OperandTypes.LITERAL, OperandTypes.LITERAL),
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING)
)
),
SqlFunctionCategory.NUMERIC,
false,
false
);
}
}
}

View File

@ -49,7 +49,7 @@ public class DoublesSketchAggregatorFactory extends AggregatorFactory
public static final Comparator<DoublesSketch> COMPARATOR =
Comparator.nullsFirst(Comparator.comparingLong(DoublesSketch::getN));
private static final int DEFAULT_K = 128;
public static final int DEFAULT_K = 128;
// Used for sketch size estimation.
private static final long MAX_STREAM_LENGTH = 1_000_000_000;

View File

@ -22,10 +22,13 @@ package org.apache.druid.query.aggregation.datasketches.quantiles;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Binder;
import com.yahoo.sketches.quantiles.DoublesSketch;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchSqlAggregator;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.sql.guice.SqlBindings;
import java.util.Collections;
import java.util.List;
@ -44,9 +47,8 @@ public class DoublesSketchModule implements DruidModule
@Override
public void configure(final Binder binder)
{
if (ComplexMetrics.getSerdeForType(DOUBLES_SKETCH) == null) {
ComplexMetrics.registerSerde(DOUBLES_SKETCH, new DoublesSketchComplexMetricSerde());
}
registerSerde();
SqlBindings.addAggregator(binder, DoublesSketchSqlAggregator.class);
}
@Override
@ -65,4 +67,11 @@ public class DoublesSketchModule implements DruidModule
);
}
@VisibleForTesting
public static void registerSerde()
{
if (ComplexMetrics.getSerdeForType(DOUBLES_SKETCH) == null) {
ComplexMetrics.registerSerde(DOUBLES_SKETCH, new DoublesSketchComplexMetricSerde());
}
}
}

View File

@ -0,0 +1,237 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation.datasketches.quantiles.sql;
import com.google.common.collect.ImmutableList;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToQuantilePostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.table.RowSignature;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.List;
public class DoublesSketchSqlAggregator implements SqlAggregator
{
private static final SqlAggFunction FUNCTION_INSTANCE = new DoublesSketchSqlAggFunction();
private static final String NAME = "APPROX_QUANTILE_DS";
@Override
public SqlAggFunction calciteFunction()
{
return FUNCTION_INSTANCE;
}
@Nullable
@Override
public Aggregation toDruidAggregation(
final PlannerContext plannerContext,
final RowSignature rowSignature,
final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
final Project project,
final List<Aggregation> existingAggregations,
final boolean finalizeAggregations
)
{
final DruidExpression input = Expressions.toDruidExpression(
plannerContext,
rowSignature,
Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(0)
)
);
if (input == null) {
return null;
}
final AggregatorFactory aggregatorFactory;
final String histogramName = StringUtils.format("%s:agg", name);
final RexNode probabilityArg = Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(1)
);
if (!probabilityArg.isA(SqlKind.LITERAL)) {
// Probability must be a literal in order to plan.
return null;
}
final float probability = ((Number) RexLiteral.value(probabilityArg)).floatValue();
final int k;
if (aggregateCall.getArgList().size() >= 3) {
final RexNode resolutionArg = Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(2)
);
if (!resolutionArg.isA(SqlKind.LITERAL)) {
// Resolution must be a literal in order to plan.
return null;
}
k = ((Number) RexLiteral.value(resolutionArg)).intValue();
} else {
k = DoublesSketchAggregatorFactory.DEFAULT_K;
}
// Look for existing matching aggregatorFactory.
for (final Aggregation existing : existingAggregations) {
for (AggregatorFactory factory : existing.getAggregatorFactories()) {
if (factory instanceof DoublesSketchAggregatorFactory) {
final DoublesSketchAggregatorFactory theFactory = (DoublesSketchAggregatorFactory) factory;
// Check input for equivalence.
final boolean inputMatches;
final VirtualColumn virtualInput = existing.getVirtualColumns()
.stream()
.filter(
virtualColumn ->
virtualColumn.getOutputName()
.equals(theFactory.getFieldName())
)
.findFirst()
.orElse(null);
if (virtualInput == null) {
inputMatches = input.isDirectColumnAccess()
&& input.getDirectColumn().equals(theFactory.getFieldName());
} else {
inputMatches = ((ExpressionVirtualColumn) virtualInput).getExpression()
.equals(input.getExpression());
}
final boolean matches = inputMatches
&& theFactory.getK() == k;
if (matches) {
// Found existing one. Use this.
return Aggregation.create(
ImmutableList.of(),
new DoublesSketchToQuantilePostAggregator(
name,
new FieldAccessPostAggregator(
factory.getName(),
factory.getName()
),
probability
)
);
}
}
}
}
// No existing match found. Create a new one.
final List<VirtualColumn> virtualColumns = new ArrayList<>();
if (input.isDirectColumnAccess()) {
aggregatorFactory = new DoublesSketchAggregatorFactory(
histogramName,
input.getDirectColumn(),
k
);
} else {
final ExpressionVirtualColumn virtualColumn = input.toVirtualColumn(
StringUtils.format("%s:v", name),
ValueType.FLOAT,
plannerContext.getExprMacroTable()
);
virtualColumns.add(virtualColumn);
aggregatorFactory = new DoublesSketchAggregatorFactory(
histogramName,
virtualColumn.getOutputName(),
k
);
}
return Aggregation.create(
virtualColumns,
ImmutableList.of(aggregatorFactory),
new DoublesSketchToQuantilePostAggregator(
name,
new FieldAccessPostAggregator(
histogramName,
histogramName
),
probability
)
);
}
private static class DoublesSketchSqlAggFunction extends SqlAggFunction
{
private static final String SIGNATURE1 = "'" + NAME + "(column, probability)'\n";
private static final String SIGNATURE2 = "'" + NAME + "(column, probability, k)'\n";
DoublesSketchSqlAggFunction()
{
super(
NAME,
null,
SqlKind.OTHER_FUNCTION,
ReturnTypes.explicit(SqlTypeName.DOUBLE),
null,
OperandTypes.or(
OperandTypes.and(
OperandTypes.sequence(SIGNATURE1, OperandTypes.ANY, OperandTypes.LITERAL),
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC)
),
OperandTypes.and(
OperandTypes.sequence(SIGNATURE2, OperandTypes.ANY, OperandTypes.LITERAL, OperandTypes.LITERAL),
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.EXACT_NUMERIC)
)
),
SqlFunctionCategory.NUMERIC,
false,
false
);
}
}
}

View File

@ -25,6 +25,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
import org.apache.druid.query.aggregation.AggregatorUtil;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.List;
@ -38,10 +39,10 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory
public SketchMergeAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName,
@JsonProperty("size") Integer size,
@JsonProperty("shouldFinalize") Boolean shouldFinalize,
@JsonProperty("isInputThetaSketch") Boolean isInputThetaSketch,
@JsonProperty("errorBoundsStdDev") Integer errorBoundsStdDev
@Nullable @JsonProperty("size") Integer size,
@Nullable @JsonProperty("shouldFinalize") Boolean shouldFinalize,
@Nullable @JsonProperty("isInputThetaSketch") Boolean isInputThetaSketch,
@Nullable @JsonProperty("errorBoundsStdDev") Integer errorBoundsStdDev
)
{
super(name, fieldName, size, AggregatorUtil.SKETCH_MERGE_CACHE_TYPE_ID);

View File

@ -22,9 +22,12 @@ package org.apache.druid.query.aggregation.datasketches.theta;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Binder;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.query.aggregation.datasketches.theta.sql.ThetaSketchSqlAggregator;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.sql.guice.SqlBindings;
import java.util.Collections;
import java.util.List;
@ -44,17 +47,8 @@ public class SketchModule implements DruidModule
@Override
public void configure(Binder binder)
{
if (ComplexMetrics.getSerdeForType(THETA_SKETCH) == null) {
ComplexMetrics.registerSerde(THETA_SKETCH, new SketchMergeComplexMetricSerde());
}
if (ComplexMetrics.getSerdeForType(THETA_SKETCH_MERGE_AGG) == null) {
ComplexMetrics.registerSerde(THETA_SKETCH_MERGE_AGG, new SketchMergeComplexMetricSerde());
}
if (ComplexMetrics.getSerdeForType(THETA_SKETCH_BUILD_AGG) == null) {
ComplexMetrics.registerSerde(THETA_SKETCH_BUILD_AGG, new SketchBuildComplexMetricSerde());
}
registerSerde();
SqlBindings.addAggregator(binder, ThetaSketchSqlAggregator.class);
}
@Override
@ -71,4 +65,20 @@ public class SketchModule implements DruidModule
.addSerializer(SketchHolder.class, new SketchHolderJsonSerializer())
);
}
@VisibleForTesting
public static void registerSerde()
{
if (ComplexMetrics.getSerdeForType(THETA_SKETCH) == null) {
ComplexMetrics.registerSerde(THETA_SKETCH, new SketchMergeComplexMetricSerde());
}
if (ComplexMetrics.getSerdeForType(THETA_SKETCH_MERGE_AGG) == null) {
ComplexMetrics.registerSerde(THETA_SKETCH_MERGE_AGG, new SketchMergeComplexMetricSerde());
}
if (ComplexMetrics.getSerdeForType(THETA_SKETCH_BUILD_AGG) == null) {
ComplexMetrics.registerSerde(THETA_SKETCH_BUILD_AGG, new SketchBuildComplexMetricSerde());
}
}
}

View File

@ -0,0 +1,192 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation.datasketches.theta.sql;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.InferTypes;
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.theta.SketchAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory;
import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.table.RowSignature;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
public class ThetaSketchSqlAggregator implements SqlAggregator
{
private static final SqlAggFunction FUNCTION_INSTANCE = new ThetaSketchSqlAggFunction();
private static final String NAME = "APPROX_COUNT_DISTINCT_DS_THETA";
@Override
public SqlAggFunction calciteFunction()
{
return FUNCTION_INSTANCE;
}
@Nullable
@Override
public Aggregation toDruidAggregation(
PlannerContext plannerContext,
RowSignature rowSignature,
RexBuilder rexBuilder,
String name,
AggregateCall aggregateCall,
Project project,
List<Aggregation> existingAggregations,
boolean finalizeAggregations
)
{
// Don't use Aggregations.getArgumentsForSimpleAggregator, since it won't let us use direct column access
// for string columns.
final RexNode columnRexNode = Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(0)
);
final DruidExpression columnArg = Expressions.toDruidExpression(plannerContext, rowSignature, columnRexNode);
if (columnArg == null) {
return null;
}
final int sketchSize;
if (aggregateCall.getArgList().size() >= 2) {
final RexNode sketchSizeArg = Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(1)
);
if (!sketchSizeArg.isA(SqlKind.LITERAL)) {
// logK must be a literal in order to plan.
return null;
}
sketchSize = ((Number) RexLiteral.value(sketchSizeArg)).intValue();
} else {
sketchSize = SketchAggregatorFactory.DEFAULT_MAX_SKETCH_SIZE;
}
final List<VirtualColumn> virtualColumns = new ArrayList<>();
final AggregatorFactory aggregatorFactory;
final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name;
if (columnArg.isDirectColumnAccess() && rowSignature.getColumnType(columnArg.getDirectColumn()) == ValueType.COMPLEX) {
aggregatorFactory = new SketchMergeAggregatorFactory(
aggregatorName,
columnArg.getDirectColumn(),
sketchSize,
null,
null,
null
);
} else {
final SqlTypeName sqlTypeName = columnRexNode.getType().getSqlTypeName();
final ValueType inputType = Calcites.getValueTypeForSqlTypeName(sqlTypeName);
if (inputType == null) {
throw new ISE("Cannot translate sqlTypeName[%s] to Druid type for field[%s]", sqlTypeName, aggregatorName);
}
final DimensionSpec dimensionSpec;
if (columnArg.isDirectColumnAccess()) {
dimensionSpec = columnArg.getSimpleExtraction().toDimensionSpec(null, inputType);
} else {
final ExpressionVirtualColumn virtualColumn = columnArg.toVirtualColumn(
Calcites.makePrefixedName(name, "v"),
inputType,
plannerContext.getExprMacroTable()
);
dimensionSpec = new DefaultDimensionSpec(virtualColumn.getOutputName(), null, inputType);
virtualColumns.add(virtualColumn);
}
aggregatorFactory = new SketchMergeAggregatorFactory(
aggregatorName,
dimensionSpec.getDimension(),
sketchSize,
null,
null,
null
);
}
return Aggregation.create(
virtualColumns,
Collections.singletonList(aggregatorFactory),
finalizeAggregations ? new FinalizingFieldAccessPostAggregator(
name,
aggregatorFactory.getName()
) : null
);
}
private static class ThetaSketchSqlAggFunction extends SqlAggFunction
{
private static final String SIGNATURE = "'" + NAME + "(column, size)'\n";
ThetaSketchSqlAggFunction()
{
super(
NAME,
null,
SqlKind.OTHER_FUNCTION,
ReturnTypes.explicit(SqlTypeName.BIGINT),
InferTypes.VARCHAR_1024,
OperandTypes.or(
OperandTypes.ANY,
OperandTypes.and(
OperandTypes.sequence(SIGNATURE, OperandTypes.ANY, OperandTypes.LITERAL),
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC)
)
),
SqlFunctionCategory.NUMERIC,
false,
false
);
}
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.query.aggregation.datasketches.hll;
import org.apache.druid.data.input.Row;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.aggregation.AggregationTestHelper;
@ -48,10 +47,9 @@ public class HllSketchAggregatorTest
public HllSketchAggregatorTest(GroupByQueryConfig config)
{
DruidModule module = new HllSketchModule();
module.configure(null);
HllSketchModule.registerSerde();
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
module.getJacksonModules(), config, tempFolder);
new HllSketchModule().getJacksonModules(), config, tempFolder);
}
@Parameterized.Parameters(name = "{0}")

View File

@ -0,0 +1,393 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation.datasketches.hll.sql;
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.Pair;
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.Query;
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;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchMergeAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchModule;
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.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.segment.IndexBuilder;
import org.apache.druid.segment.QueryableIndex;
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.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.schema.DruidSchema;
import org.apache.druid.sql.calcite.schema.SystemSchema;
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.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.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
public class HllSketchSqlAggregatorTest extends CalciteTestBase
{
private static final String DATA_SOURCE = "foo";
private static QueryRunnerFactoryConglomerate conglomerate;
private static Closer resourceCloser;
private static AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
private static final Map<String, Object> QUERY_CONTEXT_DEFAULT = ImmutableMap.of(
PlannerContext.CTX_SQL_QUERY_ID, "dummy"
);
@BeforeClass
public static void setUpClass()
{
final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerateCloserPair = CalciteTests
.createQueryRunnerFactoryConglomerate();
conglomerate = conglomerateCloserPair.lhs;
resourceCloser = conglomerateCloserPair.rhs;
}
@AfterClass
public static void tearDownClass() throws IOException
{
resourceCloser.close();
}
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@Rule
public QueryLogHook queryLogHook = QueryLogHook.create();
private SpecificSegmentsQuerySegmentWalker walker;
private SqlLifecycleFactory sqlLifecycleFactory;
@Before
public void setUp() throws Exception
{
HllSketchModule.registerSerde();
for (Module mod : new HllSketchModule().getJacksonModules()) {
CalciteTests.getJsonMapper().registerModule(mod);
}
final QueryableIndex index = IndexBuilder.create()
.tmpDir(temporaryFolder.newFolder())
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.schema(
new IncrementalIndexSchema.Builder()
.withMetrics(
new CountAggregatorFactory("cnt"),
new DoubleSumAggregatorFactory("m1", "m1"),
new HllSketchBuildAggregatorFactory(
"hllsketch_dim1",
"dim1",
null,
null
)
)
.withRollup(false)
.build()
)
.rows(CalciteTests.ROWS1)
.buildMMappedIndex();
walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
DataSegment.builder()
.dataSource(DATA_SOURCE)
.interval(index.getDataInterval())
.version("1")
.shardSpec(new LinearShardSpec(0))
.build(),
index
);
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
final DruidOperatorTable operatorTable = new DruidOperatorTable(
ImmutableSet.of(new HllSketchSqlAggregator()),
ImmutableSet.of()
);
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
new PlannerFactory(
druidSchema,
systemSchema,
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
operatorTable,
CalciteTests.createExprMacroTable(),
plannerConfig,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper()
)
);
}
@After
public void tearDown() throws Exception
{
walker.close();
walker = null;
}
@Test
public void testApproxCountDistinctHllSketch() throws Exception
{
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, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
final List<Object[]> expectedResults;
if (NullHandling.replaceWithDefault()) {
expectedResults = ImmutableList.of(
new Object[]{
6L,
2L,
2L,
1L,
2L,
5L,
5L
}
);
} else {
expectedResults = ImmutableList.of(
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(
"a3:v",
"substring(\"dim2\", 0, 1)",
ValueType.STRING,
TestExprMacroTable.INSTANCE
),
new ExpressionVirtualColumn(
"a4:v",
"concat(substring(\"dim2\", 0, 1),'x')",
ValueType.STRING,
TestExprMacroTable.INSTANCE
)
)
.aggregators(
ImmutableList.of(
new LongSumAggregatorFactory("a0", "cnt"),
new HllSketchBuildAggregatorFactory(
"a1",
"dim2",
null,
null
),
new FilteredAggregatorFactory(
new HllSketchBuildAggregatorFactory(
"a2",
"dim2",
null,
null
),
BaseCalciteQueryTest.NOT(BaseCalciteQueryTest.SELECTOR("dim2", "", null))
),
new HllSketchBuildAggregatorFactory(
"a3",
"a3:v",
null,
null
),
new HllSketchBuildAggregatorFactory(
"a4",
"a4:v",
null,
null
),
new HllSketchMergeAggregatorFactory("a5", "hllsketch_dim1", 21, "HLL_8"),
new HllSketchMergeAggregatorFactory("a6", "hllsketch_dim1", null, null)
)
)
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
}
@Test
public void testAvgDailyCountDistinctHllSketch() throws Exception
{
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 FROM druid.foo GROUP BY 1)";
// Verify results
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, 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(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(
Filtration.eternity())))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
new ExpressionVirtualColumn(
"d0:v",
"timestamp_floor(\"__time\",'P1D',null,'UTC')",
ValueType.LONG,
TestExprMacroTable.INSTANCE
)
)
.setDimensions(
Collections.singletonList(
new DefaultDimensionSpec(
"d0:v",
"d0",
ValueType.LONG
)
)
)
.setAggregatorSpecs(
Collections.singletonList(
new HllSketchBuildAggregatorFactory(
"a0:a",
"cnt",
null,
null
)
)
)
.setPostAggregatorSpecs(
ImmutableList.of(
new FinalizingFieldAccessPostAggregator("a0", "a0:a")
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(Arrays.asList(
new LongSumAggregatorFactory("_a0:sum", "a0"),
new CountAggregatorFactory("_a0:count")
))
.setPostAggregatorSpecs(
ImmutableList.of(
new ArithmeticPostAggregator(
"_a0",
"quotient",
ImmutableList.of(
new FieldAccessPostAggregator(null, "_a0:sum"),
new FieldAccessPostAggregator(null, "_a0:count")
)
)
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build();
Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
// Verify query
Assert.assertEquals(expected, actual);
}
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.query.aggregation.datasketches.quantiles;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.data.input.Row;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
@ -55,8 +54,8 @@ public class DoublesSketchAggregatorTest
public DoublesSketchAggregatorTest(final GroupByQueryConfig config)
{
DruidModule module = new DoublesSketchModule();
module.configure(null);
DoublesSketchModule.registerSerde();
DoublesSketchModule module = new DoublesSketchModule();
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
module.getJacksonModules(), config, tempFolder);
timeSeriesHelper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper(

View File

@ -0,0 +1,408 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation.datasketches.quantiles.sql;
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.Pair;
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.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;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToQuantilePostAggregator;
import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.expression.TestExprMacroTable;
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.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.segment.IndexBuilder;
import org.apache.druid.segment.QueryableIndex;
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.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.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.schema.DruidSchema;
import org.apache.druid.sql.calcite.schema.SystemSchema;
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.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.List;
import java.util.Map;
public class DoublesSketchSqlAggregatorTest extends CalciteTestBase
{
private static final String DATA_SOURCE = "foo";
private static QueryRunnerFactoryConglomerate conglomerate;
private static Closer resourceCloser;
private static AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
private static final Map<String, Object> QUERY_CONTEXT_DEFAULT = ImmutableMap.of(
PlannerContext.CTX_SQL_QUERY_ID, "dummy"
);
@BeforeClass
public static void setUpClass()
{
final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerateCloserPair = CalciteTests
.createQueryRunnerFactoryConglomerate();
conglomerate = conglomerateCloserPair.lhs;
resourceCloser = conglomerateCloserPair.rhs;
}
@AfterClass
public static void tearDownClass() throws IOException
{
resourceCloser.close();
}
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@Rule
public QueryLogHook queryLogHook = QueryLogHook.create();
private SpecificSegmentsQuerySegmentWalker walker;
private SqlLifecycleFactory sqlLifecycleFactory;
@Before
public void setUp() throws Exception
{
DoublesSketchModule.registerSerde();
for (Module mod : new DoublesSketchModule().getJacksonModules()) {
CalciteTests.getJsonMapper().registerModule(mod);
}
final QueryableIndex index = IndexBuilder.create()
.tmpDir(temporaryFolder.newFolder())
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.schema(
new IncrementalIndexSchema.Builder()
.withMetrics(
new CountAggregatorFactory("cnt"),
new DoubleSumAggregatorFactory("m1", "m1"),
new DoublesSketchAggregatorFactory(
"qsketch_m1",
"m1",
128
)
)
.withRollup(false)
.build()
)
.rows(CalciteTests.ROWS1)
.buildMMappedIndex();
walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
DataSegment.builder()
.dataSource(DATA_SOURCE)
.interval(index.getDataInterval())
.version("1")
.shardSpec(new LinearShardSpec(0))
.build(),
index
);
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
final DruidOperatorTable operatorTable = new DruidOperatorTable(
ImmutableSet.of(new DoublesSketchSqlAggregator()),
ImmutableSet.of()
);
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
new PlannerFactory(
druidSchema,
systemSchema,
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
operatorTable,
CalciteTests.createExprMacroTable(),
plannerConfig,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper()
)
);
}
@After
public void tearDown() throws Exception
{
walker.close();
walker = null;
}
@Test
public void testQuantileOnFloatAndLongs() throws Exception
{
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
final String sql = "SELECT\n"
+ "APPROX_QUANTILE_DS(m1, 0.01),\n"
+ "APPROX_QUANTILE_DS(m1, 0.5, 64),\n"
+ "APPROX_QUANTILE_DS(m1, 0.98, 256),\n"
+ "APPROX_QUANTILE_DS(m1, 0.99),\n"
+ "APPROX_QUANTILE_DS(m1 * 2, 0.97),\n"
+ "APPROX_QUANTILE_DS(m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n"
+ "APPROX_QUANTILE_DS(m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n"
+ "APPROX_QUANTILE_DS(m1, 0.999) FILTER(WHERE dim1 = 'abc'),\n"
+ "APPROX_QUANTILE_DS(cnt, 0.5)\n"
+ "FROM foo";
// Verify results
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
final List<Object[]> expectedResults = ImmutableList.of(
new Object[]{
1.0,
4.0,
6.0,
6.0,
12.0,
6.0,
5.0,
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(
"a4:v",
"(\"m1\" * 2)",
ValueType.FLOAT,
TestExprMacroTable.INSTANCE
)
)
.aggregators(ImmutableList.of(
new DoublesSketchAggregatorFactory("a0:agg", "m1", null),
new DoublesSketchAggregatorFactory("a1:agg", "m1", 64),
new DoublesSketchAggregatorFactory("a2:agg", "m1", 256),
new DoublesSketchAggregatorFactory("a4:agg", "a4:v", null),
new FilteredAggregatorFactory(
new DoublesSketchAggregatorFactory("a5:agg", "m1", null),
new SelectorDimFilter("dim1", "abc", null)
),
new FilteredAggregatorFactory(
new DoublesSketchAggregatorFactory("a6:agg", "m1", null),
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
),
new DoublesSketchAggregatorFactory("a8:agg", "cnt", null)
))
.postAggregators(
new DoublesSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.01f),
new DoublesSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a1:agg"), 0.50f),
new DoublesSketchToQuantilePostAggregator("a2", makeFieldAccessPostAgg("a2:agg"), 0.98f),
new DoublesSketchToQuantilePostAggregator("a3", makeFieldAccessPostAgg("a0:agg"), 0.99f),
new DoublesSketchToQuantilePostAggregator("a4", makeFieldAccessPostAgg("a4:agg"), 0.97f),
new DoublesSketchToQuantilePostAggregator("a5", makeFieldAccessPostAgg("a5:agg"), 0.99f),
new DoublesSketchToQuantilePostAggregator("a6", makeFieldAccessPostAgg("a6:agg"), 0.999f),
new DoublesSketchToQuantilePostAggregator("a7", makeFieldAccessPostAgg("a5:agg"), 0.999f),
new DoublesSketchToQuantilePostAggregator("a8", makeFieldAccessPostAgg("a8:agg"), 0.50f)
)
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
}
@Test
public void testQuantileOnComplexColumn() throws Exception
{
SqlLifecycle lifecycle = sqlLifecycleFactory.factorize();
final String sql = "SELECT\n"
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.01),\n"
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.5, 64),\n"
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.98, 256),\n"
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.99),\n"
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n"
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n"
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.999) FILTER(WHERE dim1 = 'abc')\n"
+ "FROM foo";
// Verify results
final List<Object[]> results = lifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
final List<Object[]> expectedResults = ImmutableList.of(
new Object[]{
1.0,
4.0,
6.0,
6.0,
6.0,
5.0,
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 DoublesSketchAggregatorFactory("a0:agg", "qsketch_m1", null),
new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", 64),
new DoublesSketchAggregatorFactory("a2:agg", "qsketch_m1", 256),
new FilteredAggregatorFactory(
new DoublesSketchAggregatorFactory("a4:agg", "qsketch_m1", null),
new SelectorDimFilter("dim1", "abc", null)
),
new FilteredAggregatorFactory(
new DoublesSketchAggregatorFactory("a5:agg", "qsketch_m1", null),
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
)
))
.postAggregators(
new DoublesSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.01f),
new DoublesSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a1:agg"), 0.50f),
new DoublesSketchToQuantilePostAggregator("a2", makeFieldAccessPostAgg("a2:agg"), 0.98f),
new DoublesSketchToQuantilePostAggregator("a3", makeFieldAccessPostAgg("a0:agg"), 0.99f),
new DoublesSketchToQuantilePostAggregator("a4", makeFieldAccessPostAgg("a4:agg"), 0.99f),
new DoublesSketchToQuantilePostAggregator("a5", makeFieldAccessPostAgg("a5:agg"), 0.999f),
new DoublesSketchToQuantilePostAggregator("a6", makeFieldAccessPostAgg("a4:agg"), 0.999f)
)
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
}
@Test
public void testQuantileOnInnerQuery() throws Exception
{
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
final String sql = "SELECT AVG(x), APPROX_QUANTILE_DS(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, authenticationResult).toList();
final List<Object[]> expectedResults;
if (NullHandling.replaceWithDefault()) {
expectedResults = ImmutableList.of(new Object[]{7.0, 11.0});
} else {
expectedResults = ImmutableList.of(new Object[]{5.25, 8.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(
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(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build()
)
)
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(
new DoubleSumAggregatorFactory("_a0:sum", "a0"),
new CountAggregatorFactory("_a0:count"),
new DoublesSketchAggregatorFactory(
"_a1:agg",
"a0",
null
)
)
.setPostAggregatorSpecs(
ImmutableList.of(
new ArithmeticPostAggregator(
"_a0",
"quotient",
ImmutableList.of(
new FieldAccessPostAggregator(null, "_a0:sum"),
new FieldAccessPostAggregator(null, "_a0:count")
)
),
new DoublesSketchToQuantilePostAggregator("_a1", makeFieldAccessPostAgg("_a1:agg"), 0.98f)
)
)
.setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
}
private static PostAggregator makeFieldAccessPostAgg(String name)
{
return new FieldAccessPostAggregator(name, name);
}
}

View File

@ -71,10 +71,9 @@ public class SketchAggregationTest
public SketchAggregationTest(final GroupByQueryConfig config)
{
SketchModule sm = new SketchModule();
sm.configure(null);
SketchModule.registerSerde();
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
sm.getJacksonModules(),
new SketchModule().getJacksonModules(),
config,
tempFolder
);

View File

@ -83,8 +83,8 @@ public class SketchAggregationWithSimpleDataTest
@Before
public void setup() throws Exception
{
SketchModule.registerSerde();
sm = new SketchModule();
sm.configure(null);
try (
final AggregationTestHelper toolchest = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
sm.getJacksonModules(),
@ -264,8 +264,8 @@ public class SketchAggregationWithSimpleDataTest
@Test
public void testSimpleDataIngestAndSelectQuery() throws Exception
{
SketchModule.registerSerde();
SketchModule sm = new SketchModule();
sm.configure(null);
AggregationTestHelper selectQueryAggregationTestHelper = AggregationTestHelper.createSelectQueryAggregationTestHelper(
sm.getJacksonModules(),
tempFolder

View File

@ -0,0 +1,402 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation.datasketches.theta.sql;
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.Pair;
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.Query;
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;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.theta.SketchModule;
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.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.segment.IndexBuilder;
import org.apache.druid.segment.QueryableIndex;
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.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.schema.DruidSchema;
import org.apache.druid.sql.calcite.schema.SystemSchema;
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.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.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
public class ThetaSketchSqlAggregatorTest extends CalciteTestBase
{
private static final String DATA_SOURCE = "foo";
private static QueryRunnerFactoryConglomerate conglomerate;
private static Closer resourceCloser;
private static AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
private static final Map<String, Object> QUERY_CONTEXT_DEFAULT = ImmutableMap.of(
PlannerContext.CTX_SQL_QUERY_ID, "dummy"
);
@BeforeClass
public static void setUpClass()
{
final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerateCloserPair = CalciteTests
.createQueryRunnerFactoryConglomerate();
conglomerate = conglomerateCloserPair.lhs;
resourceCloser = conglomerateCloserPair.rhs;
}
@AfterClass
public static void tearDownClass() throws IOException
{
resourceCloser.close();
}
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@Rule
public QueryLogHook queryLogHook = QueryLogHook.create();
private SpecificSegmentsQuerySegmentWalker walker;
private SqlLifecycleFactory sqlLifecycleFactory;
@Before
public void setUp() throws Exception
{
SketchModule.registerSerde();
for (Module mod : new SketchModule().getJacksonModules()) {
CalciteTests.getJsonMapper().registerModule(mod);
}
final QueryableIndex index = IndexBuilder.create()
.tmpDir(temporaryFolder.newFolder())
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.schema(
new IncrementalIndexSchema.Builder()
.withMetrics(
new CountAggregatorFactory("cnt"),
new DoubleSumAggregatorFactory("m1", "m1"),
new SketchMergeAggregatorFactory(
"thetasketch_dim1",
"dim1",
null,
false,
false,
null
)
)
.withRollup(false)
.build()
)
.rows(CalciteTests.ROWS1)
.buildMMappedIndex();
walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
DataSegment.builder()
.dataSource(DATA_SOURCE)
.interval(index.getDataInterval())
.version("1")
.shardSpec(new LinearShardSpec(0))
.build(),
index
);
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
final DruidOperatorTable operatorTable = new DruidOperatorTable(
ImmutableSet.of(new ThetaSketchSqlAggregator()),
ImmutableSet.of()
);
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
new PlannerFactory(
druidSchema,
systemSchema,
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
operatorTable,
CalciteTests.createExprMacroTable(),
plannerConfig,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper()
)
);
}
@After
public void tearDown() throws Exception
{
walker.close();
walker = null;
}
@Test
public void testApproxCountDistinctThetaSketch() throws Exception
{
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, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
final List<Object[]> expectedResults;
if (NullHandling.replaceWithDefault()) {
expectedResults = ImmutableList.of(
new Object[]{
6L,
2L,
2L,
1L,
2L,
5L,
5L
}
);
} else {
expectedResults = ImmutableList.of(
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(
"a3:v",
"substring(\"dim2\", 0, 1)",
ValueType.STRING,
TestExprMacroTable.INSTANCE
),
new ExpressionVirtualColumn(
"a4:v",
"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
),
new FilteredAggregatorFactory(
new SketchMergeAggregatorFactory(
"a2",
"dim2",
null,
null,
null,
null
),
BaseCalciteQueryTest.NOT(BaseCalciteQueryTest.SELECTOR("dim2", "", null))
),
new SketchMergeAggregatorFactory(
"a3",
"a3:v",
null,
null,
null,
null
),
new SketchMergeAggregatorFactory(
"a4",
"a4:v",
null,
null,
null,
null
),
new SketchMergeAggregatorFactory("a5", "thetasketch_dim1", 32768, null, null, null),
new SketchMergeAggregatorFactory("a6", "thetasketch_dim1", null, null, null, null)
)
)
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
}
@Test
public void testAvgDailyCountDistinctThetaSketch() throws Exception
{
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, QUERY_CONTEXT_DEFAULT, 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(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(
Filtration.eternity())))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
new ExpressionVirtualColumn(
"d0:v",
"timestamp_floor(\"__time\",'P1D',null,'UTC')",
ValueType.LONG,
TestExprMacroTable.INSTANCE
)
)
.setDimensions(
Collections.singletonList(
new DefaultDimensionSpec(
"d0:v",
"d0",
ValueType.LONG
)
)
)
.setAggregatorSpecs(
Collections.singletonList(
new SketchMergeAggregatorFactory(
"a0:a",
"cnt",
null,
null,
null,
null
)
)
)
.setPostAggregatorSpecs(
ImmutableList.of(
new FinalizingFieldAccessPostAggregator("a0", "a0:a")
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(Arrays.asList(
new LongSumAggregatorFactory("_a0:sum", "a0"),
new CountAggregatorFactory("_a0:count")
))
.setPostAggregatorSpecs(
ImmutableList.of(
new ArithmeticPostAggregator(
"_a0",
"quotient",
ImmutableList.of(
new FieldAccessPostAggregator(null, "_a0:sum"),
new FieldAccessPostAggregator(null, "_a0:count")
)
)
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build();
Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
// Verify query
Assert.assertEquals(expected, actual);
}
}

View File

@ -21,9 +21,11 @@ package org.apache.druid.query.aggregation.histogram;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.query.aggregation.histogram.sql.FixedBucketsHistogramQuantileSqlAggregator;
import org.apache.druid.query.aggregation.histogram.sql.QuantileSqlAggregator;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.sql.guice.SqlBindings;
@ -55,6 +57,14 @@ public class ApproximateHistogramDruidModule implements DruidModule
@Override
public void configure(Binder binder)
{
registerSerde();
SqlBindings.addAggregator(binder, QuantileSqlAggregator.class);
SqlBindings.addAggregator(binder, FixedBucketsHistogramQuantileSqlAggregator.class);
}
@VisibleForTesting
public static void registerSerde()
{
if (ComplexMetrics.getSerdeForType("approximateHistogram") == null) {
ComplexMetrics.registerSerde("approximateHistogram", new ApproximateHistogramFoldingSerde());
@ -63,10 +73,5 @@ public class ApproximateHistogramDruidModule implements DruidModule
if (ComplexMetrics.getSerdeForType(FixedBucketsHistogramAggregator.TYPE_NAME) == null) {
ComplexMetrics.registerSerde(FixedBucketsHistogramAggregator.TYPE_NAME, new FixedBucketsHistogramSerde());
}
if (binder != null) {
// Binder is null in some tests.
SqlBindings.addAggregator(binder, QuantileSqlAggregator.class);
}
}
}

View File

@ -0,0 +1,322 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation.histogram.sql;
import com.google.common.collect.ImmutableList;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.histogram.FixedBucketsHistogram;
import org.apache.druid.query.aggregation.histogram.FixedBucketsHistogramAggregatorFactory;
import org.apache.druid.query.aggregation.histogram.QuantilePostAggregator;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.table.RowSignature;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.List;
public class FixedBucketsHistogramQuantileSqlAggregator implements SqlAggregator
{
private static final SqlAggFunction FUNCTION_INSTANCE = new FixedBucketsHistogramQuantileSqlAggFunction();
private static final String NAME = "APPROX_QUANTILE_FIXED_BUCKETS";
@Override
public SqlAggFunction calciteFunction()
{
return FUNCTION_INSTANCE;
}
@Nullable
@Override
public Aggregation toDruidAggregation(
PlannerContext plannerContext,
RowSignature rowSignature,
RexBuilder rexBuilder,
String name,
AggregateCall aggregateCall,
Project project,
List<Aggregation> existingAggregations,
boolean finalizeAggregations
)
{
final DruidExpression input = Expressions.toDruidExpression(
plannerContext,
rowSignature,
Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(0)
)
);
if (input == null) {
return null;
}
final AggregatorFactory aggregatorFactory;
final String histogramName = StringUtils.format("%s:agg", name);
final RexNode probabilityArg = Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(1)
);
if (!probabilityArg.isA(SqlKind.LITERAL)) {
// Probability must be a literal in order to plan.
return null;
}
final float probability = ((Number) RexLiteral.value(probabilityArg)).floatValue();
final int numBuckets;
if (aggregateCall.getArgList().size() >= 3) {
final RexNode numBucketsArg = Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(2)
);
if (!numBucketsArg.isA(SqlKind.LITERAL)) {
// Resolution must be a literal in order to plan.
return null;
}
numBuckets = ((Number) RexLiteral.value(numBucketsArg)).intValue();
} else {
return null;
}
final double lowerLimit;
if (aggregateCall.getArgList().size() >= 4) {
final RexNode lowerLimitArg = Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(3)
);
if (!lowerLimitArg.isA(SqlKind.LITERAL)) {
// Resolution must be a literal in order to plan.
return null;
}
lowerLimit = ((Number) RexLiteral.value(lowerLimitArg)).doubleValue();
} else {
return null;
}
final double upperLimit;
if (aggregateCall.getArgList().size() >= 5) {
final RexNode upperLimitArg = Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(4)
);
if (!upperLimitArg.isA(SqlKind.LITERAL)) {
// Resolution must be a literal in order to plan.
return null;
}
upperLimit = ((Number) RexLiteral.value(upperLimitArg)).doubleValue();
} else {
return null;
}
final FixedBucketsHistogram.OutlierHandlingMode outlierHandlingMode;
if (aggregateCall.getArgList().size() >= 6) {
final RexNode outlierHandlingModeArg = Expressions.fromFieldAccess(
rowSignature,
project,
aggregateCall.getArgList().get(5)
);
if (!outlierHandlingModeArg.isA(SqlKind.LITERAL)) {
// Resolution must be a literal in order to plan.
return null;
}
outlierHandlingMode = FixedBucketsHistogram.OutlierHandlingMode.fromString(
RexLiteral.stringValue(outlierHandlingModeArg)
);
} else {
outlierHandlingMode = FixedBucketsHistogram.OutlierHandlingMode.IGNORE;
}
// Look for existing matching aggregatorFactory.
for (final Aggregation existing : existingAggregations) {
for (AggregatorFactory factory : existing.getAggregatorFactories()) {
if (factory instanceof FixedBucketsHistogramAggregatorFactory) {
final FixedBucketsHistogramAggregatorFactory theFactory = (FixedBucketsHistogramAggregatorFactory) factory;
// Check input for equivalence.
final boolean inputMatches;
final VirtualColumn virtualInput = existing.getVirtualColumns()
.stream()
.filter(
virtualColumn ->
virtualColumn.getOutputName()
.equals(theFactory.getFieldName())
)
.findFirst()
.orElse(null);
if (virtualInput == null) {
inputMatches = input.isDirectColumnAccess()
&& input.getDirectColumn().equals(theFactory.getFieldName());
} else {
inputMatches = ((ExpressionVirtualColumn) virtualInput).getExpression()
.equals(input.getExpression());
}
final boolean matches = inputMatches
&& theFactory.getOutlierHandlingMode() == outlierHandlingMode
&& theFactory.getNumBuckets() == numBuckets
&& theFactory.getLowerLimit() == lowerLimit
&& theFactory.getUpperLimit() == upperLimit;
if (matches) {
// Found existing one. Use this.
return Aggregation.create(
ImmutableList.of(),
new QuantilePostAggregator(name, factory.getName(), probability)
);
}
}
}
}
// No existing match found. Create a new one.
final List<VirtualColumn> virtualColumns = new ArrayList<>();
if (input.isDirectColumnAccess()) {
aggregatorFactory = new FixedBucketsHistogramAggregatorFactory(
histogramName,
input.getDirectColumn(),
numBuckets,
lowerLimit,
upperLimit,
outlierHandlingMode
);
} else {
final ExpressionVirtualColumn virtualColumn = input.toVirtualColumn(
StringUtils.format("%s:v", name),
ValueType.FLOAT,
plannerContext.getExprMacroTable()
);
virtualColumns.add(virtualColumn);
aggregatorFactory = new FixedBucketsHistogramAggregatorFactory(
histogramName,
virtualColumn.getOutputName(),
numBuckets,
lowerLimit,
upperLimit,
outlierHandlingMode
);
}
return Aggregation.create(
virtualColumns,
ImmutableList.of(aggregatorFactory),
new QuantilePostAggregator(name, histogramName, probability)
);
}
private static class FixedBucketsHistogramQuantileSqlAggFunction extends SqlAggFunction
{
private static final String SIGNATURE1 =
"'"
+ NAME
+ "(column, probability, numBuckets, lowerLimit, upperLimit)'\n";
private static final String SIGNATURE2 =
"'"
+ NAME
+ "(column, probability, numBuckets, lowerLimit, upperLimit, outlierHandlingMode)'\n";
FixedBucketsHistogramQuantileSqlAggFunction()
{
super(
NAME,
null,
SqlKind.OTHER_FUNCTION,
ReturnTypes.explicit(SqlTypeName.DOUBLE),
null,
OperandTypes.or(
OperandTypes.and(
OperandTypes.sequence(
SIGNATURE1,
OperandTypes.ANY,
OperandTypes.LITERAL,
OperandTypes.LITERAL,
OperandTypes.LITERAL,
OperandTypes.LITERAL
),
OperandTypes.family(
SqlTypeFamily.ANY,
SqlTypeFamily.NUMERIC,
SqlTypeFamily.NUMERIC,
SqlTypeFamily.NUMERIC,
SqlTypeFamily.NUMERIC
)
),
OperandTypes.and(
OperandTypes.sequence(
SIGNATURE2,
OperandTypes.ANY,
OperandTypes.LITERAL,
OperandTypes.LITERAL,
OperandTypes.LITERAL,
OperandTypes.LITERAL,
OperandTypes.LITERAL
),
OperandTypes.family(
SqlTypeFamily.ANY,
SqlTypeFamily.NUMERIC,
SqlTypeFamily.NUMERIC,
SqlTypeFamily.NUMERIC,
SqlTypeFamily.NUMERIC,
SqlTypeFamily.STRING
)
)
),
SqlFunctionCategory.NUMERIC,
false,
false
);
}
}
}

View File

@ -52,10 +52,9 @@ public class ApproximateHistogramAggregationTest
public ApproximateHistogramAggregationTest(final GroupByQueryConfig config)
{
ApproximateHistogramDruidModule module = new ApproximateHistogramDruidModule();
module.configure(null);
ApproximateHistogramDruidModule.registerSerde();
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
Lists.newArrayList(module.getJacksonModules()),
Lists.newArrayList(new ApproximateHistogramDruidModule().getJacksonModules()),
config,
tempFolder
);

View File

@ -145,9 +145,7 @@ public class ApproximateHistogramGroupByQueryTest
{
this.factory = factory;
this.runner = runner;
//Note: this is needed in order to properly register the serde for Histogram.
new ApproximateHistogramDruidModule().configure(null);
ApproximateHistogramDruidModule.registerSerde();
}
@After

View File

@ -52,10 +52,9 @@ public class FixedBucketsHistogramAggregationTest
public FixedBucketsHistogramAggregationTest(final GroupByQueryConfig config)
{
ApproximateHistogramDruidModule module = new ApproximateHistogramDruidModule();
module.configure(null);
ApproximateHistogramDruidModule.registerSerde();
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
Lists.newArrayList(module.getJacksonModules()),
Lists.newArrayList(new ApproximateHistogramDruidModule().getJacksonModules()),
config,
tempFolder
);

View File

@ -146,8 +146,7 @@ public class FixedBucketsHistogramGroupByQueryTest
this.factory = factory;
this.runner = runner;
//Note: this is needed in order to properly register the serde for Histogram.
new ApproximateHistogramDruidModule().configure(null);
ApproximateHistogramDruidModule.registerSerde();
}
@After

View File

@ -0,0 +1,452 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation.histogram.sql;
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.Pair;
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.Query;
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;
import org.apache.druid.query.aggregation.histogram.ApproximateHistogramDruidModule;
import org.apache.druid.query.aggregation.histogram.FixedBucketsHistogram;
import org.apache.druid.query.aggregation.histogram.FixedBucketsHistogramAggregatorFactory;
import org.apache.druid.query.aggregation.histogram.QuantilePostAggregator;
import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.expression.TestExprMacroTable;
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.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.segment.IndexBuilder;
import org.apache.druid.segment.QueryableIndex;
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.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.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.schema.DruidSchema;
import org.apache.druid.sql.calcite.schema.SystemSchema;
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.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.List;
import java.util.Map;
public class FixedBucketsHistogramQuantileSqlAggregatorTest extends CalciteTestBase
{
private static final String DATA_SOURCE = "foo";
private static QueryRunnerFactoryConglomerate conglomerate;
private static Closer resourceCloser;
private static AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
private static final Map<String, Object> QUERY_CONTEXT_DEFAULT = ImmutableMap.of(
PlannerContext.CTX_SQL_QUERY_ID, "dummy"
);
@BeforeClass
public static void setUpClass()
{
final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerateCloserPair = CalciteTests
.createQueryRunnerFactoryConglomerate();
conglomerate = conglomerateCloserPair.lhs;
resourceCloser = conglomerateCloserPair.rhs;
}
@AfterClass
public static void tearDownClass() throws IOException
{
resourceCloser.close();
}
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@Rule
public QueryLogHook queryLogHook = QueryLogHook.create();
private SpecificSegmentsQuerySegmentWalker walker;
private SqlLifecycleFactory sqlLifecycleFactory;
@Before
public void setUp() throws Exception
{
ApproximateHistogramDruidModule.registerSerde();
for (Module mod : new ApproximateHistogramDruidModule().getJacksonModules()) {
CalciteTests.getJsonMapper().registerModule(mod);
}
final QueryableIndex index = IndexBuilder.create()
.tmpDir(temporaryFolder.newFolder())
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.schema(
new IncrementalIndexSchema.Builder()
.withMetrics(
new CountAggregatorFactory("cnt"),
new DoubleSumAggregatorFactory("m1", "m1"),
new FixedBucketsHistogramAggregatorFactory(
"fbhist_m1",
"m1",
20,
0,
10,
FixedBucketsHistogram.OutlierHandlingMode.IGNORE
)
)
.withRollup(false)
.build()
)
.rows(CalciteTests.ROWS1)
.buildMMappedIndex();
walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
DataSegment.builder()
.dataSource(DATA_SOURCE)
.interval(index.getDataInterval())
.version("1")
.shardSpec(new LinearShardSpec(0))
.build(),
index
);
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
final DruidOperatorTable operatorTable = new DruidOperatorTable(
ImmutableSet.of(new QuantileSqlAggregator(), new FixedBucketsHistogramQuantileSqlAggregator()),
ImmutableSet.of()
);
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
new PlannerFactory(
druidSchema,
systemSchema,
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
operatorTable,
CalciteTests.createExprMacroTable(),
plannerConfig,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper()
)
);
}
@After
public void tearDown() throws Exception
{
walker.close();
walker = null;
}
@Test
public void testQuantileOnFloatAndLongs() throws Exception
{
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
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";
// Verify results
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
final List<Object[]> expectedResults = ImmutableList.of(
new Object[]{
1.0299999713897705,
3.5,
6.440000057220459,
6.470000267028809,
12.40999984741211,
6.494999885559082,
5.497499942779541,
6.499499797821045,
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(
"a4:v",
"(\"m1\" * 2)",
ValueType.FLOAT,
TestExprMacroTable.INSTANCE
)
)
.aggregators(ImmutableList.of(
new FixedBucketsHistogramAggregatorFactory(
"a0:agg", "m1", 20, 0.0d, 10.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE
),
new FixedBucketsHistogramAggregatorFactory(
"a4:agg", "a4:v", 40, 0.0d, 20.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE
),
new FilteredAggregatorFactory(
new FixedBucketsHistogramAggregatorFactory(
"a5:agg", "m1", 20, 0.0d, 10.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE
),
new SelectorDimFilter("dim1", "abc", null)
),
new FilteredAggregatorFactory(
new FixedBucketsHistogramAggregatorFactory(
"a6:agg", "m1", 20, 0.0d, 10.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE
),
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
),
new FixedBucketsHistogramAggregatorFactory(
"a8:agg", "cnt", 20, 0.0d, 10.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE
)
))
.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(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build();
// Verify query
Assert.assertEquals(
expected,
actual
);
}
@Test
public void testQuantileOnComplexColumn() throws Exception
{
SqlLifecycle lifecycle = sqlLifecycleFactory.factorize();
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";
// Verify results
final List<Object[]> results = lifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
final List<Object[]> expectedResults = ImmutableList.of(
new Object[]{
1.0299999713897705,
3.5,
6.293333530426025,
6.470000267028809,
6.494999885559082,
5.497499942779541,
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
),
new FixedBucketsHistogramAggregatorFactory(
"a2:agg",
"fbhist_m1",
30,
0.0,
10.0,
FixedBucketsHistogram.OutlierHandlingMode.IGNORE
),
new FilteredAggregatorFactory(
new FixedBucketsHistogramAggregatorFactory(
"a4:agg",
"fbhist_m1",
20,
0.0,
10.0,
FixedBucketsHistogram.OutlierHandlingMode.IGNORE
),
new SelectorDimFilter("dim1", "abc", null)
),
new FilteredAggregatorFactory(
new FixedBucketsHistogramAggregatorFactory(
"a5:agg",
"fbhist_m1",
20,
0.0,
10.0,
FixedBucketsHistogram.OutlierHandlingMode.IGNORE
),
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(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build();
// Verify query
Assert.assertEquals(expected, actual);
}
@Test
public void testQuantileOnInnerQuery() throws Exception
{
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
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, authenticationResult).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(ImmutableMap.of(
PlannerContext.CTX_SQL_QUERY_ID,
"dummy"
))
.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
)
)
.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(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build();
// Verify query
Assert.assertEquals(expected, actual);
}
}

View File

@ -19,6 +19,7 @@
package org.apache.druid.query.aggregation.histogram.sql;
import com.fasterxml.jackson.databind.Module;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
@ -119,8 +120,10 @@ public class QuantileSqlAggregatorTest extends CalciteTestBase
@Before
public void setUp() throws Exception
{
// Note: this is needed in order to properly register the serde for Histogram.
new ApproximateHistogramDruidModule().configure(null);
ApproximateHistogramDruidModule.registerSerde();
for (Module mod : new ApproximateHistogramDruidModule().getJacksonModules()) {
CalciteTests.getJsonMapper().registerModule(mod);
}
final QueryableIndex index = IndexBuilder.create()
.tmpDir(temporaryFolder.newFolder())
@ -188,221 +191,210 @@ public class QuantileSqlAggregatorTest extends CalciteTestBase
public void testQuantileOnFloatAndLongs() throws Exception
{
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
try {
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, QUERY_CONTEXT_DEFAULT, authenticationResult).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));
}
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 query
Assert.assertEquals(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
.granularity(Granularities.ALL)
.virtualColumns(
new ExpressionVirtualColumn(
"a4:v",
"(\"m1\" * 2)",
ValueType.FLOAT,
TestExprMacroTable.INSTANCE
)
)
.aggregators(ImmutableList.of(
new ApproximateHistogramAggregatorFactory("a0:agg", "m1", null, null, null, null),
new ApproximateHistogramAggregatorFactory("a2:agg", "m1", 200, null, null, null),
new ApproximateHistogramAggregatorFactory("a4:agg", "a4:v", null, null, null, null),
new FilteredAggregatorFactory(
new ApproximateHistogramAggregatorFactory("a5:agg", "m1", null, null, null, null),
new SelectorDimFilter("dim1", "abc", null)
),
new FilteredAggregatorFactory(
new ApproximateHistogramAggregatorFactory("a6:agg", "m1", null, null, null, null),
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
),
new ApproximateHistogramAggregatorFactory("a8:agg", "cnt", null, null, null, 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.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(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
}
catch (Exception e) {
throw e;
// Verify results
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).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(
"a4:v",
"(\"m1\" * 2)",
ValueType.FLOAT,
TestExprMacroTable.INSTANCE
)
)
.aggregators(ImmutableList.of(
new ApproximateHistogramAggregatorFactory("a0:agg", "m1", null, null, null, null),
new ApproximateHistogramAggregatorFactory("a2:agg", "m1", 200, null, null, null),
new ApproximateHistogramAggregatorFactory("a4:agg", "a4:v", null, null, null, null),
new FilteredAggregatorFactory(
new ApproximateHistogramAggregatorFactory("a5:agg", "m1", null, null, null, null),
new SelectorDimFilter("dim1", "abc", null)
),
new FilteredAggregatorFactory(
new ApproximateHistogramAggregatorFactory("a6:agg", "m1", null, null, null, null),
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
),
new ApproximateHistogramAggregatorFactory("a8:agg", "cnt", null, null, null, 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.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(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
}
@Test
public void testQuantileOnComplexColumn() throws Exception
{
SqlLifecycle lifecycle = sqlLifecycleFactory.factorize();
try {
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";
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, QUERY_CONTEXT_DEFAULT, authenticationResult).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 results
final List<Object[]> results = lifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).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),
new ApproximateHistogramFoldingAggregatorFactory("a2:agg", "hist_m1", 200, null, null, null),
new FilteredAggregatorFactory(
new ApproximateHistogramFoldingAggregatorFactory("a4:agg", "hist_m1", null, null, null, null),
new SelectorDimFilter("dim1", "abc", null)
),
new FilteredAggregatorFactory(
new ApproximateHistogramFoldingAggregatorFactory("a5:agg", "hist_m1", null, null, null, null),
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(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
}
catch (Exception e) {
throw e;
}
// 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),
new ApproximateHistogramFoldingAggregatorFactory("a2:agg", "hist_m1", 200, null, null, null),
new FilteredAggregatorFactory(
new ApproximateHistogramFoldingAggregatorFactory("a4:agg", "hist_m1", null, null, null, null),
new SelectorDimFilter("dim1", "abc", null)
),
new FilteredAggregatorFactory(
new ApproximateHistogramFoldingAggregatorFactory("a5:agg", "hist_m1", null, null, null, null),
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(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
}
@Test
public void testQuantileOnInnerQuery() throws Exception
{
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
try {
final String sql = "SELECT AVG(x), APPROX_QUANTILE(x, 0.98)\n"
+ "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)";
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, authenticationResult).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 results
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).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")
)
)
.setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.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
))
.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(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
}
catch (Exception e) {
throw e;
}
// 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")
)
)
.setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.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
)
)
.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(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
}
}