SQL: Remove "useFallback" feature. (#7567)

This feature allows Calcite's Bindable interpreter to be bolted on
top of Druid queries and table scans. I think it should be removed for
a few reasons:

1. It is not recommended for production anyway, because it generates
unscalable query plans (e.g. it will plan a join into two table scans
and then try to do the entire join in memory on the broker).
2. It doesn't work with Druid-specific SQL functions, like TIME_FLOOR,
REGEXP_EXTRACT, APPROX_COUNT_DISTINCT, etc.
3. It makes the SQL planning code needlessly complicated.

With SQL coming out of experimental status soon, it's a good opportunity
to remove this feature.
This commit is contained in:
Gian Merlino 2019-04-28 18:26:44 -07:00 committed by Fangjin Yang
parent f02251ab2d
commit c648775b5b
14 changed files with 25 additions and 315 deletions

View File

@ -1424,7 +1424,6 @@ The Druid SQL server is configured through the following properties on the Broke
|`druid.sql.planner.selectThreshold`|Page size threshold for [Select queries](../querying/select-query.html). Select queries for larger resultsets will be issued back-to-back using pagination.|1000|
|`druid.sql.planner.useApproximateCountDistinct`|Whether to use an approximate cardinalty algorithm for `COUNT(DISTINCT foo)`.|true|
|`druid.sql.planner.useApproximateTopN`|Whether to use approximate [TopN queries](../querying/topnquery.html) when a SQL query could be expressed as such. If false, exact [GroupBy queries](../querying/groupbyquery.html) will be used instead.|true|
|`druid.sql.planner.useFallback`|Whether to evaluate operations on the Broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|false|
|`druid.sql.planner.requireTimeCondition`|Whether to require SQL to have filter conditions on __time column so that all generated native queries will have user specified intervals. If true, all queries wihout filter condition on __time column will fail|false|
|`druid.sql.planner.sqlTimeZone`|Sets the default time zone for the server, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|UTC|
|`druid.sql.planner.serializeComplexValues`|Whether to serialize "complex" output values, false will return the class name instead of the serialized value.|true|

View File

@ -524,7 +524,6 @@ Connection context can be specified as JDBC connection properties or as a "conte
|`sqlTimeZone`|Sets the time zone for this connection, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|druid.sql.planner.sqlTimeZone on the Broker (default: UTC)|
|`useApproximateCountDistinct`|Whether to use an approximate cardinalty algorithm for `COUNT(DISTINCT foo)`.|druid.sql.planner.useApproximateCountDistinct on the Broker (default: true)|
|`useApproximateTopN`|Whether to use approximate [TopN queries](topnquery.html) when a SQL query could be expressed as such. If false, exact [GroupBy queries](groupbyquery.html) will be used instead.|druid.sql.planner.useApproximateTopN on the Broker (default: true)|
|`useFallback`|Whether to evaluate operations on the Broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|druid.sql.planner.useFallback on the Broker (default: false)|
### Retrieving metadata
@ -725,7 +724,6 @@ The Druid SQL server is configured through the following properties on the Broke
|`druid.sql.planner.metadataRefreshPeriod`|Throttle for metadata refreshes.|PT1M|
|`druid.sql.planner.useApproximateCountDistinct`|Whether to use an approximate cardinalty algorithm for `COUNT(DISTINCT foo)`.|true|
|`druid.sql.planner.useApproximateTopN`|Whether to use approximate [TopN queries](../querying/topnquery.html) when a SQL query could be expressed as such. If false, exact [GroupBy queries](../querying/groupbyquery.html) will be used instead.|true|
|`druid.sql.planner.useFallback`|Whether to evaluate operations on the Broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|false|
|`druid.sql.planner.requireTimeCondition`|Whether to require SQL to have filter conditions on __time column so that all generated native queries will have user specified intervals. If true, all queries wihout filter condition on __time column will fail|false|
|`druid.sql.planner.sqlTimeZone`|Sets the default time zone for the server, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|UTC|
|`druid.sql.planner.metadataSegmentCacheEnable`|Whether to keep a cache of published segments in broker. If true, broker polls coordinator in background to get segments from metadata store and maintains a local cache. If false, coordinator's REST api will be invoked when broker needs published segments info.|false|

View File

@ -32,11 +32,9 @@ import org.apache.calcite.interpreter.Bindables;
import org.apache.calcite.linq4j.Enumerable;
import org.apache.calcite.linq4j.Enumerator;
import org.apache.calcite.plan.RelOptPlanner;
import org.apache.calcite.plan.RelOptTable;
import org.apache.calcite.plan.RelOptUtil;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelRoot;
import org.apache.calcite.rel.RelVisitor;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexNode;
@ -57,7 +55,6 @@ import org.apache.druid.sql.calcite.rel.DruidRel;
import java.io.Closeable;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
@ -92,7 +89,7 @@ public class DruidPlanner implements Closeable
return planWithDruidConvention(explain, root);
}
catch (RelOptPlanner.CannotPlanException e) {
// Try again with BINDABLE convention. Used for querying Values, metadata tables, and fallback.
// Try again with BINDABLE convention. Used for querying Values and metadata tables.
try {
return planWithBindableConvention(explain, root);
}
@ -193,29 +190,8 @@ public class DruidPlanner implements Closeable
);
}
final Set<String> datasourceNames = new HashSet<>();
bindableRel.childrenAccept(
new RelVisitor()
{
@Override
public void visit(RelNode node, int ordinal, RelNode parent)
{
if (node instanceof DruidRel) {
datasourceNames.addAll(((DruidRel) node).getDataSourceNames());
}
if (node instanceof Bindables.BindableTableScan) {
Bindables.BindableTableScan bts = (Bindables.BindableTableScan) node;
RelOptTable table = bts.getTable();
String tableName = table.getQualifiedName().get(0);
datasourceNames.add(tableName);
}
node.childrenAccept(this);
}
}
);
if (explain != null) {
return planExplanation(bindableRel, explain, datasourceNames);
return planExplanation(bindableRel, explain, ImmutableSet.of());
} else {
final BindableRel theRel = bindableRel;
final DataContext dataContext = plannerContext.createDataContext((JavaTypeFactory) planner.getTypeFactory());
@ -252,7 +228,7 @@ public class DruidPlanner implements Closeable
}
), () -> enumerator.close());
};
return new PlannerResult(resultsSupplier, root.validatedRowType, datasourceNames);
return new PlannerResult(resultsSupplier, root.validatedRowType, ImmutableSet.of());
}
}

