Improved SQL support for floats and doubles. (#4598)

* Improved SQL support for floats and doubles.

- Use Druid FLOAT for SQL FLOAT, and Druid DOUBLE for SQL DOUBLE, REAL,
  and DECIMAL.
- Use float* aggregators when appropriate.
- Add tests involving both float and double columns.
- Adjust documentation accordingly.

* CR comments.

* Fix braces.
This commit is contained in:
Gian Merlino 2017-07-25 13:54:44 -07:00 committed by GitHub
parent ae6780f62a
commit d4ef0f6d94
11 changed files with 502 additions and 170 deletions

View File

@ -207,13 +207,13 @@ Additionally, some Druid features are not supported by the SQL language. Some un
## Data types and casts
Druid natively supports four main column types: "long" (64 bit signed int), "float" (32 bit float), "string" (UTF-8
encoded strings), and "complex" (catch-all for more exotic data types like hyperUnique and approxHistogram columns).
Timestamps (including the `__time` column) are stored as longs, with the value being the number of milliseconds since 1
January 1970 UTC.
Druid natively supports five basic column types: "long" (64 bit signed int), "float" (32 bit float), "double" (64 bit
float) "string" (UTF-8 encoded strings), and "complex" (catch-all for more exotic data types like hyperUnique and
approxHistogram columns). Timestamps (including the `__time` column) are stored as longs, with the value being the
number of milliseconds since 1 January 1970 UTC.
At runtime, Druid will widen floats to "double" (64 bit float) for certain features, like `SUM` aggregators. But this
widening is not universal; some floating point operations retain 32 bit precision.
At runtime, Druid may widen 32-bit floats to 64-bit for certain operators, like SUM aggregators. The reverse will not
happen: 64-bit floats are not be narrowed to 32-bit.
Druid generally treats NULLs and empty strings interchangeably, rather than according to the SQL standard. As such,
Druid SQL only has partial support for NULLs. For example, the expressions `col IS NULL` and `col = ''` are equivalent,
@ -238,10 +238,10 @@ converted to zeroes).
|--------|------------------|-------------|-----|
|CHAR|STRING|`''`||
|VARCHAR|STRING|`''`|Druid STRING columns are reported as VARCHAR|
|DECIMAL|FLOAT or DOUBLE|`0.0`|DECIMAL uses floating point, not fixed point math|
|FLOAT|FLOAT or DOUBLE|`0.0`|Druid FLOAT columns are reported as FLOAT|
|REAL|FLOAT or DOUBLE|`0.0`||
|DOUBLE|FLOAT or DOUBLE|`0.0`||
|DECIMAL|DOUBLE|`0.0`|DECIMAL uses floating point, not fixed point math|
|FLOAT|FLOAT|`0.0`|Druid FLOAT columns are reported as FLOAT|
|REAL|DOUBLE|`0.0`||
|DOUBLE|DOUBLE|`0.0`|Druid DOUBLE columns are reported as DOUBLE|
|BOOLEAN|LONG|`false`||
|TINYINT|LONG|`0`||
|SMALLINT|LONG|`0`||

View File

@ -40,6 +40,7 @@ import io.druid.query.filter.OrDimFilter;
import io.druid.query.ordering.StringComparator;
import io.druid.query.ordering.StringComparators;
import io.druid.segment.column.Column;
import io.druid.segment.column.ValueType;
import io.druid.sql.calcite.filtration.BoundRefKey;
import io.druid.sql.calcite.filtration.Bounds;
import io.druid.sql.calcite.filtration.Filtration;
@ -121,33 +122,6 @@ public class Expressions
.put(SqlStdOperatorTable.OR, "||")
.build();
private static final Map<SqlTypeName, ExprType> EXPRESSION_TYPES;
static {
final ImmutableMap.Builder<SqlTypeName, ExprType> builder = ImmutableMap.builder();
for (SqlTypeName type : SqlTypeName.FRACTIONAL_TYPES) {
builder.put(type, ExprType.DOUBLE);
}
for (SqlTypeName type : SqlTypeName.INT_TYPES) {
builder.put(type, ExprType.LONG);
}
for (SqlTypeName type : SqlTypeName.STRING_TYPES) {
builder.put(type, ExprType.STRING);
}
// Booleans are treated as longs in Druid expressions, using two-value logic (positive = true, nonpositive = false).
builder.put(SqlTypeName.BOOLEAN, ExprType.LONG);
// Timestamps are treated as longs (millis since the epoch) in Druid expressions.
builder.put(SqlTypeName.TIMESTAMP, ExprType.LONG);
builder.put(SqlTypeName.DATE, ExprType.LONG);
EXPRESSION_TYPES = builder.build();
}
private Expressions()
{
// No instantiation.
@ -277,8 +251,8 @@ public class Expressions
);
} else {
// Handle other casts.
final ExprType fromExprType = EXPRESSION_TYPES.get(fromType);
final ExprType toExprType = EXPRESSION_TYPES.get(toType);
final ExprType fromExprType = exprTypeForValueType(Calcites.getValueTypeForSqlTypeName(fromType));
final ExprType toExprType = exprTypeForValueType(Calcites.getValueTypeForSqlTypeName(toType));
if (fromExprType == null || toExprType == null) {
// We have no runtime type for these SQL types.
@ -644,6 +618,21 @@ public class Expressions
}
}
public static ExprType exprTypeForValueType(final ValueType valueType)
{
switch (valueType) {
case LONG:
return ExprType.LONG;
case FLOAT:
case DOUBLE:
return ExprType.DOUBLE;
case STRING:
return ExprType.STRING;
default:
throw new ISE("No ExprType for valueType[%s]", valueType);
}
}
/**
* Translates to an "expression" type leaf filter. Used as a fallback if we can't use a simple leaf filter.
*/

