Upgrade Calcite to 1.21 ()

* Upgrade Calcite to 1.21

* Checkstyle, test fix'

* Exclude calcite yaml deps, update license.yaml

* Add method for exception chain handling

* Checkstyle

* PR comments, Add outer limit context flag

* Revert project settings change

* Update subquery test comment

* Checkstyle fix

* Fix test in sql compat mode

* Fix test

* Fix dependency analysis

* Address PR comments

* Checkstyle

* Adjust testSelectStarFromSelectSingleColumnWithLimitDescending
This commit is contained in:
Jonathan Wei 2019-11-20 21:22:55 -08:00 committed by Gian Merlino
parent ff6217365b
commit dc6178d1f2
13 changed files with 425 additions and 158 deletions
core/src/main/java/org/apache/druid/math/expr
extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql
licenses.yamlpom.xml
sql

View File

@ -29,6 +29,7 @@ import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.format.DateTimeFormat;
import javax.annotation.Nullable;
import java.math.BigDecimal;
import java.math.RoundingMode;
import java.util.ArrayList;
@ -224,7 +225,7 @@ interface Function
return eval(x.asString(), y.asInt());
}
protected abstract ExprEval eval(String x, int y);
protected abstract ExprEval eval(@Nullable String x, int y);
}
/**
@ -1455,7 +1456,7 @@ interface Function
}
@Override
protected ExprEval eval(String x, int y)
protected ExprEval eval(@Nullable String x, int y)
{
if (y < 0) {
throw new IAE(
@ -1463,6 +1464,9 @@ interface Function
name()
);
}
if (x == null) {
return ExprEval.of(null);
}
int len = x.length();
return ExprEval.of(y < len ? x.substring(len - y) : x);
}
@ -1477,7 +1481,7 @@ interface Function
}
@Override
protected ExprEval eval(String x, int y)
protected ExprEval eval(@Nullable String x, int y)
{
if (y < 0) {
throw new IAE(
@ -1485,6 +1489,9 @@ interface Function
name()
);
}
if (x == null) {
return ExprEval.of(null);
}
return ExprEval.of(y < x.length() ? x.substring(0, y) : x);
}
}

View File

@ -144,7 +144,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
// fool the planner to make an expression virtual column to test bloom filter Druid expression
testQuery(
StringUtils.format("SELECT COUNT(*) FROM druid.foo WHERE bloom_filter_test(concat(dim2, '-foo'), '%s') = TRUE", base64),
StringUtils.format("SELECT COUNT(*) FROM druid.foo WHERE nullif(bloom_filter_test(concat(dim2, '-foo'), '%s'), 1) is null", base64),
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
@ -152,7 +152,12 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
.granularity(Granularities.ALL)
.filters(
new ExpressionDimFilter(
StringUtils.format("(bloom_filter_test(concat(\"dim2\",'-foo'),'%s') == 1)", base64),
StringUtils.format(
"case_searched(bloom_filter_test(concat(\"dim2\",'-foo'),'%s'),1,isnull(bloom_filter_test(concat(\"dim2\",'-foo'),'%s')))",
base64,
base64
),
null,
createExprMacroTable()
)
)

View File

@ -169,7 +169,7 @@ name: Esri Geometry API for Java
license_category: binary
module: java-core
license_name: Apache License version 2.0
version: 2.0.0
version: 2.2.0
libraries:
- com.esri.geometry: esri-geometry-api
@ -1147,17 +1147,17 @@ name: Apache Calcite
license_category: binary
module: java-core
license_name: Apache License version 2.0
version: 1.17.0
version: 1.21.0
libraries:
- org.apache.calcite: calcite-core
- org.apache.calcite: calcite-linq4j
notices:
- calcite-core: |
Calcite Core
Copyright 2012-2018 The Apache Software Foundation
Copyright 2012-2019 The Apache Software Foundation
- calcite-linq4j: |
Calcite Linq4j
Copyright 2012-2018 The Apache Software Foundation
Copyright 2012-2019 The Apache Software Foundation
---
@ -3349,7 +3349,7 @@ name: Janino and Commons Compiler
license_category: binary
module: java-core
license_name: BSD-3-Clause License
version: 2.7.6
version: 3.0.11
copyright: Arno Unkrig and TIBCO Software Inc.
license_file_path: licenses/bin/janino.BSD3
libraries:

View File

@ -80,7 +80,7 @@
<apache.curator.test.version>2.12.0</apache.curator.test.version>
<avatica.version>1.12.0</avatica.version>
<avro.version>1.9.1</avro.version>
<calcite.version>1.17.0</calcite.version>
<calcite.version>1.21.0</calcite.version>
<derby.version>10.14.2.0</derby.version>
<dropwizard.metrics.version>4.0.0</dropwizard.metrics.version>
<guava.version>16.0.1</guava.version>

View File

@ -67,6 +67,17 @@
<groupId>com.yahoo.datasketches</groupId>
<artifactId>sketches-core</artifactId>
</exclusion>
<!--
~ We don't use Calcite's YAML features.
-->
<exclusion>
<groupId>com.fasterxml.jackson.dataformat</groupId>
<artifactId>jackson-dataformat-yaml</artifactId>
</exclusion>
<exclusion>
<groupId>org.yaml</groupId>
<artifactId>snakeyaml</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
@ -155,6 +166,12 @@
<groupId>org.apache.curator</groupId>
<artifactId>curator-x-discovery</artifactId>
</dependency>
<dependency>
<groupId>org.checkerframework</groupId>
<artifactId>checker-qual</artifactId>
<version>${checkerframework.version}</version>
<scope>provided</scope>
</dependency>
<!-- Tests -->
<dependency>

View File

@ -68,6 +68,8 @@ public class DruidConvertletTable implements SqlRexConvertletTable
.add(SqlStdOperatorTable.TIMESTAMP_DIFF)
.add(SqlStdOperatorTable.UNION)
.add(SqlStdOperatorTable.UNION_ALL)
.add(SqlStdOperatorTable.NULLIF)
.add(SqlStdOperatorTable.COALESCE)
.add(OracleSqlOperatorTable.NVL)
.build();

View File

@ -29,6 +29,7 @@ import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.SqlOperatorTable;
import org.apache.calcite.sql.SqlSyntax;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.validate.SqlNameMatcher;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
@ -358,7 +359,8 @@ public class DruidOperatorTable implements SqlOperatorTable
final SqlIdentifier opName,
final SqlFunctionCategory category,
final SqlSyntax syntax,
final List<SqlOperator> operatorList
final List<SqlOperator> operatorList,
final SqlNameMatcher nameMatcher
)
{
if (opName == null) {

View File

@ -35,6 +35,7 @@ import org.apache.calcite.plan.RelOptPlanner;
import org.apache.calcite.plan.RelOptUtil;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelRoot;
import org.apache.calcite.rel.logical.LogicalSort;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexNode;
@ -42,6 +43,7 @@ import org.apache.calcite.sql.SqlExplain;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.parser.SqlParseException;
import org.apache.calcite.sql.type.BasicSqlType;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.tools.Planner;
import org.apache.calcite.tools.RelConversionException;
@ -50,9 +52,11 @@ import org.apache.calcite.util.Pair;
import org.apache.druid.java.util.common.guava.BaseSequence;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.sql.calcite.rel.DruidConvention;
import org.apache.druid.sql.calcite.rel.DruidRel;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.util.ArrayList;
import java.util.Iterator;
@ -63,6 +67,7 @@ public class DruidPlanner implements Closeable
{
private final Planner planner;
private final PlannerContext plannerContext;
private RexBuilder rexBuilder;
public DruidPlanner(
final Planner planner,
@ -82,6 +87,9 @@ public class DruidPlanner implements Closeable
explain = (SqlExplain) parsed;
parsed = explain.getExplicandum();
}
// the planner's type factory is not available until after parsing
this.rexBuilder = new RexBuilder(planner.getTypeFactory());
final SqlNode validated = planner.validate(parsed);
final RelRoot root = planner.rel(validated);
@ -116,12 +124,14 @@ public class DruidPlanner implements Closeable
final RelRoot root
) throws RelConversionException
{
final RelNode possiblyWrappedRootRel = possiblyWrapRootWithOuterLimitFromContext(root);
final DruidRel<?> druidRel = (DruidRel<?>) planner.transform(
Rules.DRUID_CONVENTION_RULES,
planner.getEmptyTraitSet()
.replace(DruidConvention.instance())
.plus(root.collation),
root.rel
possiblyWrappedRootRel
);
final Set<String> dataSourceNames = ImmutableSet.copyOf(druidRel.getDataSourceNames());
@ -232,6 +242,42 @@ public class DruidPlanner implements Closeable
}
}
/**
* This method wraps the root with a logical sort that applies a limit (no ordering change).
* The CTX_SQL_OUTER_LIMIT flag that controls this wrapping is meant for internal use only by the
* web console, allowing it to apply a limit to queries without rewriting the original SQL.
*
* @param root root node
* @return root node wrapped with a limiting logical sort if a limit is specified in the query context.
*/
@Nullable
private RelNode possiblyWrapRootWithOuterLimitFromContext(
RelRoot root
)
{
Object outerLimitObj = plannerContext.getQueryContext().get(PlannerContext.CTX_SQL_OUTER_LIMIT);
Long outerLimit = DimensionHandlerUtils.convertObjectToLong(outerLimitObj, true);
if (outerLimit == null) {
return root.rel;
}
return LogicalSort.create(
root.rel,
root.collation,
makeBigIntLiteral(0),
makeBigIntLiteral(outerLimit)
);
}
private RexNode makeBigIntLiteral(long value)
{
return rexBuilder.makeLiteral(
value,
new BasicSqlType(DruidTypeSystem.INSTANCE, SqlTypeName.BIGINT),
false
);
}
private static class EnumeratorIterator<T> implements Iterator<T>
{
private final Iterator<T> it;

View File

@ -50,6 +50,10 @@ public class PlannerContext
public static final String CTX_SQL_CURRENT_TIMESTAMP = "sqlCurrentTimestamp";
public static final String CTX_SQL_TIME_ZONE = "sqlTimeZone";
// This context parameter is an undocumented parameter, used internally, to allow the web console to
// apply a limit without having to rewrite the SQL query.
public static final String CTX_SQL_OUTER_LIMIT = "sqlOuterLimit";
// DataContext keys
public static final String DATA_CTX_AUTHENTICATION_RESULT = "authenticationResult";

View File

@ -138,6 +138,12 @@ public class PlannerFactory
final Properties props = new Properties();
return (C) new CalciteConnectionConfigImpl(props)
{
@Override
public <T> T typeSystem(Class<T> typeSystemClass, T defaultTypeSystem)
{
return (T) DruidTypeSystem.INSTANCE;
}
@Override
public SqlConformance conformance()
{

View File

@ -49,7 +49,8 @@ public class SortCollapseRule extends RelOptRule
final Sort first = call.rel(1);
final Sort second = call.rel(0);
if (second.collation.getFieldCollations().isEmpty()) {
if (second.collation.getFieldCollations().isEmpty()
|| second.collation.getFieldCollations().equals(first.collation.getFieldCollations())) {
// Add up the offsets.
final int firstOffset = (first.offset != null ? RexLiteral.intValue(first.offset) : 0);
final int secondOffset = (second.offset != null ? RexLiteral.intValue(second.offset) : 0);
@ -81,7 +82,7 @@ public class SortCollapseRule extends RelOptRule
first.getInput(),
first.getCollation(),
offset == 0 ? null : call.builder().literal(offset),
call.builder().literal(fetch)
fetch < 0 ? null : call.builder().literal(fetch)
);
call.transformTo(combined);

View File

@ -25,6 +25,7 @@ import org.apache.calcite.runtime.CalciteContextException;
import org.apache.calcite.tools.ValidationException;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
@ -78,6 +79,7 @@ import org.apache.druid.segment.column.ValueType;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.rel.CannotBuildQueryException;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.hamcrest.CoreMatchers;
@ -85,6 +87,7 @@ import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.Interval;
import org.joda.time.Period;
import org.junit.Assert;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.internal.matchers.ThrowableMessageMatcher;
@ -92,7 +95,9 @@ import org.junit.internal.matchers.ThrowableMessageMatcher;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class CalciteQueryTest extends BaseCalciteQueryTest
{
@ -144,7 +149,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.filters(selector("dim2", "0", null))
.filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
.granularity(Granularities.ALL)
.aggregators(aggregators(
new CountAggregatorFactory("a0"),
@ -760,8 +765,34 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testSelectStarFromSelectSingleColumnWithLimitDescending() throws Exception
{
// After upgrading to Calcite 1.21, Calcite no longer respects the ORDER BY __time DESC
// in the inner query. This is valid, as the SQL standard considers the subquery results to be an unordered
// set of rows.
testQuery(
"SELECT * FROM (SELECT dim1 FROM druid.foo ORDER BY __time DESC) LIMIT 2",
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns(ImmutableList.of("dim1"))
.limit(2)
.order(ScanQuery.Order.NONE)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{""},
new Object[]{"10.1"}
)
);
// The outer limit wrapping behavior that was used in the query above can be applied with a context flag instead
Map<String, Object> outerLimitContext = new HashMap<>(QUERY_CONTEXT_DEFAULT);
outerLimitContext.put(PlannerContext.CTX_SQL_OUTER_LIMIT, 2);
testQuery(
"SELECT dim1 FROM druid.foo ORDER BY __time DESC",
outerLimitContext,
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
@ -770,7 +801,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.limit(2)
.order(ScanQuery.Order.DESCENDING)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.context(outerLimitContext)
.build()
),
ImmutableList.of(
@ -809,6 +840,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
{
// Regression test for https://github.com/apache/incubator-druid/issues/7768.
// After upgrading to Calcite 1.21, Calcite no longer respects the ORDER BY __time DESC
// in the inner query. This is valid, as the SQL standard considers the subquery results to be an unordered
// set of rows. This test now validates that the inner ordering is not applied.
testQuery(
"SELECT 'beep ' || dim1 FROM (SELECT dim1 FROM druid.foo ORDER BY __time DESC)",
ImmutableList.of(
@ -816,19 +850,19 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(expressionVirtualColumn("v0", "concat('beep ',\"dim1\")", ValueType.STRING))
.columns(ImmutableList.of("__time", "v0"))
.order(ScanQuery.Order.DESCENDING)
.columns(ImmutableList.of("v0"))
.order(ScanQuery.Order.NONE)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{"beep abc"},
new Object[]{"beep def"},
new Object[]{"beep 1"},
new Object[]{"beep 2"},
new Object[]{"beep "},
new Object[]{"beep 10.1"},
new Object[]{"beep "}
new Object[]{"beep 2"},
new Object[]{"beep 1"},
new Object[]{"beep def"},
new Object[]{"beep abc"}
)
);
}
@ -1465,7 +1499,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimFilter(expressionFilter("((\"m1\" - 1) == \"dim1\")"))
.setDimFilter(expressionFilter("((\"m1\" - 1) == CAST(\"dim1\", 'DOUBLE'))"))
.setDimensions(dimensions(
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("m1", "d1", ValueType.FLOAT)
@ -1919,7 +1953,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(expressionFilter("case_searched((\"dim2\" == 'a'),1,isnull(\"dim2\"))"))
// Ideally the following filter should be simplified to (dim2 == 'a' || dim2 IS NULL), the
// (dim2 != 'a') component is unnecessary.
.filters(
or(
selector("dim2", "a", null),
and(
selector("dim2", null, null),
not(selector("dim2", "a", null))
)
)
)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
@ -1937,58 +1981,61 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testEmptyStringEquality() throws Exception
{
testQuery(
"SELECT COUNT(*)\n"
+ "FROM druid.foo\n"
+ "WHERE NULLIF(dim2, 'a') = ''",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(expressionFilter("case_searched((\"dim2\" == 'a'),"
+ (NullHandling.replaceWithDefault() ? "1" : "0")
+ ",(\"dim2\" == ''))"))
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
NullHandling.replaceWithDefault() ?
// Matches everything but "abc"
new Object[]{5L} :
// match only empty string
new Object[]{1L}
)
);
if (NullHandling.replaceWithDefault()) {
testQuery(
"SELECT COUNT(*)\n"
+ "FROM druid.foo\n"
+ "WHERE NULLIF(dim2, 'a') = ''",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(in("dim2", ImmutableList.of("", "a"), null))
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
// Matches everything but "abc"
new Object[]{5L}
)
);
} else {
testQuery(
"SELECT COUNT(*)\n"
+ "FROM druid.foo\n"
+ "WHERE NULLIF(dim2, 'a') = ''",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(selector("dim2", "", null))
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
// match only empty string
new Object[]{1L}
)
);
}
}
@Test
public void testNullStringEquality() throws Exception
{
// In Calcite 1.21, this query is optimized to return 0 without generating a native Druid query, since
// null is not equal to null or any other value.
testQuery(
"SELECT COUNT(*)\n"
+ "FROM druid.foo\n"
+ "WHERE NULLIF(dim2, 'a') = null",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(expressionFilter("case_searched((\"dim2\" == 'a'),"
+ (NullHandling.replaceWithDefault() ? "1" : "0")
+ ",(\"dim2\" == null))"))
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
NullHandling.replaceWithDefault() ?
// Matches everything but "abc"
ImmutableList.of(new Object[]{5L}) :
// null is not eqaual to null or any other value
ImmutableList.of()
ImmutableList.of(),
ImmutableList.of(new Object[]{0L})
);
}
@Test
@ -2060,7 +2107,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
}
@Test
public void testUnplannableQueries()
public void testUnplannableQueries() throws Exception
{
// All of these queries are unplannable because they rely on features Druid doesn't support.
// This test is here to confirm that we don't fall back to Calcite's interpreter or enumerable implementation.
@ -2276,9 +2323,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testCountNullableExpression() throws Exception
{
// Cannot vectorize due to expression filter.
cannotVectorize();
testQuery(
"SELECT COUNT(CASE WHEN dim2 = 'abc' THEN 'yes' WHEN dim2 = 'def' THEN 'yes' END) FROM druid.foo",
ImmutableList.of(
@ -2286,19 +2330,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.virtualColumns(
expressionVirtualColumn(
"v0",
"case_searched((\"dim2\" == 'abc'),'yes',(\"dim2\" == 'def'),'yes',"
+ DruidExpression.nullLiteral()
+ ")",
ValueType.STRING
)
)
.aggregators(aggregators(
new FilteredAggregatorFactory(
new CountAggregatorFactory("a0"),
not(selector("v0", NullHandling.defaultStringValue(), null))
in("dim2", ImmutableList.of("abc", "def"), null)
)
))
.context(TIMESERIES_CONTEXT_DEFAULT)
@ -2583,7 +2618,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
)
.setDimFilter(
or(
selector("dim1", "10", null),
bound("dim1", "10", "10", false, false, null, StringComparators.NUMERIC),
and(
selector("v0", "10.00", null),
bound("dim1", "9", "10.5", true, false, null, StringComparators.NUMERIC)
@ -3234,20 +3269,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
and(
selector("dim2", "a", null),
or(
bound("dim1", "a", null, true, false, null, StringComparators.LEXICOGRAPHIC),
not(selector("dim1", null, null))
)
)
selector("dim2", "a", null)
)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{NullHandling.sqlCompatible() ? 2L : 1L}
new Object[]{2L}
)
);
}
@ -3457,20 +3486,23 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
public void testCountStarWithTimeFilterUsingStringLiteralsInvalid() throws Exception
{
// Strings are implicitly cast to timestamps. Test an invalid string.
// This error message isn't ideal but it is at least better than silently ignoring the problem.
expectedException.expect(RuntimeException.class);
expectedException.expectMessage("Error while applying rule ReduceExpressionsRule");
expectedException.expectCause(
ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Illegal TIMESTAMP constant"))
);
testQuery(
"SELECT COUNT(*) FROM druid.foo\n"
+ "WHERE __time >= 'z2000-01-01 00:00:00' AND __time < '2001-01-01 00:00:00'\n",
ImmutableList.of(),
ImmutableList.of()
);
try {
testQuery(
"SELECT COUNT(*) FROM druid.foo\n"
+ "WHERE __time >= 'z2000-01-01 00:00:00' AND __time < '2001-01-01 00:00:00'\n",
ImmutableList.of(),
ImmutableList.of()
);
}
catch (Throwable t) {
Throwable rootException = CalciteTests.getRootCauseFromInvocationTargetExceptionChain(t);
Assert.assertEquals(IAE.class, rootException.getClass());
Assert.assertEquals(
"Illegal TIMESTAMP constant: CAST('z2000-01-01 00:00:00'):TIMESTAMP(3) NOT NULL",
rootException.getMessage()
);
}
}
@Test
@ -3946,7 +3978,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn("v0", "strlen(\"dim1\")", ValueType.LONG),
expressionVirtualColumn("v1", "CAST(strlen(\"dim1\"), 'STRING')", ValueType.STRING)
// The two layers of CASTs here are unusual, they should really be collapsed into one
expressionVirtualColumn("v1", "CAST(CAST(strlen(\"dim1\"), 'STRING'), 'LONG')", ValueType.LONG)
)
.setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0")))
.setDimFilter(
@ -4286,8 +4319,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("dim1", "d1")
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("dim2", "d1")
))
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
@ -4296,12 +4329,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("d0", "_d0")))
.setDimensions(dimensions(new DefaultDimensionSpec("d1", "_d0")))
.setAggregatorSpecs(aggregators(
new LongSumAggregatorFactory("_a0", "a0"),
new FilteredAggregatorFactory(
new CountAggregatorFactory("_a1"),
not(selector("d1", null, null))
not(selector("d0", null, null))
)
))
.setContext(QUERY_CONTEXT_DEFAULT)
@ -4432,8 +4465,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(
new DefaultDimensionSpec("m2", "d0", ValueType.DOUBLE),
new DefaultDimensionSpec("dim1", "d1")
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("m2", "d1", ValueType.DOUBLE)
))
.setDimFilter(new SelectorDimFilter("m1", "5.0", null))
.setAggregatorSpecs(aggregators(new LongMaxAggregatorFactory("a0", "__time")))
@ -4451,7 +4484,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
)
.setDimensions(dimensions(
new DefaultDimensionSpec("v0", "v0", ValueType.LONG),
new DefaultDimensionSpec("d1", "_d0", ValueType.STRING)
new DefaultDimensionSpec("d0", "_d0", ValueType.STRING)
))
.setAggregatorSpecs(aggregators(
new CountAggregatorFactory("_a0")
@ -7271,18 +7304,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
new OrderByColumnSpec(
"d0",
OrderByColumnSpec.Direction.DESCENDING,
StringComparators.LEXICOGRAPHIC
)
),
Integer.MAX_VALUE
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build(),
newScanQueryBuilder()
@ -7521,15 +7542,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new DefaultDimensionSpec("dim2", "d1")
)
)
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(
new OrderByColumnSpec("a0", Direction.ASCENDING, StringComparators.NUMERIC)
),
Integer.MAX_VALUE
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@ -7564,17 +7576,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
aggregators(new CountAggregatorFactory("a0"), new DoubleSumAggregatorFactory("a1", "m2"))
)
.setPostAggregatorSpecs(Collections.singletonList(expressionPostAgg(
"s0",
"p0",
"(\"a1\" / \"a0\")"
)))
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(
new OrderByColumnSpec("a0", Direction.ASCENDING, StringComparators.NUMERIC)
),
Integer.MAX_VALUE
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@ -7589,7 +7593,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
/**
* In Calcite 1.17, this test worked, but after upgrading to Calcite 1.21, this query fails with:
* org.apache.calcite.sql.validate.SqlValidatorException: Column 'dim1' is ambiguous
*/
@Test
@Ignore
public void testProjectAfterSort3() throws Exception
{
testQuery(
@ -7627,6 +7636,38 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testProjectAfterSort3WithoutAmbiguity() throws Exception
{
// This query is equivalent to the one in testProjectAfterSort3 but renames the second grouping column
// to avoid the ambiguous name exception. The inner sort is also optimized out in Calcite 1.21.
testQuery(
"select copydim1 from (select dim1, dim1 AS copydim1, count(*) cnt from druid.foo group by dim1, dim1 order by cnt)",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
dimensions(
new DefaultDimensionSpec("dim1", "d0")
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{""},
new Object[]{"1"},
new Object[]{"10.1"},
new Object[]{"2"},
new Object[]{"abc"},
new Object[]{"def"}
)
);
}
@Test
public void testSortProjectAfterNestedGroupBy() throws Exception
{
@ -7658,8 +7699,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(
new DefaultDimensionSpec("__time", "d0", ValueType.LONG),
new DefaultDimensionSpec("m2", "d1", ValueType.DOUBLE),
new DefaultDimensionSpec("dim1", "d2")
new DefaultDimensionSpec("dim1", "d1"),
new DefaultDimensionSpec("m2", "d2", ValueType.DOUBLE)
))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
@ -7668,19 +7709,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(
new DefaultDimensionSpec("d0", "_d0", ValueType.LONG),
new DefaultDimensionSpec("d2", "_d1", ValueType.STRING)
new DefaultDimensionSpec("d1", "_d1", ValueType.STRING)
))
.setAggregatorSpecs(aggregators(
new CountAggregatorFactory("a0")
))
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(
new OrderByColumnSpec("a0", Direction.ASCENDING, StringComparators.NUMERIC)
),
Integer.MAX_VALUE
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@ -8154,17 +8187,19 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.filters(selector("dim2", "0", null))
.filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
.granularity(Granularities.ALL)
.aggregators(aggregators(
new CountAggregatorFactory("a0")
))
// after upgrading to Calcite 1.21, expressions like sin(pi/6) that only reference
// literals are optimized into literals
.postAggregators(
expressionPostAgg("p0", "(exp(\"a0\") + 10)"),
expressionPostAgg("p1", "sin((pi() / 6))"),
expressionPostAgg("p2", "cos((pi() / 6))"),
expressionPostAgg("p3", "tan((pi() / 6))"),
expressionPostAgg("p4", "cot((pi() / 6))"),
expressionPostAgg("p1", "0.49999999999999994"),
expressionPostAgg("p2", "0.8660254037844387"),
expressionPostAgg("p3", "0.5773502691896257"),
expressionPostAgg("p4", "1.7320508075688776"),
expressionPostAgg("p5", "asin((exp(\"a0\") / 2))"),
expressionPostAgg("p6", "acos((exp(\"a0\") / 2))"),
expressionPostAgg("p7", "atan((exp(\"a0\") / 2))"),
@ -9215,4 +9250,130 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
results
);
}
@Test
public void testLeftRightStringOperators() throws Exception
{
testQuery(
"SELECT\n"
+ " dim1,"
+ " LEFT(dim1, 2),\n"
+ " RIGHT(dim1, 2)\n"
+ "FROM druid.foo\n"
+ "GROUP BY dim1\n",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0")))
.setPostAggregatorSpecs(ImmutableList.of(
expressionPostAgg("p0", "left(\"d0\",2)"),
expressionPostAgg("p1", "right(\"d0\",2)")
))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{"", "", ""},
new Object[]{"1", "1", "1"},
new Object[]{"10.1", "10", ".1"},
new Object[]{"2", "2", "2"},
new Object[]{"abc", "ab", "bc"},
new Object[]{"def", "de", "ef"}
)
);
}
@Test
public void testQueryContextOuterLimit() throws Exception
{
Map<String, Object> outerLimitContext = new HashMap<>(QUERY_CONTEXT_DEFAULT);
outerLimitContext.put(PlannerContext.CTX_SQL_OUTER_LIMIT, 4);
TopNQueryBuilder baseBuilder = new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim1", "d0"))
.metric(
new InvertedTopNMetricSpec(
new DimensionTopNMetricSpec(
null,
StringComparators.LEXICOGRAPHIC
)
)
)
.context(outerLimitContext);
List<Object[]> results1;
if (NullHandling.replaceWithDefault()) {
results1 = ImmutableList.of(
new Object[]{""},
new Object[]{"def"},
new Object[]{"abc"},
new Object[]{"2"}
);
} else {
results1 = ImmutableList.of(
new Object[]{"def"},
new Object[]{"abc"},
new Object[]{"2"},
new Object[]{"10.1"}
);
}
// no existing limit
testQuery(
PLANNER_CONFIG_DEFAULT,
outerLimitContext,
"SELECT dim1 FROM druid.foo GROUP BY dim1 ORDER BY dim1 DESC",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(
baseBuilder.threshold(4).build()
),
results1
);
// existing limit greater than context limit, override existing limit
testQuery(
PLANNER_CONFIG_DEFAULT,
outerLimitContext,
"SELECT dim1 FROM druid.foo GROUP BY dim1 ORDER BY dim1 DESC LIMIT 9",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(
baseBuilder.threshold(4).build()
),
results1
);
List<Object[]> results2;
if (NullHandling.replaceWithDefault()) {
results2 = ImmutableList.of(
new Object[]{""},
new Object[]{"def"}
);
} else {
results2 = ImmutableList.of(
new Object[]{"def"},
new Object[]{"abc"}
);
}
// existing limit less than context limit, keep existing limit
testQuery(
PLANNER_CONFIG_DEFAULT,
outerLimitContext,
"SELECT dim1 FROM druid.foo GROUP BY dim1 ORDER BY dim1 DESC LIMIT 2",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(
baseBuilder.threshold(2).build()
),
results2
);
}
}

View File

@ -129,6 +129,7 @@ import org.joda.time.DateTime;
import org.joda.time.chrono.ISOChronology;
import java.io.File;
import java.lang.reflect.InvocationTargetException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
@ -804,4 +805,19 @@ public class CalciteTests
);
return schema;
}
/**
* Some Calcite exceptions (such as that thrown by
* {@link org.apache.druid.sql.calcite.CalciteQueryTest#testCountStarWithTimeFilterUsingStringLiteralsInvalid)},
* are structured as a chain of RuntimeExceptions caused by InvocationTargetExceptions. To get the root exception
* it is necessary to make getTargetException calls on the InvocationTargetExceptions.
*/
public static Throwable getRootCauseFromInvocationTargetExceptionChain(Throwable t)
{
Throwable curThrowable = t;
while (curThrowable.getCause() instanceof InvocationTargetException) {
curThrowable = ((InvocationTargetException) curThrowable.getCause()).getTargetException();
}
return curThrowable;
}
}