View File

@ -31,7 +31,6 @@ public class PlannerConfig
{
public static final String CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT = "useApproximateCountDistinct";
public static final String CTX_KEY_USE_APPROXIMATE_TOPN = "useApproximateTopN";
public static final String CTX_KEY_USE_FALLBACK = "useFallback";
@JsonProperty
private Period metadataRefreshPeriod = new Period("PT1M");
@ -51,9 +50,6 @@ public class PlannerConfig
@JsonProperty
private boolean useApproximateTopN = true;
@JsonProperty
private boolean useFallback = false;
@JsonProperty
private boolean requireTimeCondition = false;
@ -111,11 +107,6 @@ public class PlannerConfig
return useApproximateTopN;
}
public boolean isUseFallback()
{
return useFallback;
}
public boolean isRequireTimeCondition()
{
return requireTimeCondition;
@ -157,11 +148,6 @@ public class PlannerConfig
CTX_KEY_USE_APPROXIMATE_TOPN,
isUseApproximateTopN()
);
newConfig.useFallback = getContextBoolean(
context,
CTX_KEY_USE_FALLBACK,
isUseFallback()
);
newConfig.requireTimeCondition = isRequireTimeCondition();
newConfig.sqlTimeZone = getSqlTimeZone();
newConfig.awaitInitializationOnStart = isAwaitInitializationOnStart();
@ -204,7 +190,6 @@ public class PlannerConfig
maxQueryCount == that.maxQueryCount &&
useApproximateCountDistinct == that.useApproximateCountDistinct &&
useApproximateTopN == that.useApproximateTopN &&
useFallback == that.useFallback &&
requireTimeCondition == that.requireTimeCondition &&
awaitInitializationOnStart == that.awaitInitializationOnStart &&
metadataSegmentCacheEnable == that.metadataSegmentCacheEnable &&
@ -225,7 +210,6 @@ public class PlannerConfig
maxQueryCount,
useApproximateCountDistinct,
useApproximateTopN,
useFallback,
requireTimeCondition,
awaitInitializationOnStart,
sqlTimeZone,
@ -245,7 +229,6 @@ public class PlannerConfig
", maxQueryCount=" + maxQueryCount +
", useApproximateCountDistinct=" + useApproximateCountDistinct +
", useApproximateTopN=" + useApproximateTopN +
", useFallback=" + useFallback +
", requireTimeCondition=" + requireTimeCondition +
", awaitInitializationOnStart=" + awaitInitializationOnStart +
", metadataSegmentCacheEnable=" + metadataSegmentCacheEnable +

View File