View File

@ -22,6 +22,7 @@ package io.druid.sql.calcite.planner;
import com.google.common.io.BaseEncoding;
import com.google.common.primitives.Chars;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.query.ordering.StringComparator;
import io.druid.query.ordering.StringComparators;
@ -110,8 +111,10 @@ public class Calcites
public static ValueType getValueTypeForSqlTypeName(SqlTypeName sqlTypeName)
{
if (SqlTypeName.FRACTIONAL_TYPES.contains(sqlTypeName)) {
if (SqlTypeName.FLOAT == sqlTypeName) {
return ValueType.FLOAT;
} else if (SqlTypeName.FRACTIONAL_TYPES.contains(sqlTypeName)) {
return ValueType.DOUBLE;
} else if (SqlTypeName.TIMESTAMP == sqlTypeName
|| SqlTypeName.DATE == sqlTypeName
|| SqlTypeName.BOOLEAN == sqlTypeName
@ -129,10 +132,12 @@ public class Calcites
public static StringComparator getStringComparatorForSqlTypeName(SqlTypeName sqlTypeName)
{
final ValueType valueType = getValueTypeForSqlTypeName(sqlTypeName);
if (valueType == ValueType.LONG || valueType == ValueType.FLOAT) {
if (ValueType.isNumeric(valueType)) {
return StringComparators.NUMERIC;
} else {
} else if (ValueType.STRING == valueType) {
return StringComparators.LEXICOGRAPHIC;
} else {
throw new ISE("Unrecognized valueType[%s]", valueType);
}
}

View File

@ -0,0 +1,107 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.sql.calcite.planner;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelDataTypeSystem;
import org.apache.calcite.sql.type.SqlTypeName;
public class DruidTypeSystem implements RelDataTypeSystem
{
public static final DruidTypeSystem INSTANCE = new DruidTypeSystem();
private DruidTypeSystem()
{
// Singleton.
}
@Override
public int getMaxScale(final SqlTypeName typeName)
{
return RelDataTypeSystem.DEFAULT.getMaxScale(typeName);
}
@Override
public int getDefaultPrecision(final SqlTypeName typeName)
{
return RelDataTypeSystem.DEFAULT.getDefaultPrecision(typeName);
}
@Override
public int getMaxPrecision(final SqlTypeName typeName)
{
return RelDataTypeSystem.DEFAULT.getMaxPrecision(typeName);
}
@Override
public int getMaxNumericScale()
{
return RelDataTypeSystem.DEFAULT.getMaxNumericScale();
}
@Override
public int getMaxNumericPrecision()
{
return RelDataTypeSystem.DEFAULT.getMaxNumericPrecision();
}
@Override
public String getLiteral(final SqlTypeName typeName, final boolean isPrefix)
{
return RelDataTypeSystem.DEFAULT.getLiteral(typeName, isPrefix);
}
@Override
public boolean isCaseSensitive(final SqlTypeName typeName)
{
return RelDataTypeSystem.DEFAULT.isCaseSensitive(typeName);
}
@Override
public boolean isAutoincrement(final SqlTypeName typeName)
{
return RelDataTypeSystem.DEFAULT.isAutoincrement(typeName);
}
@Override
public int getNumTypeRadix(final SqlTypeName typeName)
{
return RelDataTypeSystem.DEFAULT.getNumTypeRadix(typeName);
}
@Override
public RelDataType deriveSumType(final RelDataTypeFactory typeFactory, final RelDataType argumentType)
{
// Widen all sums to 64-bits regardless of the size of the inputs.
if (SqlTypeName.INT_TYPES.contains(argumentType.getSqlTypeName())) {
return typeFactory.createSqlType(SqlTypeName.BIGINT);
} else {
return typeFactory.createSqlType(SqlTypeName.DOUBLE);
}
}
@Override
public boolean isSchemaCaseSensitive()
{
return RelDataTypeSystem.DEFAULT.isSchemaCaseSensitive();
}
}

View File

@ -86,6 +86,7 @@ public class PlannerFactory
.context(Contexts.EMPTY_CONTEXT)
.typeSystem(RelDataTypeSystem.DEFAULT)
.defaultSchema(rootSchema.getSubSchema(DruidSchema.NAME))
.typeSystem(DruidTypeSystem.INSTANCE)
.build();
return new DruidPlanner(Frameworks.getPlanner(frameworkConfig), plannerContext);

View File

@ -23,9 +23,7 @@ import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.collect.Iterables;
import com.google.common.primitives.Doubles;
import com.google.common.primitives.Ints;
import io.druid.common.guava.GuavaUtils;
import io.druid.data.input.Row;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.guava.Sequence;
@ -45,6 +43,7 @@ import io.druid.query.timeseries.TimeseriesResultValue;
import io.druid.query.topn.DimensionAndMetricValueExtractor;
import io.druid.query.topn.TopNQuery;
import io.druid.query.topn.TopNResultValue;
import io.druid.segment.DimensionHandlerUtils;
import io.druid.segment.column.Column;
import io.druid.server.QueryLifecycleFactory;
import io.druid.sql.calcite.planner.Calcites;
@ -346,7 +345,9 @@ public class QueryMaker
return ColumnMetaData.Rep.of(Integer.class);
} else if (sqlType == SqlTypeName.BIGINT) {
return ColumnMetaData.Rep.of(Long.class);
} else if (sqlType == SqlTypeName.FLOAT || sqlType == SqlTypeName.DOUBLE || sqlType == SqlTypeName.DECIMAL) {
} else if (sqlType == SqlTypeName.FLOAT) {
return ColumnMetaData.Rep.of(Float.class);
} else if (sqlType == SqlTypeName.DOUBLE || sqlType == SqlTypeName.DECIMAL) {
return ColumnMetaData.Rep.of(Double.class);
} else if (sqlType == SqlTypeName.OTHER) {
return ColumnMetaData.Rep.of(Object.class);
@ -416,19 +417,24 @@ public class QueryMaker
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
}
} else if (sqlType == SqlTypeName.BIGINT) {
if (value instanceof String) {
coercedValue = GuavaUtils.tryParseLong((String) value);
} else if (value instanceof Number) {
coercedValue = ((Number) value).longValue();
} else {
try {
coercedValue = DimensionHandlerUtils.convertObjectToLong(value);
}
catch (Exception e) {
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
}
} else if (sqlType == SqlTypeName.FLOAT || sqlType == SqlTypeName.DOUBLE || sqlType == SqlTypeName.DECIMAL) {
if (value instanceof String) {
coercedValue = Doubles.tryParse((String) value);
} else if (value instanceof Number) {
coercedValue = ((Number) value).doubleValue();
} else {
} else if (sqlType == SqlTypeName.FLOAT) {
try {
coercedValue = DimensionHandlerUtils.convertObjectToFloat(value);
}
catch (Exception e) {
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
}
} else if (SqlTypeName.FRACTIONAL_TYPES.contains(sqlType)) {
try {
coercedValue = DimensionHandlerUtils.convertObjectToDouble(value);
}
catch (Exception e) {
throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType);
}
} else if (sqlType == SqlTypeName.OTHER) {

View File

@ -26,11 +26,15 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import io.druid.java.util.common.ISE;
import io.druid.math.expr.ExprMacroTable;
import io.druid.math.expr.ExprType;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
import io.druid.query.aggregation.DoubleMinAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.query.aggregation.FloatMaxAggregatorFactory;
import io.druid.query.aggregation.FloatMinAggregatorFactory;
import io.druid.query.aggregation.FloatSumAggregatorFactory;
import io.druid.query.aggregation.LongMaxAggregatorFactory;
import io.druid.query.aggregation.LongMinAggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
@ -477,23 +481,19 @@ public class GroupByRules
// Walk through the postProject expressions.
int projectPostAggregatorCount = 0;
for (final RexNode projectExpression : postProject.getChildExps()) {
for (final RexNode postAggregatorRexNode : postProject.getChildExps()) {
// Attempt to convert to PostAggregator.
final DruidExpression postAggregatorExpression = Expressions.toDruidExpression(
druidRel.getPlannerContext(),
druidRel.getOutputRowSignature(),
projectExpression
postAggregatorRexNode
);
if (postAggregatorExpression == null) {
return null;
}
if (postAggregatorExpression.isDirectColumnAccess()
&& druidRel.getQueryBuilder()
.getOutputRowSignature()
.getColumnType(postAggregatorExpression.getDirectColumn())
.equals(Calcites.getValueTypeForSqlTypeName(projectExpression.getType().getSqlTypeName()))) {
if (postAggregatorDirectColumnIsOk(druidRel, postAggregatorExpression, postAggregatorRexNode)) {
// Direct column access, without any type cast as far as Druid's runtime is concerned.
// (There might be a SQL-level type cast that we don't care about)
newRowOrder.add(postAggregatorExpression.getDirectColumn());
@ -523,6 +523,40 @@ public class GroupByRules
);
}
/**
* Returns true if a post-aggregation "expression" can be realized as a direct field access. This is true if it's
* a direct column access that doesn't require an implicit cast.
*
* @param druidRel druid aggregation rel
* @param expression post-aggregation expression
* @param rexNode RexNode for the post-aggregation expression
*
* @return
*/
private static boolean postAggregatorDirectColumnIsOk(
final DruidRel druidRel,
final DruidExpression expression,
final RexNode rexNode
)
{
if (!expression.isDirectColumnAccess()) {
return false;
}
// Check if a cast is necessary.
final ExprType toExprType = Expressions.exprTypeForValueType(
druidRel.getQueryBuilder()
.getOutputRowSignature()
.getColumnType(expression.getDirectColumn())
);
final ExprType fromExprType = Expressions.exprTypeForValueType(
Calcites.getValueTypeForSqlTypeName(rexNode.getType().getSqlTypeName())
);
return toExprType.equals(fromExprType);
}
private static boolean canApplyHaving(final DruidRel druidRel)
{
return druidRel.getQueryBuilder().getGrouping() != null
@ -779,9 +813,24 @@ public class GroupByRules
// Built-in aggregator that is not COUNT.
final Aggregation retVal;
final boolean isLong = SqlTypeName.INT_TYPES.contains(outputType)
|| SqlTypeName.TIMESTAMP == outputType
|| SqlTypeName.DATE == outputType;
final ValueType aggregationType;
if (SqlTypeName.INT_TYPES.contains(outputType)
|| SqlTypeName.TIMESTAMP == outputType
|| SqlTypeName.DATE == outputType) {
aggregationType = ValueType.LONG;
} else if (SqlTypeName.FLOAT == outputType) {
aggregationType = ValueType.FLOAT;
} else if (SqlTypeName.FRACTIONAL_TYPES.contains(outputType)) {
aggregationType = ValueType.DOUBLE;
} else {
throw new ISE(
"Cannot determine aggregation type for SQL operator[%s] type[%s]",
call.getAggregation().getName(),
outputType
);
}
final String fieldName;
final String expression;
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
@ -795,23 +844,27 @@ public class GroupByRules
}
if (kind == SqlKind.SUM || kind == SqlKind.SUM0) {
retVal = isLong
? Aggregation.create(new LongSumAggregatorFactory(name, fieldName, expression, macroTable))
: Aggregation.create(new DoubleSumAggregatorFactory(name, fieldName, expression, macroTable));
retVal = Aggregation.create(
createSumAggregatorFactory(aggregationType, name, fieldName, expression, macroTable)
);
} else if (kind == SqlKind.MIN) {
retVal = isLong
? Aggregation.create(new LongMinAggregatorFactory(name, fieldName, expression, macroTable))
: Aggregation.create(new DoubleMinAggregatorFactory(name, fieldName, expression, macroTable));
retVal = Aggregation.create(
createMinAggregatorFactory(aggregationType, name, fieldName, expression, macroTable)
);
} else if (kind == SqlKind.MAX) {
retVal = isLong
? Aggregation.create(new LongMaxAggregatorFactory(name, fieldName, expression, macroTable))
: Aggregation.create(new DoubleMaxAggregatorFactory(name, fieldName, expression, macroTable));
retVal = Aggregation.create(
createMaxAggregatorFactory(aggregationType, name, fieldName, expression, macroTable)
);
} else if (kind == SqlKind.AVG) {
final String sumName = String.format("%s:sum", name);
final String countName = String.format("%s:count", name);
final AggregatorFactory sum = isLong
? new LongSumAggregatorFactory(sumName, fieldName, expression, macroTable)
: new DoubleSumAggregatorFactory(sumName, fieldName, expression, macroTable);
final AggregatorFactory sum = createSumAggregatorFactory(
aggregationType,
sumName,
fieldName,
expression,
macroTable
);
final AggregatorFactory count = new CountAggregatorFactory(countName);
retVal = Aggregation.create(
ImmutableList.of(sum, count),
@ -881,4 +934,64 @@ public class GroupByRules
.optimizeFilterOnly(sourceRowSignature)
.getDimFilter();
}
private static AggregatorFactory createSumAggregatorFactory(
final ValueType aggregationType,
final String name,
final String fieldName,
final String expression,
final ExprMacroTable macroTable
)
{
switch (aggregationType) {
case LONG:
return new LongSumAggregatorFactory(name, fieldName, expression, macroTable);
case FLOAT:
return new FloatSumAggregatorFactory(name, fieldName, expression, macroTable);
case DOUBLE:
return new DoubleSumAggregatorFactory(name, fieldName, expression, macroTable);
default:
throw new ISE("Cannot create aggregator factory for type[%s]", aggregationType);
}
}
private static AggregatorFactory createMinAggregatorFactory(
final ValueType aggregationType,
final String name,
final String fieldName,
final String expression,
final ExprMacroTable macroTable
)
{
switch (aggregationType) {
case LONG:
return new LongMinAggregatorFactory(name, fieldName, expression, macroTable);
case FLOAT:
return new FloatMinAggregatorFactory(name, fieldName, expression, macroTable);
case DOUBLE:
return new DoubleMinAggregatorFactory(name, fieldName, expression, macroTable);
default:
throw new ISE("Cannot create aggregator factory for type[%s]", aggregationType);
}
}
private static AggregatorFactory createMaxAggregatorFactory(
final ValueType aggregationType,
final String name,
final String fieldName,
final String expression,
final ExprMacroTable macroTable
)
{
switch (aggregationType) {
case LONG:
return new LongMaxAggregatorFactory(name, fieldName, expression, macroTable);
case FLOAT:
return new FloatMaxAggregatorFactory(name, fieldName, expression, macroTable);
case DOUBLE:
return new DoubleMaxAggregatorFactory(name, fieldName, expression, macroTable);
default:
throw new ISE("Cannot create aggregator factory for type[%s]", aggregationType);
}
}
}

View File

@ -355,6 +355,14 @@ public class DruidAvaticaHandlerTest
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "m1"),
Pair.of("DATA_TYPE", Types.FLOAT),
Pair.of("TYPE_NAME", "FLOAT"),
Pair.of("IS_NULLABLE", "NO")
),
ROW(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "m2"),
Pair.of("DATA_TYPE", Types.DOUBLE),
Pair.of("TYPE_NAME", "DOUBLE"),
Pair.of("IS_NULLABLE", "NO")

View File

@ -98,7 +98,8 @@ public class DruidStatementTest
Lists.newArrayList("cnt", "BIGINT", "java.lang.Long"),
Lists.newArrayList("dim1", "VARCHAR", "java.lang.String"),
Lists.newArrayList("dim2", "VARCHAR", "java.lang.String"),
Lists.newArrayList("m1", "DOUBLE", "java.lang.Double"),
Lists.newArrayList("m1", "FLOAT", "java.lang.Float"),
Lists.newArrayList("m2", "DOUBLE", "java.lang.Double"),
Lists.newArrayList("unique_dim1", "OTHER", "java.lang.Object")
),
Lists.transform(
@ -132,12 +133,12 @@ public class DruidStatementTest
0,
true,
Lists.<Object>newArrayList(
new Object[]{new DateTime("2000-01-01").getMillis(), 1L, "", "a", 1.0},
new Object[]{new DateTime("2000-01-02").getMillis(), 1L, "10.1", "", 2.0},
new Object[]{new DateTime("2000-01-03").getMillis(), 1L, "2", "", 3.0},
new Object[]{new DateTime("2001-01-01").getMillis(), 1L, "1", "a", 4.0},
new Object[]{new DateTime("2001-01-02").getMillis(), 1L, "def", "abc", 5.0},
new Object[]{new DateTime("2001-01-03").getMillis(), 1L, "abc", "", 6.0}
new Object[]{new DateTime("2000-01-01").getMillis(), 1L, "", "a", 1.0f},
new Object[]{new DateTime("2000-01-02").getMillis(), 1L, "10.1", "", 2.0f},
new Object[]{new DateTime("2000-01-03").getMillis(), 1L, "2", "", 3.0f},
new Object[]{new DateTime("2001-01-01").getMillis(), 1L, "1", "a", 4.0f},
new Object[]{new DateTime("2001-01-02").getMillis(), 1L, "def", "abc", 5.0f},
new Object[]{new DateTime("2001-01-03").getMillis(), 1L, "abc", "", 6.0f}
)
),
frame
@ -158,8 +159,8 @@ public class DruidStatementTest
0,
false,
Lists.<Object>newArrayList(
new Object[]{new DateTime("2000-01-01").getMillis(), 1L, "", "a", 1.0},
new Object[]{new DateTime("2000-01-02").getMillis(), 1L, "10.1", "", 2.0}
new Object[]{new DateTime("2000-01-01").getMillis(), 1L, "", "a", 1.0f},
new Object[]{new DateTime("2000-01-02").getMillis(), 1L, "10.1", "", 2.0f}
)
),
frame
@ -173,10 +174,10 @@ public class DruidStatementTest
2,
true,
Lists.<Object>newArrayList(
new Object[]{new DateTime("2000-01-03").getMillis(), 1L, "2", "", 3.0},
new Object[]{new DateTime("2001-01-01").getMillis(), 1L, "1", "a", 4.0},
new Object[]{new DateTime("2001-01-02").getMillis(), 1L, "def", "abc", 5.0},
new Object[]{new DateTime("2001-01-03").getMillis(), 1L, "abc", "", 6.0}
new Object[]{new DateTime("2000-01-03").getMillis(), 1L, "2", "", 3.0f},
new Object[]{new DateTime("2001-01-01").getMillis(), 1L, "1", "a", 4.0f},
new Object[]{new DateTime("2001-01-02").getMillis(), 1L, "def", "abc", 5.0f},
new Object[]{new DateTime("2001-01-03").getMillis(), 1L, "abc", "", 6.0f}
)
),
frame

View File

@ -36,9 +36,10 @@ import io.druid.query.QueryDataSource;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
import io.druid.query.aggregation.DoubleMinAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.query.aggregation.FilteredAggregatorFactory;
import io.druid.query.aggregation.FloatMaxAggregatorFactory;
import io.druid.query.aggregation.FloatMinAggregatorFactory;
import io.druid.query.aggregation.LongMaxAggregatorFactory;
import io.druid.query.aggregation.LongMinAggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
@ -304,7 +305,8 @@ public class CalciteQueryTest
new Object[]{"cnt", "BIGINT", "NO"},
new Object[]{"dim1", "VARCHAR", "YES"},
new Object[]{"dim2", "VARCHAR", "YES"},
new Object[]{"m1", "DOUBLE", "NO"},
new Object[]{"m1", "FLOAT", "NO"},
new Object[]{"m2", "DOUBLE", "NO"},
new Object[]{"unique_dim1", "OTHER", "NO"}
)
);
@ -354,7 +356,7 @@ public class CalciteQueryTest
.intervals(QSS(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.pagingSpec(FIRST_PAGING_SPEC)
.context(QUERY_CONTEXT_DEFAULT)
.build(),
@ -363,7 +365,7 @@ public class CalciteQueryTest
.intervals(QSS(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.pagingSpec(
new PagingSpec(
ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 5),
@ -375,12 +377,12 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{T("2000-01-01"), 1L, "", "a", 1.0, HLLCV1.class.getName()},
new Object[]{T("2000-01-02"), 1L, "10.1", "", 2.0, HLLCV1.class.getName()},
new Object[]{T("2000-01-03"), 1L, "2", "", 3.0, HLLCV1.class.getName()},
new Object[]{T("2001-01-01"), 1L, "1", "a", 4.0, HLLCV1.class.getName()},
new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0, HLLCV1.class.getName()},
new Object[]{T("2001-01-03"), 1L, "abc", "", 6.0, HLLCV1.class.getName()}
new Object[]{T("2000-01-01"), 1L, "", "a", 1f, 1.0, HLLCV1.class.getName()},
new Object[]{T("2000-01-02"), 1L, "10.1", "", 2f, 2.0, HLLCV1.class.getName()},
new Object[]{T("2000-01-03"), 1L, "2", "", 3f, 3.0, HLLCV1.class.getName()},
new Object[]{T("2001-01-01"), 1L, "1", "a", 4f, 4.0, HLLCV1.class.getName()},
new Object[]{T("2001-01-02"), 1L, "def", "abc", 5f, 5.0, HLLCV1.class.getName()},
new Object[]{T("2001-01-03"), 1L, "abc", "", 6f, 6.0, HLLCV1.class.getName()}
)
);
}
@ -430,14 +432,14 @@ public class CalciteQueryTest
.intervals(QSS(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.pagingSpec(FIRST_PAGING_SPEC)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{T("2000-01-01"), 1L, "", "a", 1.0, HLLCV1.class.getName()},
new Object[]{T("2000-01-02"), 1L, "10.1", "", 2.0, HLLCV1.class.getName()}
new Object[]{T("2000-01-01"), 1L, "", "a", 1.0f, 1.0, HLLCV1.class.getName()},
new Object[]{T("2000-01-02"), 1L, "10.1", "", 2.0f, 2.0, HLLCV1.class.getName()}
)
);
}
@ -453,15 +455,15 @@ public class CalciteQueryTest
.intervals(QSS(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.descending(true)
.pagingSpec(FIRST_PAGING_SPEC)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{T("2001-01-03"), 1L, "abc", "", 6.0, HLLCV1.class.getName()},
new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0, HLLCV1.class.getName()}
new Object[]{T("2001-01-03"), 1L, "abc", "", 6f, 6d, HLLCV1.class.getName()},
new Object[]{T("2001-01-02"), 1L, "def", "abc", 5f, 5d, HLLCV1.class.getName()}
)
);
}
@ -569,7 +571,7 @@ public class CalciteQueryTest
.intervals(QSS(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.pagingSpec(FIRST_PAGING_SPEC)
.context(QUERY_CONTEXT_DEFAULT)
.build(),
@ -578,7 +580,7 @@ public class CalciteQueryTest
.intervals(QSS(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.pagingSpec(
new PagingSpec(
ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 5),
@ -594,7 +596,7 @@ public class CalciteQueryTest
.granularity(Granularities.ALL)
.filters(NOT(SELECTOR("dim1", "", null)))
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.pagingSpec(FIRST_PAGING_SPEC)
.context(QUERY_CONTEXT_DEFAULT)
.build(),
@ -604,7 +606,7 @@ public class CalciteQueryTest
.granularity(Granularities.ALL)
.filters(NOT(SELECTOR("dim1", "", null)))
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.pagingSpec(
new PagingSpec(
ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 4),
@ -635,8 +637,8 @@ public class CalciteQueryTest
ImmutableList.of(),
ImmutableList.of(
new Object[]{
"BindableProject(dim1=[$8], dim10=[$2], dim2=[$3])\n"
+ " BindableJoin(condition=[=($8, $3)], joinType=[inner])\n"
"BindableProject(dim1=[$9], dim10=[$2], dim2=[$3])\n"
+ " BindableJoin(condition=[=($9, $3)], joinType=[inner])\n"
+ " DruidQueryRel(dataSource=[foo])\n"
+ " DruidQueryRel(dataSource=[foo], filter=[!dim1 = ])\n"
}
@ -680,6 +682,32 @@ public class CalciteQueryTest
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{1.0f, 1L},
new Object[]{2.0f, 1L},
new Object[]{3.0f, 1L},
new Object[]{4.0f, 1L},
new Object[]{5.0f, 1L},
new Object[]{6.0f, 1L}
)
);
}
@Test
public void testGroupByDouble() throws Exception
{
testQuery(
"SELECT m2, COUNT(*) FROM druid.foo GROUP BY m2",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(DIMS(new DefaultDimensionSpec("m2", "d0", ValueType.DOUBLE)))
.setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{1.0d, 1L},
new Object[]{2.0d, 1L},
@ -713,7 +741,28 @@ public class CalciteQueryTest
}
@Test
public void testHavingOnFloat() throws Exception
public void testFilterOnDouble() throws Exception
{
testQuery(
"SELECT COUNT(*) FROM druid.foo WHERE m2 = 1.0",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(AGGS(new CountAggregatorFactory("a0")))
.filters(SELECTOR("m2", "1.0", null))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{1L}
)
);
}
@Test
public void testHavingOnDoubleSum() throws Exception
{
testQuery(
"SELECT dim1, SUM(m1) AS m1_sum FROM druid.foo GROUP BY dim1 HAVING SUM(m1) > 1",
@ -751,6 +800,45 @@ public class CalciteQueryTest
);
}
@Test
public void testHavingOnFloatSum() throws Exception
{
testQuery(
"SELECT dim1, CAST(SUM(m1) AS FLOAT) AS m1_sum FROM druid.foo GROUP BY dim1 HAVING CAST(SUM(m1) AS FLOAT) > 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
.setAggregatorSpecs(AGGS(new DoubleSumAggregatorFactory("a0", "m1")))
.setHavingSpec(
new DimFilterHavingSpec(
new BoundDimFilter(
"a0",
"1",
null,
true,
false,
false,
null,
StringComparators.NUMERIC
)
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{"1", 4.0f},
new Object[]{"10.1", 2.0f},
new Object[]{"2", 3.0f},
new Object[]{"abc", 6.0f},
new Object[]{"def", 5.0f}
)
);
}
@Test
public void testColumnComparison() throws Exception
{
@ -771,8 +859,8 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{"", 1.0d, 1L},
new Object[]{"2", 3.0d, 1L}
new Object[]{"", 1.0f, 1L},
new Object[]{"2", 3.0f, 1L}
)
);
}
@ -1187,7 +1275,7 @@ public class CalciteQueryTest
)
)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.context(QUERY_CONTEXT_DEFAULT)
.build(),
Druids.newSelectQueryBuilder()
@ -1208,14 +1296,14 @@ public class CalciteQueryTest
)
)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{T("2000-01-01"), 1L, "", "a", 1.0, HLLCV1.class.getName()},
new Object[]{T("2001-01-01"), 1L, "1", "a", 4.0, HLLCV1.class.getName()},
new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0, HLLCV1.class.getName()}
new Object[]{T("2000-01-01"), 1L, "", "a", 1.0f, 1d, HLLCV1.class.getName()},
new Object[]{T("2001-01-01"), 1L, "1", "a", 4.0f, 4d, HLLCV1.class.getName()},
new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0f, 5d, HLLCV1.class.getName()}
)
);
}
@ -1239,7 +1327,7 @@ public class CalciteQueryTest
)
)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.context(QUERY_CONTEXT_DEFAULT)
.build(),
Druids.newSelectQueryBuilder()
@ -1260,7 +1348,7 @@ public class CalciteQueryTest
)
)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.context(QUERY_CONTEXT_DEFAULT)
.build(),
Druids.newSelectQueryBuilder()
@ -1281,14 +1369,14 @@ public class CalciteQueryTest
)
)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "unique_dim1"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1"))
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{T("2000-01-01"), 1L, "", "a", 1.0, HLLCV1.class.getName()},
new Object[]{T("2001-01-01"), 1L, "1", "a", 4.0, HLLCV1.class.getName()},
new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0, HLLCV1.class.getName()}
new Object[]{T("2000-01-01"), 1L, "", "a", 1.0f, 1.0d, HLLCV1.class.getName()},
new Object[]{T("2001-01-01"), 1L, "1", "a", 4.0f, 4.0d, HLLCV1.class.getName()},
new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0f, 5.0d, HLLCV1.class.getName()}
)
);
}
@ -1762,8 +1850,8 @@ public class CalciteQueryTest
.dimension(new DefaultDimensionSpec("dim1", "d0"))
.metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec("p0")))
.aggregators(AGGS(
new DoubleMinAggregatorFactory("a0", "m1"),
new DoubleMaxAggregatorFactory("a1", "m1")
new FloatMinAggregatorFactory("a0", "m1"),
new FloatMaxAggregatorFactory("a1", "m1")
))
.postAggregators(
ImmutableList.of(
@ -1775,9 +1863,9 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{"", 2.0},
new Object[]{"10.1", 4.0},
new Object[]{"2", 6.0}
new Object[]{"", 2.0f},
new Object[]{"10.1", 4.0f},
new Object[]{"2", 6.0f}
)
);
}
@ -1798,8 +1886,8 @@ public class CalciteQueryTest
.setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
.setAggregatorSpecs(
ImmutableList.of(
new DoubleMinAggregatorFactory("a0", "m1"),
new DoubleMaxAggregatorFactory("a1", "m1")
new FloatMinAggregatorFactory("a0", "m1"),
new FloatMaxAggregatorFactory("a1", "m1")
)
)
.setPostAggregatorSpecs(ImmutableList.of(EXPRESSION_POST_AGG("p0", "(\"a0\" + \"a1\")")))
@ -1819,9 +1907,9 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{"", 2.0},
new Object[]{"10.1", 4.0},
new Object[]{"2", 6.0}
new Object[]{"", 2.0f},
new Object[]{"10.1", 4.0f},
new Object[]{"2", 6.0f}
)
);
}
@ -1843,8 +1931,8 @@ public class CalciteQueryTest
.setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
.setAggregatorSpecs(
ImmutableList.of(
new DoubleMinAggregatorFactory("a0", "m1"),
new DoubleMaxAggregatorFactory("a1", "m1")
new FloatMinAggregatorFactory("a0", "m1"),
new FloatMaxAggregatorFactory("a1", "m1")
)
)
.setPostAggregatorSpecs(
@ -1868,9 +1956,9 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{"", 2.0},
new Object[]{"10.1", 4.0},
new Object[]{"2", 6.0}
new Object[]{"", 2.0f},
new Object[]{"10.1", 4.0f},
new Object[]{"2", 6.0f}
)
);
}
@ -1948,7 +2036,7 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{1L, 5L, 1L, 2L, 5L, 5, 2L, 1L, 5L, 1L}
new Object[]{1L, 5L, 1L, 2L, 5L, 5L, 2L, 1L, 5L, 1L}
)
);
}
@ -1986,7 +2074,7 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{18L, 3.295836866004329, 2, 12, 3f + (Math.log(5.0))}
new Object[]{18L, 3.295836866004329, 2, 12L, 3f + (Math.log(5.0))}
)
);
}
@ -2029,10 +2117,10 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{6.0, 1L},
new Object[]{4.0, 2L},
new Object[]{2.0, 2L},
new Object[]{0.0, 1L}
new Object[]{6.0f, 1L},
new Object[]{4.0f, 2L},
new Object[]{2.0f, 2L},
new Object[]{0.0f, 1L}
)
);
}
@ -2129,9 +2217,9 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{10.0, 1L},
new Object[]{2.0, 1L},
new Object[]{0.0, 4L}
new Object[]{10.0f, 1L},
new Object[]{2.0f, 1L},
new Object[]{0.0f, 4L}
)
);
}
@ -2616,7 +2704,7 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{3}
new Object[]{3L}
)
);
}
@ -2645,7 +2733,7 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{3}
new Object[]{3L}
)
);
}
@ -3471,7 +3559,7 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{5L, 5L, -0.1222693591629298}
new Object[]{5L, 5L, -0.12226936f}
)
);
}
@ -3606,7 +3694,7 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{6L, 3L, 3.0021994137521975, 1L, 4L, 4.9985347983600805}
new Object[]{6L, 3L, 3.0021994f, 1L, 4L, 4.9985347f}
)
);
}
@ -4094,10 +4182,10 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{0.0, 3L},
new Object[]{1.0, 1L},
new Object[]{2.0, 1L},
new Object[]{10.0, 1L}
new Object[]{0.0f, 3L},
new Object[]{1.0f, 1L},
new Object[]{2.0f, 1L},
new Object[]{10.0f, 1L}
)
);
}
@ -4145,10 +4233,10 @@ public class CalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{10.0, 1L},
new Object[]{2.0, 1L},
new Object[]{1.0, 1L},
new Object[]{0.0, 3L}
new Object[]{10.0f, 1L},
new Object[]{2.0f, 1L},
new Object[]{1.0f, 1L},
new Object[]{0.0f, 3L}
)
);
}