@ -72,7 +72,6 @@ import org.apache.calcite.tools.Programs;
import org.apache.calcite.tools.RelBuilder;
import org.apache.druid.sql.calcite.rel.QueryMaker;
import org.apache.druid.sql.calcite.rule.CaseFilteredAggregatorRule;
import org.apache.druid.sql.calcite.rule.DruidRelToBindableRule;
import org.apache.druid.sql.calcite.rule.DruidRelToDruidRule;
import org.apache.druid.sql.calcite.rule.DruidRules;
import org.apache.druid.sql.calcite.rule.DruidSemiJoinRule;
@ -187,7 +186,7 @@ public class Rules
);
return ImmutableList.of(
Programs.sequence(hepProgram, Programs.ofRules(druidConventionRuleSet(plannerContext, queryMaker))),
Programs.sequence(hepProgram, Programs.ofRules(bindableConventionRuleSet(plannerContext, queryMaker)))
Programs.sequence(hepProgram, Programs.ofRules(bindableConventionRuleSet(plannerContext)))
);
}
@ -196,28 +195,29 @@ public class Rules
final QueryMaker queryMaker
)
{
return ImmutableList.<RelOptRule>builder()
.addAll(baseRuleSet(plannerContext, queryMaker))
final ImmutableList.Builder<RelOptRule> retVal = ImmutableList.<RelOptRule>builder()
.addAll(baseRuleSet(plannerContext))
.add(DruidRelToDruidRule.instance())
.build();
.add(new DruidTableScanRule(queryMaker))
.addAll(DruidRules.rules());
if (plannerContext.getPlannerConfig().getMaxSemiJoinRowsInMemory() > 0) {
retVal.add(DruidSemiJoinRule.instance());
}
return retVal.build();
}
private static List<RelOptRule> bindableConventionRuleSet(
final PlannerContext plannerContext,
final QueryMaker queryMaker
)
private static List<RelOptRule> bindableConventionRuleSet(final PlannerContext plannerContext)
{
return ImmutableList.<RelOptRule>builder()
.addAll(baseRuleSet(plannerContext, queryMaker))
.addAll(baseRuleSet(plannerContext))
.addAll(Bindables.RULES)
.add(AggregateReduceFunctionsRule.INSTANCE)
.build();
}
private static List<RelOptRule> baseRuleSet(
final PlannerContext plannerContext,
final QueryMaker queryMaker
)
private static List<RelOptRule> baseRuleSet(final PlannerContext plannerContext)
{
final PlannerConfig plannerConfig = plannerContext.getPlannerConfig();
final ImmutableList.Builder<RelOptRule> rules = ImmutableList.builder();
@ -236,22 +236,10 @@ public class Rules
rules.add(AggregateExpandDistinctAggregatesRule.JOIN);
}
if (plannerConfig.isUseFallback()) {
rules.add(DruidRelToBindableRule.instance());
}
rules.add(SortCollapseRule.instance());
rules.add(CaseFilteredAggregatorRule.instance());
rules.add(ProjectAggregatePruneUnusedCallRule.instance());
// Druid-specific rules.
rules.add(new DruidTableScanRule(queryMaker));
rules.addAll(DruidRules.rules());
if (plannerConfig.getMaxSemiJoinRowsInMemory() > 0) {
rules.add(DruidSemiJoinRule.instance());
}
return rules.build();
}

View File

@ -22,7 +22,6 @@ package org.apache.druid.sql.calcite.rel;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import org.apache.calcite.interpreter.BindableConvention;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptCost;
import org.apache.calcite.plan.RelOptPlanner;
@ -154,18 +153,6 @@ public class DruidOuterQueryRel extends DruidRel<DruidOuterQueryRel>
);
}
@Override
public DruidOuterQueryRel asBindable()
{
return new DruidOuterQueryRel(
getCluster(),
getTraitSet().plus(BindableConvention.INSTANCE),
sourceRel,
partialQuery,
getQueryMaker()
);
}
@Override
public DruidOuterQueryRel asDruidConvention()
{

View File

@ -21,7 +21,6 @@ package org.apache.druid.sql.calcite.rel;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.google.common.base.Preconditions;
import org.apache.calcite.interpreter.BindableConvention;
import org.apache.calcite.plan.Convention;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptCost;
@ -110,19 +109,6 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
return toDruidQuery(false);
}
@Override
public DruidQueryRel asBindable()
{
return new DruidQueryRel(
getCluster(),
getTraitSet().plus(BindableConvention.INSTANCE),
table,
druidTable,
getQueryMaker(),
partialQuery
);
}
@Override
public DruidQueryRel asDruidConvention()
{

View File

@ -19,12 +19,6 @@
package org.apache.druid.sql.calcite.rel;
import org.apache.calcite.DataContext;
import org.apache.calcite.interpreter.BindableRel;
import org.apache.calcite.interpreter.Node;
import org.apache.calcite.interpreter.Row;
import org.apache.calcite.interpreter.Sink;
import org.apache.calcite.linq4j.Enumerable;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.AbstractRelNode;
@ -34,7 +28,7 @@ import org.apache.druid.sql.calcite.planner.PlannerContext;
import javax.annotation.Nullable;
import java.util.List;
public abstract class DruidRel<T extends DruidRel> extends AbstractRelNode implements BindableRel
public abstract class DruidRel<T extends DruidRel> extends AbstractRelNode
{
private final QueryMaker queryMaker;
@ -103,8 +97,6 @@ public abstract class DruidRel<T extends DruidRel> extends AbstractRelNode imple
*/
public abstract DruidQuery toDruidQueryForExplaining();
public abstract T asBindable();
public QueryMaker getQueryMaker()
{
return queryMaker;
@ -121,34 +113,4 @@ public abstract class DruidRel<T extends DruidRel> extends AbstractRelNode imple
* Get a list of names of datasources read by this DruidRel
*/
public abstract List<String> getDataSourceNames();
@Override
public Class<Object[]> getElementType()
{
return Object[].class;
}
@Override
public Node implement(InterpreterImplementor implementor)
{
final Sink sink = implementor.compiler.sink(this);
return () -> runQuery().accumulate(
sink,
(Sink theSink, Object[] in) -> {
try {
theSink.send(Row.of(in));
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
return theSink;
}
);
}
@Override
public Enumerable<Object[]> bind(final DataContext dataContext)
{
throw new UnsupportedOperationException();
}
}

View File

@ -22,7 +22,6 @@ package org.apache.druid.sql.calcite.rel;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import org.apache.calcite.interpreter.BindableConvention;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptCost;
import org.apache.calcite.plan.RelOptPlanner;
@ -156,21 +155,6 @@ public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
return left.toDruidQueryForExplaining();
}
@Override
public DruidSemiJoin asBindable()
{
return new DruidSemiJoin(
getCluster(),
getTraitSet().replace(BindableConvention.INSTANCE),
left,
RelOptRule.convert(right, BindableConvention.INSTANCE),
leftExpressions,
rightKeys,
maxSemiJoinRowsInMemory,
getQueryMaker()
);
}
@Override
public DruidSemiJoin asDruidConvention()
{

View File

@ -21,7 +21,6 @@ package org.apache.druid.sql.calcite.rel;
import com.google.common.base.Preconditions;
import com.google.common.collect.FluentIterable;
import org.apache.calcite.interpreter.BindableConvention;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptCost;
import org.apache.calcite.plan.RelOptPlanner;
@ -128,19 +127,6 @@ public class DruidUnionRel extends DruidRel<DruidUnionRel>
throw new UnsupportedOperationException();
}
@Override
public DruidUnionRel asBindable()
{
return new DruidUnionRel(
getCluster(),
getTraitSet().replace(BindableConvention.INSTANCE),
getQueryMaker(),
rowType,
rels.stream().map(rel -> RelOptRule.convert(rel, BindableConvention.INSTANCE)).collect(Collectors.toList()),
limit
);
}
@Override
public DruidUnionRel asDruidConvention()
{

View File

@ -1,52 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.sql.calcite.rule;
import org.apache.calcite.interpreter.BindableConvention;
import org.apache.calcite.plan.Convention;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.convert.ConverterRule;
import org.apache.druid.sql.calcite.rel.DruidRel;
public class DruidRelToBindableRule extends ConverterRule
{
private static DruidRelToBindableRule INSTANCE = new DruidRelToBindableRule();
private DruidRelToBindableRule()
{
super(
DruidRel.class,
Convention.NONE,
BindableConvention.INSTANCE,
DruidRelToBindableRule.class.getSimpleName()
);
}
public static DruidRelToBindableRule instance()
{
return INSTANCE;
}
@Override
public RelNode convert(RelNode rel)
{
return ((DruidRel) rel).asBindable();
}
}

View File

@ -27,7 +27,7 @@ import org.apache.druid.sql.calcite.rel.DruidRel;
public class DruidRelToDruidRule extends ConverterRule
{
private static DruidRelToDruidRule INSTANCE = new DruidRelToDruidRule();
private static final DruidRelToDruidRule INSTANCE = new DruidRelToDruidRule();
private DruidRelToDruidRule()
{

View File

@ -134,14 +134,6 @@ public class BaseCalciteQueryTest extends CalciteTestBase
return false;
}
};
public static final PlannerConfig PLANNER_CONFIG_FALLBACK = new PlannerConfig()
{
@Override
public boolean isUseFallback()
{
return true;
}
};
public static final PlannerConfig PLANNER_CONFIG_SINGLE_NESTING_ONLY = new PlannerConfig()
{
@Override

View File

@ -88,11 +88,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testSelectConstantExpression() throws Exception
{
// Test with a Druid-specific function, to make sure they are hooked up correctly even when not selecting
// from a table.
testQuery(
"SELECT 1 + 1",
"SELECT REGEXP_EXTRACT('foo', '^(.)')",
ImmutableList.of(),
ImmutableList.of(
new Object[]{2}
new Object[]{"f"}
)
);
}
@ -776,67 +778,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testSelfJoinWithFallback() throws Exception
{
testQuery(
PLANNER_CONFIG_FALLBACK,
"SELECT x.dim1, y.dim1, y.dim2\n"
+ "FROM\n"
+ " druid.foo x INNER JOIN druid.foo y ON x.dim1 = y.dim2\n"
+ "WHERE\n"
+ " x.dim1 <> ''",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim1")
.filters(not(selector("dim1", "", null)))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build(),
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim1", "dim2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{"abc", "def", "abc"}
)
);
}
@Test
public void testExplainSelfJoinWithFallback() throws Exception
{
String emptyStringEq = NullHandling.replaceWithDefault() ? null : "\"\"";
final String explanation =
"BindableJoin(condition=[=($0, $2)], joinType=[inner])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"order\":\"none\",\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":"
+ emptyStringEq
+ ",\"extractionFn\":null}},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"order\":\"none\",\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING, dim2:STRING}])\n";
testQuery(
PLANNER_CONFIG_FALLBACK,
"EXPLAIN PLAN FOR\n"
+ "SELECT x.dim1, y.dim1, y.dim2\n"
+ "FROM\n"
+ " druid.foo x INNER JOIN druid.foo y ON x.dim1 = y.dim2\n"
+ "WHERE\n"
+ " x.dim1 <> ''",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(),
ImmutableList.of(
new Object[]{explanation}
)
);
}
@Test
public void testGroupByLong() throws Exception
{
@ -1238,9 +1179,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
public void testHavingOnFloatSum() throws Exception
{
testQuery(
PLANNER_CONFIG_FALLBACK,
"SELECT dim1, CAST(SUM(m1) AS FLOAT) AS m1_sum FROM druid.foo GROUP BY dim1 HAVING CAST(SUM(m1) AS FLOAT) > 1",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
@ -6722,34 +6661,16 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
}
@Test
public void testUsingSubqueryAsPartOfOrFilter() throws Exception
public void testUsingSubqueryAsPartOfOrFilter()
{
// This query should ideally be plannable without fallback, but it's not. The "OR" means it isn't really
// This query should ideally be plannable, but it's not. The "OR" means it isn't really
// a semiJoin and so the filter condition doesn't get converted.
final String explanation =
"BindableSort(sort0=[$1], dir0=[ASC])\n"
+ " BindableAggregate(group=[{0, 1}], EXPR$2=[COUNT()])\n"
+ " BindableFilter(condition=[OR(=($0, 'xxx'), CAST(AND(IS NOT NULL($4), <>($2, 0), IS NOT NULL($1))):BOOLEAN)])\n"
+ " BindableJoin(condition=[=($1, $3)], joinType=[left])\n"
+ " BindableJoin(condition=[true], joinType=[inner])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"order\":\"none\",\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING, dim2:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":{\"type\":\"like\",\"dimension\":\"dim1\",\"pattern\":\"%bc\",\"escape\":null,\"extractionFn\":null},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"}}], signature=[{a0:LONG}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"1\",\"outputType\":\"LONG\"}],\"filter\":{\"type\":\"like\",\"dimension\":\"dim1\",\"pattern\":\"%bc\",\"escape\":null,\"extractionFn\":null},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"},{\"type\":\"default\",\"dimension\":\"v0\",\"outputName\":\"v0\",\"outputType\":\"LONG\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false}], signature=[{d0:STRING, v0:LONG}])\n";
final String theQuery = "SELECT dim1, dim2, COUNT(*) FROM druid.foo\n"
+ "WHERE dim1 = 'xxx' OR dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 LIKE '%bc')\n"
+ "group by dim1, dim2 ORDER BY dim2";
assertQueryIsUnplannable(theQuery);
testQuery(
PLANNER_CONFIG_FALLBACK,
"EXPLAIN PLAN FOR " + theQuery,
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(),
ImmutableList.of(new Object[]{explanation})
);
}
@Test