View File

@ -54,6 +54,7 @@ import io.druid.query.QueryToolChest;
import io.druid.query.QueryToolChestWarehouse;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.query.aggregation.FloatSumAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import io.druid.query.expression.LookupExprMacro;
import io.druid.query.expression.TestExprMacroTable;
@ -250,19 +251,32 @@ public class CalciteTests
private static final IncrementalIndexSchema INDEX_SCHEMA = new IncrementalIndexSchema.Builder()
.withMetrics(
new CountAggregatorFactory("cnt"),
new DoubleSumAggregatorFactory("m1", "m1"),
new FloatSumAggregatorFactory("m1", "m1"),
new DoubleSumAggregatorFactory("m2", "m2"),
new HyperUniquesAggregatorFactory("unique_dim1", "dim1")
)
.withRollup(false)
.build();
public static final List<InputRow> ROWS1 = ImmutableList.of(
createRow(ImmutableMap.of("t", "2000-01-01", "m1", "1.0", "dim1", "", "dim2", ImmutableList.of("a"))),
createRow(ImmutableMap.of("t", "2000-01-02", "m1", "2.0", "dim1", "10.1", "dim2", ImmutableList.of())),
createRow(ImmutableMap.of("t", "2000-01-03", "m1", "3.0", "dim1", "2", "dim2", ImmutableList.of(""))),
createRow(ImmutableMap.of("t", "2001-01-01", "m1", "4.0", "dim1", "1", "dim2", ImmutableList.of("a"))),
createRow(ImmutableMap.of("t", "2001-01-02", "m1", "5.0", "dim1", "def", "dim2", ImmutableList.of("abc"))),
createRow(ImmutableMap.of("t", "2001-01-03", "m1", "6.0", "dim1", "abc"))
createRow(
ImmutableMap.of("t", "2000-01-01", "m1", "1.0", "m2", "1.0", "dim1", "", "dim2", ImmutableList.of("a"))
),
createRow(
ImmutableMap.of("t", "2000-01-02", "m1", "2.0", "m2", "2.0", "dim1", "10.1", "dim2", ImmutableList.of())
),
createRow(
ImmutableMap.of("t", "2000-01-03", "m1", "3.0", "m2", "3.0", "dim1", "2", "dim2", ImmutableList.of(""))
),
createRow(
ImmutableMap.of("t", "2001-01-01", "m1", "4.0", "m2", "4.0", "dim1", "1", "dim2", ImmutableList.of("a"))
),
createRow(
ImmutableMap.of("t", "2001-01-02", "m1", "5.0", "m2", "5.0", "dim1", "def", "dim2", ImmutableList.of("abc"))
),
createRow(
ImmutableMap.of("t", "2001-01-03", "m1", "6.0", "m2", "6.0", "dim1", "abc")
)
);
public static final List<InputRow> ROWS2 = ImmutableList.of(