mirror of https://github.com/apache/druid.git
Decouple logical planning and native query generation in SQL planning (#14232)
Add a new planning strategy that explicitly decouples the DAG from building the native query. With this mode, it is Calcite's job to generate a "logical DAG" which is all of the various DruidProject, DruidFilter, etc. nodes. We then take those nodes and use them to build a native query. The current commit doesn't pass all tests, but it does work for some things and is a decent starting baseline.
This commit is contained in:
parent
50461c3bd5
commit
09d6c5a45e
|
@ -0,0 +1,288 @@
|
||||||
|
/*
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
|
||||||
|
//CHECKSTYLE.OFF: PackageName - Must be in Calcite
|
||||||
|
|
||||||
|
package org.apache.calcite.plan.volcano;
|
||||||
|
|
||||||
|
import org.apache.calcite.plan.RelOptCost;
|
||||||
|
import org.apache.calcite.plan.RelOptCostFactory;
|
||||||
|
import org.apache.calcite.plan.RelOptUtil;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Druid's extension to {@link VolcanoCost}. The difference between the two is in
|
||||||
|
* comparing two costs. Druid's cost model gives most weightage to rowCount, then to cpuCost and then lastly ioCost.
|
||||||
|
*/
|
||||||
|
public class DruidVolcanoCost implements RelOptCost
|
||||||
|
{
|
||||||
|
|
||||||
|
static final DruidVolcanoCost INFINITY = new DruidVolcanoCost(
|
||||||
|
Double.POSITIVE_INFINITY,
|
||||||
|
Double.POSITIVE_INFINITY,
|
||||||
|
Double.POSITIVE_INFINITY
|
||||||
|
)
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "{inf}";
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
//CHECKSTYLE.OFF: Regexp
|
||||||
|
static final DruidVolcanoCost HUGE = new DruidVolcanoCost(Double.MAX_VALUE, Double.MAX_VALUE, Double.MAX_VALUE) {
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "{huge}";
|
||||||
|
}
|
||||||
|
};
|
||||||
|
//CHECKSTYLE.ON: Regexp
|
||||||
|
|
||||||
|
static final DruidVolcanoCost ZERO =
|
||||||
|
new DruidVolcanoCost(0.0, 0.0, 0.0)
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "{0}";
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
static final DruidVolcanoCost TINY =
|
||||||
|
new DruidVolcanoCost(1.0, 1.0, 0.0)
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "{tiny}";
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
public static final RelOptCostFactory FACTORY = new Factory();
|
||||||
|
|
||||||
|
final double cpu;
|
||||||
|
final double io;
|
||||||
|
final double rowCount;
|
||||||
|
|
||||||
|
DruidVolcanoCost(double rowCount, double cpu, double io)
|
||||||
|
{
|
||||||
|
this.rowCount = rowCount;
|
||||||
|
this.cpu = cpu;
|
||||||
|
this.io = io;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double getCpu()
|
||||||
|
{
|
||||||
|
return cpu;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isInfinite()
|
||||||
|
{
|
||||||
|
return (this == INFINITY)
|
||||||
|
|| (this.rowCount == Double.POSITIVE_INFINITY)
|
||||||
|
|| (this.cpu == Double.POSITIVE_INFINITY)
|
||||||
|
|| (this.io == Double.POSITIVE_INFINITY);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double getIo()
|
||||||
|
{
|
||||||
|
return io;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isLe(RelOptCost other)
|
||||||
|
{
|
||||||
|
DruidVolcanoCost that = (DruidVolcanoCost) other;
|
||||||
|
return (this == that)
|
||||||
|
|| ((this.rowCount < that.rowCount)
|
||||||
|
|| (this.rowCount == that.rowCount && this.cpu < that.cpu)
|
||||||
|
|| (this.rowCount == that.rowCount && this.cpu == that.cpu && this.io <= that.io));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isLt(RelOptCost other)
|
||||||
|
{
|
||||||
|
return isLe(other) && !equals(other);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double getRows()
|
||||||
|
{
|
||||||
|
return rowCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(rowCount, cpu, io);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(RelOptCost other)
|
||||||
|
{
|
||||||
|
return this == other
|
||||||
|
|| other instanceof DruidVolcanoCost
|
||||||
|
&& (this.rowCount == ((DruidVolcanoCost) other).rowCount)
|
||||||
|
&& (this.cpu == ((DruidVolcanoCost) other).cpu)
|
||||||
|
&& (this.io == ((DruidVolcanoCost) other).io);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object obj)
|
||||||
|
{
|
||||||
|
if (obj instanceof DruidVolcanoCost) {
|
||||||
|
return equals((DruidVolcanoCost) obj);
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isEqWithEpsilon(RelOptCost other)
|
||||||
|
{
|
||||||
|
if (!(other instanceof DruidVolcanoCost)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
DruidVolcanoCost that = (DruidVolcanoCost) other;
|
||||||
|
return (this == that)
|
||||||
|
|| ((Math.abs(this.rowCount - that.rowCount) < RelOptUtil.EPSILON)
|
||||||
|
&& (Math.abs(this.cpu - that.cpu) < RelOptUtil.EPSILON)
|
||||||
|
&& (Math.abs(this.io - that.io) < RelOptUtil.EPSILON));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost minus(RelOptCost other)
|
||||||
|
{
|
||||||
|
if (this == INFINITY) {
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
DruidVolcanoCost that = (DruidVolcanoCost) other;
|
||||||
|
return new DruidVolcanoCost(
|
||||||
|
this.rowCount - that.rowCount,
|
||||||
|
this.cpu - that.cpu,
|
||||||
|
this.io - that.io
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost multiplyBy(double factor)
|
||||||
|
{
|
||||||
|
if (this == INFINITY) {
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
return new DruidVolcanoCost(rowCount * factor, cpu * factor, io * factor);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double divideBy(RelOptCost cost)
|
||||||
|
{
|
||||||
|
// Compute the geometric average of the ratios of all of the factors
|
||||||
|
// which are non-zero and finite.
|
||||||
|
DruidVolcanoCost that = (DruidVolcanoCost) cost;
|
||||||
|
double d = 1;
|
||||||
|
double n = 0;
|
||||||
|
if ((this.rowCount != 0)
|
||||||
|
&& !Double.isInfinite(this.rowCount)
|
||||||
|
&& (that.rowCount != 0)
|
||||||
|
&& !Double.isInfinite(that.rowCount)) {
|
||||||
|
d *= this.rowCount / that.rowCount;
|
||||||
|
++n;
|
||||||
|
}
|
||||||
|
if ((this.cpu != 0)
|
||||||
|
&& !Double.isInfinite(this.cpu)
|
||||||
|
&& (that.cpu != 0)
|
||||||
|
&& !Double.isInfinite(that.cpu)) {
|
||||||
|
d *= this.cpu / that.cpu;
|
||||||
|
++n;
|
||||||
|
}
|
||||||
|
if ((this.io != 0)
|
||||||
|
&& !Double.isInfinite(this.io)
|
||||||
|
&& (that.io != 0)
|
||||||
|
&& !Double.isInfinite(that.io)) {
|
||||||
|
d *= this.io / that.io;
|
||||||
|
++n;
|
||||||
|
}
|
||||||
|
if (n == 0) {
|
||||||
|
return 1.0;
|
||||||
|
}
|
||||||
|
return Math.pow(d, 1 / n);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost plus(RelOptCost other)
|
||||||
|
{
|
||||||
|
DruidVolcanoCost that = (DruidVolcanoCost) other;
|
||||||
|
if ((this == INFINITY) || (that == INFINITY)) {
|
||||||
|
return INFINITY;
|
||||||
|
}
|
||||||
|
return new DruidVolcanoCost(
|
||||||
|
this.rowCount + that.rowCount,
|
||||||
|
this.cpu + that.cpu,
|
||||||
|
this.io + that.io
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "{" + rowCount + " rows, " + cpu + " cpu, " + io + " io}";
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Implementation of {@link RelOptCostFactory}
|
||||||
|
* that creates {@link DruidVolcanoCost}s.
|
||||||
|
*/
|
||||||
|
public static class Factory implements RelOptCostFactory
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public RelOptCost makeCost(double dRows, double dCpu, double dIo)
|
||||||
|
{
|
||||||
|
return new DruidVolcanoCost(dRows, dCpu, dIo);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost makeHugeCost()
|
||||||
|
{
|
||||||
|
return DruidVolcanoCost.HUGE;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost makeInfiniteCost()
|
||||||
|
{
|
||||||
|
return DruidVolcanoCost.INFINITY;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost makeTinyCost()
|
||||||
|
{
|
||||||
|
return DruidVolcanoCost.TINY;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost makeZeroCost()
|
||||||
|
{
|
||||||
|
return DruidVolcanoCost.ZERO;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -26,6 +26,7 @@ import org.apache.calcite.plan.RelOptLattice;
|
||||||
import org.apache.calcite.plan.RelOptMaterialization;
|
import org.apache.calcite.plan.RelOptMaterialization;
|
||||||
import org.apache.calcite.plan.RelOptPlanner;
|
import org.apache.calcite.plan.RelOptPlanner;
|
||||||
import org.apache.calcite.plan.RelOptRule;
|
import org.apache.calcite.plan.RelOptRule;
|
||||||
|
import org.apache.calcite.plan.RelOptUtil;
|
||||||
import org.apache.calcite.plan.RelTraitSet;
|
import org.apache.calcite.plan.RelTraitSet;
|
||||||
import org.apache.calcite.plan.hep.HepProgram;
|
import org.apache.calcite.plan.hep.HepProgram;
|
||||||
import org.apache.calcite.plan.hep.HepProgramBuilder;
|
import org.apache.calcite.plan.hep.HepProgramBuilder;
|
||||||
|
@ -74,11 +75,14 @@ import org.apache.calcite.rel.rules.TableScanRule;
|
||||||
import org.apache.calcite.rel.rules.UnionPullUpConstantsRule;
|
import org.apache.calcite.rel.rules.UnionPullUpConstantsRule;
|
||||||
import org.apache.calcite.rel.rules.UnionToDistinctRule;
|
import org.apache.calcite.rel.rules.UnionToDistinctRule;
|
||||||
import org.apache.calcite.rel.rules.ValuesReduceRule;
|
import org.apache.calcite.rel.rules.ValuesReduceRule;
|
||||||
|
import org.apache.calcite.sql.SqlExplainFormat;
|
||||||
|
import org.apache.calcite.sql.SqlExplainLevel;
|
||||||
import org.apache.calcite.sql2rel.RelDecorrelator;
|
import org.apache.calcite.sql2rel.RelDecorrelator;
|
||||||
import org.apache.calcite.sql2rel.RelFieldTrimmer;
|
import org.apache.calcite.sql2rel.RelFieldTrimmer;
|
||||||
import org.apache.calcite.tools.Program;
|
import org.apache.calcite.tools.Program;
|
||||||
import org.apache.calcite.tools.Programs;
|
import org.apache.calcite.tools.Programs;
|
||||||
import org.apache.calcite.tools.RelBuilder;
|
import org.apache.calcite.tools.RelBuilder;
|
||||||
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import org.apache.druid.sql.calcite.external.ExternalTableScanRule;
|
import org.apache.druid.sql.calcite.external.ExternalTableScanRule;
|
||||||
import org.apache.druid.sql.calcite.rule.DruidLogicalValuesRule;
|
import org.apache.druid.sql.calcite.rule.DruidLogicalValuesRule;
|
||||||
import org.apache.druid.sql.calcite.rule.DruidRelToDruidRule;
|
import org.apache.druid.sql.calcite.rule.DruidRelToDruidRule;
|
||||||
|
@ -88,6 +92,8 @@ import org.apache.druid.sql.calcite.rule.ExtensionCalciteRuleProvider;
|
||||||
import org.apache.druid.sql.calcite.rule.FilterJoinExcludePushToChildRule;
|
import org.apache.druid.sql.calcite.rule.FilterJoinExcludePushToChildRule;
|
||||||
import org.apache.druid.sql.calcite.rule.ProjectAggregatePruneUnusedCallRule;
|
import org.apache.druid.sql.calcite.rule.ProjectAggregatePruneUnusedCallRule;
|
||||||
import org.apache.druid.sql.calcite.rule.SortCollapseRule;
|
import org.apache.druid.sql.calcite.rule.SortCollapseRule;
|
||||||
|
import org.apache.druid.sql.calcite.rule.logical.DruidAggregateCaseToFilterRule;
|
||||||
|
import org.apache.druid.sql.calcite.rule.logical.DruidLogicalRules;
|
||||||
import org.apache.druid.sql.calcite.run.EngineFeature;
|
import org.apache.druid.sql.calcite.run.EngineFeature;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -95,8 +101,11 @@ import java.util.Set;
|
||||||
|
|
||||||
public class CalciteRulesManager
|
public class CalciteRulesManager
|
||||||
{
|
{
|
||||||
|
private static final Logger log = new Logger(CalciteRulesManager.class);
|
||||||
|
|
||||||
public static final int DRUID_CONVENTION_RULES = 0;
|
public static final int DRUID_CONVENTION_RULES = 0;
|
||||||
public static final int BINDABLE_CONVENTION_RULES = 1;
|
public static final int BINDABLE_CONVENTION_RULES = 1;
|
||||||
|
public static final int DRUID_DAG_CONVENTION_RULES = 2;
|
||||||
|
|
||||||
// Due to Calcite bug (CALCITE-3845), ReduceExpressionsRule can considered expression which is the same as the
|
// Due to Calcite bug (CALCITE-3845), ReduceExpressionsRule can considered expression which is the same as the
|
||||||
// previous input expression as reduced. Basically, the expression is actually not reduced but is still considered as
|
// previous input expression as reduced. Basically, the expression is actually not reduced but is still considered as
|
||||||
|
@ -249,12 +258,56 @@ public class CalciteRulesManager
|
||||||
buildHepProgram(REDUCTION_RULES, true, DefaultRelMetadataProvider.INSTANCE, HEP_DEFAULT_MATCH_LIMIT)
|
buildHepProgram(REDUCTION_RULES, true, DefaultRelMetadataProvider.INSTANCE, HEP_DEFAULT_MATCH_LIMIT)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
boolean isDebug = plannerContext.queryContext().isDebug();
|
||||||
return ImmutableList.of(
|
return ImmutableList.of(
|
||||||
Programs.sequence(preProgram, Programs.ofRules(druidConventionRuleSet(plannerContext))),
|
Programs.sequence(preProgram, Programs.ofRules(druidConventionRuleSet(plannerContext))),
|
||||||
Programs.sequence(preProgram, Programs.ofRules(bindableConventionRuleSet(plannerContext)))
|
Programs.sequence(preProgram, Programs.ofRules(bindableConventionRuleSet(plannerContext))),
|
||||||
|
Programs.sequence(
|
||||||
|
// currently, adding logging program after every stage for easier debugging
|
||||||
|
new LoggingProgram("Start", isDebug),
|
||||||
|
Programs.subQuery(DefaultRelMetadataProvider.INSTANCE),
|
||||||
|
new LoggingProgram("After subquery program", isDebug),
|
||||||
|
DecorrelateAndTrimFieldsProgram.INSTANCE,
|
||||||
|
new LoggingProgram("After trim fields and decorelate program", isDebug),
|
||||||
|
buildHepProgram(REDUCTION_RULES, true, DefaultRelMetadataProvider.INSTANCE, HEP_DEFAULT_MATCH_LIMIT),
|
||||||
|
new LoggingProgram("After hep planner program", isDebug),
|
||||||
|
Programs.ofRules(logicalConventionRuleSet(plannerContext)),
|
||||||
|
new LoggingProgram("After volcano planner program", isDebug)
|
||||||
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static class LoggingProgram implements Program
|
||||||
|
{
|
||||||
|
private final String stage;
|
||||||
|
private final boolean isDebug;
|
||||||
|
|
||||||
|
public LoggingProgram(String stage, boolean isDebug)
|
||||||
|
{
|
||||||
|
this.stage = stage;
|
||||||
|
this.isDebug = isDebug;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode run(
|
||||||
|
RelOptPlanner planner,
|
||||||
|
RelNode rel,
|
||||||
|
RelTraitSet requiredOutputTraits,
|
||||||
|
List<RelOptMaterialization> materializations,
|
||||||
|
List<RelOptLattice> lattices
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (isDebug) {
|
||||||
|
log.info(
|
||||||
|
"%s%n%s",
|
||||||
|
stage,
|
||||||
|
RelOptUtil.dumpPlan("", rel, SqlExplainFormat.TEXT, SqlExplainLevel.ALL_ATTRIBUTES)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
return rel;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public Program buildHepProgram(
|
public Program buildHepProgram(
|
||||||
final Iterable<? extends RelOptRule> rules,
|
final Iterable<? extends RelOptRule> rules,
|
||||||
final boolean noDag,
|
final boolean noDag,
|
||||||
|
@ -287,6 +340,16 @@ public class CalciteRulesManager
|
||||||
return retVal.build();
|
return retVal.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public List<RelOptRule> logicalConventionRuleSet(final PlannerContext plannerContext)
|
||||||
|
{
|
||||||
|
final ImmutableList.Builder<RelOptRule> retVal = ImmutableList
|
||||||
|
.<RelOptRule>builder()
|
||||||
|
.addAll(baseRuleSet(plannerContext))
|
||||||
|
.add(DruidAggregateCaseToFilterRule.INSTANCE)
|
||||||
|
.add(new DruidLogicalRules(plannerContext).rules().toArray(new RelOptRule[0]));
|
||||||
|
return retVal.build();
|
||||||
|
}
|
||||||
|
|
||||||
public List<RelOptRule> bindableConventionRuleSet(final PlannerContext plannerContext)
|
public List<RelOptRule> bindableConventionRuleSet(final PlannerContext plannerContext)
|
||||||
{
|
{
|
||||||
return ImmutableList.<RelOptRule>builder()
|
return ImmutableList.<RelOptRule>builder()
|
||||||
|
|
|
@ -0,0 +1,336 @@
|
||||||
|
/*
|
||||||
|
* 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.planner;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import org.apache.calcite.plan.RelOptTable;
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
import org.apache.calcite.rel.RelShuttleImpl;
|
||||||
|
import org.apache.calcite.rel.core.Aggregate;
|
||||||
|
import org.apache.calcite.rel.core.Filter;
|
||||||
|
import org.apache.calcite.rel.core.Project;
|
||||||
|
import org.apache.calcite.rel.core.Sort;
|
||||||
|
import org.apache.calcite.rel.core.TableFunctionScan;
|
||||||
|
import org.apache.calcite.rel.core.TableScan;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalAggregate;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalCorrelate;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalExchange;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalFilter;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalIntersect;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalJoin;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalMatch;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalMinus;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalProject;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalSort;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalUnion;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalValues;
|
||||||
|
import org.apache.calcite.rex.RexLiteral;
|
||||||
|
import org.apache.druid.java.util.common.ISE;
|
||||||
|
import org.apache.druid.query.InlineDataSource;
|
||||||
|
import org.apache.druid.segment.column.RowSignature;
|
||||||
|
import org.apache.druid.sql.calcite.rel.PartialDruidQuery;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidTableScan;
|
||||||
|
import org.apache.druid.sql.calcite.rule.DruidLogicalValuesRule;
|
||||||
|
import org.apache.druid.sql.calcite.table.DruidTable;
|
||||||
|
import org.apache.druid.sql.calcite.table.InlineTable;
|
||||||
|
import org.apache.druid.sql.calcite.table.RowSignatures;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Converts a DAG of {@link org.apache.druid.sql.calcite.rel.logical.DruidLogicalNode} convention to a native
|
||||||
|
* Druid query for execution. The convertion is done via a {@link org.apache.calcite.rel.RelShuttle} visitor
|
||||||
|
* implementation.
|
||||||
|
*/
|
||||||
|
public class DruidQueryGenerator extends RelShuttleImpl
|
||||||
|
{
|
||||||
|
private final List<PartialDruidQuery> queryList = new ArrayList<>();
|
||||||
|
private final List<DruidTable> queryTables = new ArrayList<>();
|
||||||
|
private final PlannerContext plannerContext;
|
||||||
|
private PartialDruidQuery partialDruidQuery;
|
||||||
|
private PartialDruidQuery.Stage currentStage = null;
|
||||||
|
private DruidTable currentTable = null;
|
||||||
|
private boolean isRoot = true;
|
||||||
|
|
||||||
|
public DruidQueryGenerator(PlannerContext plannerContext)
|
||||||
|
{
|
||||||
|
this.plannerContext = plannerContext;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(TableScan scan)
|
||||||
|
{
|
||||||
|
if (!(scan instanceof DruidTableScan)) {
|
||||||
|
throw new ISE("Planning hasn't converted logical table scan to druid convention");
|
||||||
|
}
|
||||||
|
DruidTableScan druidTableScan = (DruidTableScan) scan;
|
||||||
|
isRoot = false;
|
||||||
|
RelNode result = super.visit(scan);
|
||||||
|
partialDruidQuery = PartialDruidQuery.create(scan);
|
||||||
|
currentStage = PartialDruidQuery.Stage.SCAN;
|
||||||
|
final RelOptTable table = scan.getTable();
|
||||||
|
final DruidTable druidTable = table.unwrap(DruidTable.class);
|
||||||
|
if (druidTable != null) {
|
||||||
|
currentTable = druidTable;
|
||||||
|
}
|
||||||
|
if (druidTableScan.getProject() != null) {
|
||||||
|
partialDruidQuery = partialDruidQuery.withSelectProject(druidTableScan.getProject());
|
||||||
|
currentStage = PartialDruidQuery.Stage.SELECT_PROJECT;
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(TableFunctionScan scan)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(LogicalValues values)
|
||||||
|
{
|
||||||
|
isRoot = false;
|
||||||
|
RelNode result = super.visit(values);
|
||||||
|
final List<ImmutableList<RexLiteral>> tuples = values.getTuples();
|
||||||
|
final List<Object[]> objectTuples = tuples
|
||||||
|
.stream()
|
||||||
|
.map(tuple -> tuple
|
||||||
|
.stream()
|
||||||
|
.map(v -> DruidLogicalValuesRule.getValueFromLiteral(v, plannerContext))
|
||||||
|
.collect(Collectors.toList())
|
||||||
|
.toArray(new Object[0])
|
||||||
|
)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
RowSignature rowSignature = RowSignatures.fromRelDataType(
|
||||||
|
values.getRowType().getFieldNames(),
|
||||||
|
values.getRowType()
|
||||||
|
);
|
||||||
|
currentTable = new InlineTable(InlineDataSource.fromIterable(objectTuples, rowSignature));
|
||||||
|
if (currentStage == null) {
|
||||||
|
partialDruidQuery = PartialDruidQuery.create(values);
|
||||||
|
currentStage = PartialDruidQuery.Stage.SCAN;
|
||||||
|
} else {
|
||||||
|
throw new ISE("Values node found at non leaf node in the plan");
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(LogicalFilter filter)
|
||||||
|
{
|
||||||
|
return visitFilter(filter);
|
||||||
|
}
|
||||||
|
|
||||||
|
public RelNode visitFilter(Filter filter)
|
||||||
|
{
|
||||||
|
isRoot = false;
|
||||||
|
RelNode result = super.visit(filter);
|
||||||
|
if (currentStage == PartialDruidQuery.Stage.AGGREGATE) {
|
||||||
|
partialDruidQuery = partialDruidQuery.withHavingFilter(filter);
|
||||||
|
currentStage = PartialDruidQuery.Stage.HAVING_FILTER;
|
||||||
|
} else if (currentStage == PartialDruidQuery.Stage.SCAN) {
|
||||||
|
partialDruidQuery = partialDruidQuery.withWhereFilter(filter);
|
||||||
|
currentStage = PartialDruidQuery.Stage.WHERE_FILTER;
|
||||||
|
} else if (currentStage == PartialDruidQuery.Stage.SELECT_PROJECT) {
|
||||||
|
PartialDruidQuery old = partialDruidQuery;
|
||||||
|
partialDruidQuery = PartialDruidQuery.create(old.getScan());
|
||||||
|
partialDruidQuery = partialDruidQuery.withWhereFilter(filter);
|
||||||
|
partialDruidQuery = partialDruidQuery.withSelectProject(old.getSelectProject());
|
||||||
|
currentStage = PartialDruidQuery.Stage.SELECT_PROJECT;
|
||||||
|
} else {
|
||||||
|
queryList.add(partialDruidQuery);
|
||||||
|
queryTables.add(currentTable);
|
||||||
|
partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withWhereFilter(filter);
|
||||||
|
currentStage = PartialDruidQuery.Stage.WHERE_FILTER;
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(LogicalProject project)
|
||||||
|
{
|
||||||
|
return visitProject(project);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(LogicalJoin join)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Found join");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(LogicalCorrelate correlate)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(LogicalUnion union)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Found union");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(LogicalIntersect intersect)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(LogicalMinus minus)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(LogicalAggregate aggregate)
|
||||||
|
{
|
||||||
|
isRoot = false;
|
||||||
|
RelNode result = super.visit(aggregate);
|
||||||
|
if (PartialDruidQuery.Stage.AGGREGATE.canFollow(currentStage)) {
|
||||||
|
partialDruidQuery = partialDruidQuery.withAggregate(aggregate);
|
||||||
|
} else {
|
||||||
|
queryList.add(partialDruidQuery);
|
||||||
|
queryTables.add(currentTable);
|
||||||
|
partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withAggregate(aggregate);
|
||||||
|
}
|
||||||
|
currentStage = PartialDruidQuery.Stage.AGGREGATE;
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(LogicalMatch match)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(LogicalSort sort)
|
||||||
|
{
|
||||||
|
return visitSort(sort);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(LogicalExchange exchange)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
private RelNode visitProject(Project project)
|
||||||
|
{
|
||||||
|
boolean rootForReal = isRoot;
|
||||||
|
isRoot = false;
|
||||||
|
RelNode result = super.visit(project);
|
||||||
|
if (rootForReal && (currentStage == PartialDruidQuery.Stage.AGGREGATE
|
||||||
|
|| currentStage == PartialDruidQuery.Stage.HAVING_FILTER)) {
|
||||||
|
partialDruidQuery = partialDruidQuery.withAggregateProject(project);
|
||||||
|
currentStage = PartialDruidQuery.Stage.AGGREGATE_PROJECT;
|
||||||
|
} else if (currentStage == PartialDruidQuery.Stage.SCAN || currentStage == PartialDruidQuery.Stage.WHERE_FILTER) {
|
||||||
|
partialDruidQuery = partialDruidQuery.withSelectProject(project);
|
||||||
|
currentStage = PartialDruidQuery.Stage.SELECT_PROJECT;
|
||||||
|
} else if (currentStage == PartialDruidQuery.Stage.SELECT_PROJECT) {
|
||||||
|
partialDruidQuery = partialDruidQuery.mergeProject(project);
|
||||||
|
currentStage = PartialDruidQuery.Stage.SELECT_PROJECT;
|
||||||
|
} else if (currentStage == PartialDruidQuery.Stage.SORT) {
|
||||||
|
partialDruidQuery = partialDruidQuery.withSortProject(project);
|
||||||
|
currentStage = PartialDruidQuery.Stage.SORT_PROJECT;
|
||||||
|
} else {
|
||||||
|
queryList.add(partialDruidQuery);
|
||||||
|
queryTables.add(currentTable);
|
||||||
|
partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withSelectProject(project);
|
||||||
|
currentStage = PartialDruidQuery.Stage.SELECT_PROJECT;
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
private RelNode visitSort(Sort sort)
|
||||||
|
{
|
||||||
|
isRoot = false;
|
||||||
|
RelNode result = super.visit(sort);
|
||||||
|
if (PartialDruidQuery.Stage.SORT.canFollow(currentStage)) {
|
||||||
|
partialDruidQuery = partialDruidQuery.withSort(sort);
|
||||||
|
} else {
|
||||||
|
queryList.add(partialDruidQuery);
|
||||||
|
queryTables.add(currentTable);
|
||||||
|
partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withSort(sort);
|
||||||
|
}
|
||||||
|
currentStage = PartialDruidQuery.Stage.SORT;
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
private RelNode visitAggregate(Aggregate aggregate)
|
||||||
|
{
|
||||||
|
isRoot = false;
|
||||||
|
RelNode result = super.visit(aggregate);
|
||||||
|
if (PartialDruidQuery.Stage.AGGREGATE.canFollow(currentStage)) {
|
||||||
|
partialDruidQuery = partialDruidQuery.withAggregate(aggregate);
|
||||||
|
} else {
|
||||||
|
queryList.add(partialDruidQuery);
|
||||||
|
queryTables.add(currentTable);
|
||||||
|
partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withAggregate(aggregate);
|
||||||
|
}
|
||||||
|
currentStage = PartialDruidQuery.Stage.AGGREGATE;
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode visit(RelNode other)
|
||||||
|
{
|
||||||
|
if (other instanceof TableScan) {
|
||||||
|
return visit((TableScan) other);
|
||||||
|
} else if (other instanceof Project) {
|
||||||
|
return visitProject((Project) other);
|
||||||
|
} else if (other instanceof Sort) {
|
||||||
|
return visitSort((Sort) other);
|
||||||
|
} else if (other instanceof Aggregate) {
|
||||||
|
return visitAggregate((Aggregate) other);
|
||||||
|
} else if (other instanceof Filter) {
|
||||||
|
return visitFilter((Filter) other);
|
||||||
|
} else if (other instanceof LogicalValues) {
|
||||||
|
return visit((LogicalValues) other);
|
||||||
|
}
|
||||||
|
|
||||||
|
return super.visit(other);
|
||||||
|
}
|
||||||
|
|
||||||
|
public PartialDruidQuery getPartialDruidQuery()
|
||||||
|
{
|
||||||
|
return partialDruidQuery;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<PartialDruidQuery> getQueryList()
|
||||||
|
{
|
||||||
|
return queryList;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<DruidTable> getQueryTables()
|
||||||
|
{
|
||||||
|
return queryTables;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DruidTable getCurrentTable()
|
||||||
|
{
|
||||||
|
return currentTable;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -36,6 +36,7 @@ public class PlannerConfig
|
||||||
public static final String CTX_KEY_USE_NATIVE_QUERY_EXPLAIN = "useNativeQueryExplain";
|
public static final String CTX_KEY_USE_NATIVE_QUERY_EXPLAIN = "useNativeQueryExplain";
|
||||||
public static final String CTX_KEY_FORCE_EXPRESSION_VIRTUAL_COLUMNS = "forceExpressionVirtualColumns";
|
public static final String CTX_KEY_FORCE_EXPRESSION_VIRTUAL_COLUMNS = "forceExpressionVirtualColumns";
|
||||||
public static final String CTX_MAX_NUMERIC_IN_FILTERS = "maxNumericInFilters";
|
public static final String CTX_MAX_NUMERIC_IN_FILTERS = "maxNumericInFilters";
|
||||||
|
public static final String CTX_NATIVE_QUERY_SQL_PLANNING_MODE = "plannerStrategy";
|
||||||
public static final int NUM_FILTER_NOT_USED = -1;
|
public static final int NUM_FILTER_NOT_USED = -1;
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -71,6 +72,11 @@ public class PlannerConfig
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
private int maxNumericInFilters = NUM_FILTER_NOT_USED;
|
private int maxNumericInFilters = NUM_FILTER_NOT_USED;
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
private String nativeQuerySqlPlanningMode = NATIVE_QUERY_SQL_PLANNING_MODE_COUPLED; // can be COUPLED or DECOUPLED
|
||||||
|
public static final String NATIVE_QUERY_SQL_PLANNING_MODE_COUPLED = "COUPLED";
|
||||||
|
public static final String NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED = "DECOUPLED";
|
||||||
|
|
||||||
private boolean serializeComplexValues = true;
|
private boolean serializeComplexValues = true;
|
||||||
|
|
||||||
public int getMaxNumericInFilters()
|
public int getMaxNumericInFilters()
|
||||||
|
@ -137,6 +143,11 @@ public class PlannerConfig
|
||||||
return forceExpressionVirtualColumns;
|
return forceExpressionVirtualColumns;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public String getNativeQuerySqlPlanningMode()
|
||||||
|
{
|
||||||
|
return nativeQuerySqlPlanningMode;
|
||||||
|
}
|
||||||
|
|
||||||
public PlannerConfig withOverrides(final Map<String, Object> queryContext)
|
public PlannerConfig withOverrides(final Map<String, Object> queryContext)
|
||||||
{
|
{
|
||||||
if (queryContext.isEmpty()) {
|
if (queryContext.isEmpty()) {
|
||||||
|
@ -168,7 +179,8 @@ public class PlannerConfig
|
||||||
useGroupingSetForExactDistinct == that.useGroupingSetForExactDistinct &&
|
useGroupingSetForExactDistinct == that.useGroupingSetForExactDistinct &&
|
||||||
computeInnerJoinCostAsFilter == that.computeInnerJoinCostAsFilter &&
|
computeInnerJoinCostAsFilter == that.computeInnerJoinCostAsFilter &&
|
||||||
authorizeSystemTablesDirectly == that.authorizeSystemTablesDirectly &&
|
authorizeSystemTablesDirectly == that.authorizeSystemTablesDirectly &&
|
||||||
maxNumericInFilters == that.maxNumericInFilters;
|
maxNumericInFilters == that.maxNumericInFilters &&
|
||||||
|
nativeQuerySqlPlanningMode.equals(that.nativeQuerySqlPlanningMode);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -183,7 +195,8 @@ public class PlannerConfig
|
||||||
sqlTimeZone,
|
sqlTimeZone,
|
||||||
serializeComplexValues,
|
serializeComplexValues,
|
||||||
useNativeQueryExplain,
|
useNativeQueryExplain,
|
||||||
forceExpressionVirtualColumns
|
forceExpressionVirtualColumns,
|
||||||
|
nativeQuerySqlPlanningMode
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -198,6 +211,7 @@ public class PlannerConfig
|
||||||
", sqlTimeZone=" + sqlTimeZone +
|
", sqlTimeZone=" + sqlTimeZone +
|
||||||
", serializeComplexValues=" + serializeComplexValues +
|
", serializeComplexValues=" + serializeComplexValues +
|
||||||
", useNativeQueryExplain=" + useNativeQueryExplain +
|
", useNativeQueryExplain=" + useNativeQueryExplain +
|
||||||
|
", nativeQuerySqlPlanningMode=" + nativeQuerySqlPlanningMode +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -231,6 +245,7 @@ public class PlannerConfig
|
||||||
private boolean forceExpressionVirtualColumns;
|
private boolean forceExpressionVirtualColumns;
|
||||||
private int maxNumericInFilters;
|
private int maxNumericInFilters;
|
||||||
private boolean serializeComplexValues;
|
private boolean serializeComplexValues;
|
||||||
|
private String nativeQuerySqlPlanningMode;
|
||||||
|
|
||||||
public Builder(PlannerConfig base)
|
public Builder(PlannerConfig base)
|
||||||
{
|
{
|
||||||
|
@ -249,6 +264,7 @@ public class PlannerConfig
|
||||||
forceExpressionVirtualColumns = base.isForceExpressionVirtualColumns();
|
forceExpressionVirtualColumns = base.isForceExpressionVirtualColumns();
|
||||||
maxNumericInFilters = base.getMaxNumericInFilters();
|
maxNumericInFilters = base.getMaxNumericInFilters();
|
||||||
serializeComplexValues = base.shouldSerializeComplexValues();
|
serializeComplexValues = base.shouldSerializeComplexValues();
|
||||||
|
nativeQuerySqlPlanningMode = base.getNativeQuerySqlPlanningMode();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder requireTimeCondition(boolean option)
|
public Builder requireTimeCondition(boolean option)
|
||||||
|
@ -317,6 +333,12 @@ public class PlannerConfig
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder nativeQuerySqlPlanningMode(String mode)
|
||||||
|
{
|
||||||
|
this.nativeQuerySqlPlanningMode = mode;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public Builder withOverrides(final Map<String, Object> queryContext)
|
public Builder withOverrides(final Map<String, Object> queryContext)
|
||||||
{
|
{
|
||||||
useApproximateCountDistinct = QueryContexts.parseBoolean(
|
useApproximateCountDistinct = QueryContexts.parseBoolean(
|
||||||
|
@ -357,6 +379,11 @@ public class PlannerConfig
|
||||||
maxNumericInFilters = validateMaxNumericInFilters(
|
maxNumericInFilters = validateMaxNumericInFilters(
|
||||||
queryContextMaxNumericInFilters,
|
queryContextMaxNumericInFilters,
|
||||||
maxNumericInFilters);
|
maxNumericInFilters);
|
||||||
|
nativeQuerySqlPlanningMode = QueryContexts.parseString(
|
||||||
|
queryContext,
|
||||||
|
CTX_NATIVE_QUERY_SQL_PLANNING_MODE,
|
||||||
|
nativeQuerySqlPlanningMode
|
||||||
|
);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -397,6 +424,7 @@ public class PlannerConfig
|
||||||
config.maxNumericInFilters = maxNumericInFilters;
|
config.maxNumericInFilters = maxNumericInFilters;
|
||||||
config.forceExpressionVirtualColumns = forceExpressionVirtualColumns;
|
config.forceExpressionVirtualColumns = forceExpressionVirtualColumns;
|
||||||
config.serializeComplexValues = serializeComplexValues;
|
config.serializeComplexValues = serializeComplexValues;
|
||||||
|
config.nativeQuerySqlPlanningMode = nativeQuerySqlPlanningMode;
|
||||||
return config;
|
return config;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.calcite.config.CalciteConnectionConfig;
|
||||||
import org.apache.calcite.config.CalciteConnectionConfigImpl;
|
import org.apache.calcite.config.CalciteConnectionConfigImpl;
|
||||||
import org.apache.calcite.plan.Context;
|
import org.apache.calcite.plan.Context;
|
||||||
import org.apache.calcite.plan.ConventionTraitDef;
|
import org.apache.calcite.plan.ConventionTraitDef;
|
||||||
|
import org.apache.calcite.plan.volcano.DruidVolcanoCost;
|
||||||
import org.apache.calcite.rel.RelCollationTraitDef;
|
import org.apache.calcite.rel.RelCollationTraitDef;
|
||||||
import org.apache.calcite.sql.parser.SqlParser;
|
import org.apache.calcite.sql.parser.SqlParser;
|
||||||
import org.apache.calcite.sql.validate.SqlConformance;
|
import org.apache.calcite.sql.validate.SqlConformance;
|
||||||
|
@ -145,7 +146,7 @@ public class PlannerFactory extends PlannerToolbox
|
||||||
plannerContext.queryContext().getInSubQueryThreshold()
|
plannerContext.queryContext().getInSubQueryThreshold()
|
||||||
)
|
)
|
||||||
.build();
|
.build();
|
||||||
return Frameworks
|
Frameworks.ConfigBuilder frameworkConfigBuilder = Frameworks
|
||||||
.newConfigBuilder()
|
.newConfigBuilder()
|
||||||
.parserConfig(PARSER_CONFIG)
|
.parserConfig(PARSER_CONFIG)
|
||||||
.traitDefs(ConventionTraitDef.INSTANCE, RelCollationTraitDef.INSTANCE)
|
.traitDefs(ConventionTraitDef.INSTANCE, RelCollationTraitDef.INSTANCE)
|
||||||
|
@ -184,7 +185,15 @@ public class PlannerFactory extends PlannerToolbox
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
})
|
});
|
||||||
.build();
|
|
||||||
|
if (PlannerConfig.NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED
|
||||||
|
.equals(plannerConfig().getNativeQuerySqlPlanningMode())
|
||||||
|
) {
|
||||||
|
frameworkConfigBuilder.costFactory(new DruidVolcanoCost.Factory());
|
||||||
|
}
|
||||||
|
|
||||||
|
return frameworkConfigBuilder.build();
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -56,10 +56,12 @@ import org.apache.calcite.tools.ValidationException;
|
||||||
import org.apache.calcite.util.Pair;
|
import org.apache.calcite.util.Pair;
|
||||||
import org.apache.druid.error.DruidException;
|
import org.apache.druid.error.DruidException;
|
||||||
import org.apache.druid.error.InvalidSqlInput;
|
import org.apache.druid.error.InvalidSqlInput;
|
||||||
|
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.apache.druid.java.util.common.guava.BaseSequence;
|
import org.apache.druid.java.util.common.guava.BaseSequence;
|
||||||
import org.apache.druid.java.util.common.guava.Sequences;
|
import org.apache.druid.java.util.common.guava.Sequences;
|
||||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||||
import org.apache.druid.query.Query;
|
import org.apache.druid.query.Query;
|
||||||
|
import org.apache.druid.query.QueryDataSource;
|
||||||
import org.apache.druid.server.QueryResponse;
|
import org.apache.druid.server.QueryResponse;
|
||||||
import org.apache.druid.server.security.Action;
|
import org.apache.druid.server.security.Action;
|
||||||
import org.apache.druid.server.security.Resource;
|
import org.apache.druid.server.security.Resource;
|
||||||
|
@ -68,6 +70,7 @@ import org.apache.druid.sql.calcite.rel.DruidConvention;
|
||||||
import org.apache.druid.sql.calcite.rel.DruidQuery;
|
import org.apache.druid.sql.calcite.rel.DruidQuery;
|
||||||
import org.apache.druid.sql.calcite.rel.DruidRel;
|
import org.apache.druid.sql.calcite.rel.DruidRel;
|
||||||
import org.apache.druid.sql.calcite.rel.DruidUnionRel;
|
import org.apache.druid.sql.calcite.rel.DruidUnionRel;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention;
|
||||||
import org.apache.druid.sql.calcite.run.EngineFeature;
|
import org.apache.druid.sql.calcite.run.EngineFeature;
|
||||||
import org.apache.druid.sql.calcite.run.QueryMaker;
|
import org.apache.druid.sql.calcite.run.QueryMaker;
|
||||||
import org.apache.druid.sql.calcite.table.DruidTable;
|
import org.apache.druid.sql.calcite.table.DruidTable;
|
||||||
|
@ -531,6 +534,58 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
||||||
);
|
);
|
||||||
QueryValidations.validateLogicalQueryForDruid(handlerContext.plannerContext(), parameterized);
|
QueryValidations.validateLogicalQueryForDruid(handlerContext.plannerContext(), parameterized);
|
||||||
CalcitePlanner planner = handlerContext.planner();
|
CalcitePlanner planner = handlerContext.planner();
|
||||||
|
|
||||||
|
if (plannerContext.getPlannerConfig()
|
||||||
|
.getNativeQuerySqlPlanningMode()
|
||||||
|
.equals(PlannerConfig.NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED)
|
||||||
|
) {
|
||||||
|
RelNode newRoot = parameterized;
|
||||||
|
newRoot = planner.transform(
|
||||||
|
CalciteRulesManager.DRUID_DAG_CONVENTION_RULES,
|
||||||
|
planner.getEmptyTraitSet()
|
||||||
|
.plus(rootQueryRel.collation)
|
||||||
|
.plus(DruidLogicalConvention.instance()),
|
||||||
|
newRoot
|
||||||
|
);
|
||||||
|
DruidQueryGenerator shuttle = new DruidQueryGenerator(plannerContext);
|
||||||
|
newRoot.accept(shuttle);
|
||||||
|
log.info("PartialDruidQuery : " + shuttle.getPartialDruidQuery());
|
||||||
|
shuttle.getQueryList().add(shuttle.getPartialDruidQuery()); // add topmost query to the list
|
||||||
|
shuttle.getQueryTables().add(shuttle.getCurrentTable());
|
||||||
|
assert !shuttle.getQueryList().isEmpty();
|
||||||
|
log.info("query list size " + shuttle.getQueryList().size());
|
||||||
|
log.info("query tables size " + shuttle.getQueryTables().size());
|
||||||
|
// build bottom-most query
|
||||||
|
DruidQuery baseQuery = shuttle.getQueryList().get(0).build(
|
||||||
|
shuttle.getQueryTables().get(0).getDataSource(),
|
||||||
|
shuttle.getQueryTables().get(0).getRowSignature(),
|
||||||
|
plannerContext,
|
||||||
|
rexBuilder,
|
||||||
|
shuttle.getQueryList().size() != 1,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
// build outer queries
|
||||||
|
for (int i = 1; i < shuttle.getQueryList().size(); i++) {
|
||||||
|
baseQuery = shuttle.getQueryList().get(i).build(
|
||||||
|
new QueryDataSource(baseQuery.getQuery()),
|
||||||
|
baseQuery.getOutputRowSignature(),
|
||||||
|
plannerContext,
|
||||||
|
rexBuilder,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
log.info("final query : " +
|
||||||
|
new DefaultObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(baseQuery.getQuery()));
|
||||||
|
}
|
||||||
|
catch (JsonProcessingException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
DruidQuery finalBaseQuery = baseQuery;
|
||||||
|
final Supplier<QueryResponse<Object[]>> resultsSupplier = () -> plannerContext.getQueryMaker().runQuery(finalBaseQuery);
|
||||||
|
|
||||||
|
return new PlannerResult(resultsSupplier, finalBaseQuery.getOutputRowType());
|
||||||
|
} else {
|
||||||
final DruidRel<?> druidRel = (DruidRel<?>) planner.transform(
|
final DruidRel<?> druidRel = (DruidRel<?>) planner.transform(
|
||||||
CalciteRulesManager.DRUID_CONVENTION_RULES,
|
CalciteRulesManager.DRUID_CONVENTION_RULES,
|
||||||
planner.getEmptyTraitSet()
|
planner.getEmptyTraitSet()
|
||||||
|
@ -539,7 +594,6 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
||||||
parameterized
|
parameterized
|
||||||
);
|
);
|
||||||
handlerContext.hook().captureDruidRel(druidRel);
|
handlerContext.hook().captureDruidRel(druidRel);
|
||||||
|
|
||||||
if (explain != null) {
|
if (explain != null) {
|
||||||
return planExplanation(possiblyLimitedRoot, druidRel, true);
|
return planExplanation(possiblyLimitedRoot, druidRel, true);
|
||||||
} else {
|
} else {
|
||||||
|
@ -569,6 +623,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
||||||
return new PlannerResult(resultsSupplier, rowType);
|
return new PlannerResult(resultsSupplier, rowType);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method wraps the root with a {@link LogicalSort} that applies a limit (no ordering change). If the outer rel
|
* This method wraps the root with a {@link LogicalSort} that applies a limit (no ordering change). If the outer rel
|
||||||
|
|
|
@ -32,61 +32,61 @@ public class CostEstimates
|
||||||
* Per-row base cost. This represents the cost of walking through every row, but not actually reading anything
|
* Per-row base cost. This represents the cost of walking through every row, but not actually reading anything
|
||||||
* from them or computing any aggregations.
|
* from them or computing any aggregations.
|
||||||
*/
|
*/
|
||||||
static final double COST_BASE = 1;
|
public static final double COST_BASE = 1;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cost to include a column in query output.
|
* Cost to include a column in query output.
|
||||||
*/
|
*/
|
||||||
static final double COST_OUTPUT_COLUMN = 0.05;
|
public static final double COST_OUTPUT_COLUMN = 0.05;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cost to compute and read an expression.
|
* Cost to compute and read an expression.
|
||||||
*/
|
*/
|
||||||
static final double COST_EXPRESSION = 0.25;
|
public static final double COST_EXPRESSION = 0.25;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cost to compute an aggregation.
|
* Cost to compute an aggregation.
|
||||||
*/
|
*/
|
||||||
static final double COST_AGGREGATION = 0.05;
|
public static final double COST_AGGREGATION = 0.05;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cost per GROUP BY dimension.
|
* Cost per GROUP BY dimension.
|
||||||
*/
|
*/
|
||||||
static final double COST_DIMENSION = 0.25;
|
public static final double COST_DIMENSION = 0.25;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Multiplier to apply when there is a WHERE filter. Encourages pushing down filters and limits through joins and
|
* Multiplier to apply when there is a WHERE filter. Encourages pushing down filters and limits through joins and
|
||||||
* subqueries when possible.
|
* subqueries when possible.
|
||||||
*/
|
*/
|
||||||
static final double MULTIPLIER_FILTER = 0.1;
|
public static final double MULTIPLIER_FILTER = 0.1;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Multiplier to apply when there is an ORDER BY. Encourages avoiding them when possible.
|
* Multiplier to apply when there is an ORDER BY. Encourages avoiding them when possible.
|
||||||
*/
|
*/
|
||||||
static final double MULTIPLIER_ORDER_BY = 10;
|
public static final double MULTIPLIER_ORDER_BY = 10;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Multiplier to apply when there is a LIMIT. Encourages pushing down limits when possible.
|
* Multiplier to apply when there is a LIMIT. Encourages pushing down limits when possible.
|
||||||
*/
|
*/
|
||||||
static final double MULTIPLIER_LIMIT = 0.5;
|
public static final double MULTIPLIER_LIMIT = 0.5;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Multiplier to apply to an outer query via {@link DruidOuterQueryRel}. Encourages pushing down time-saving
|
* Multiplier to apply to an outer query via {@link DruidOuterQueryRel}. Encourages pushing down time-saving
|
||||||
* operations to the lowest level of the query stack, because they'll have bigger impact there.
|
* operations to the lowest level of the query stack, because they'll have bigger impact there.
|
||||||
*/
|
*/
|
||||||
static final double MULTIPLIER_OUTER_QUERY = .1;
|
public static final double MULTIPLIER_OUTER_QUERY = .1;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cost to add to a subquery. Strongly encourages avoiding subqueries, since they must be inlined and then the join
|
* Cost to add to a subquery. Strongly encourages avoiding subqueries, since they must be inlined and then the join
|
||||||
* must run on the Broker.
|
* must run on the Broker.
|
||||||
*/
|
*/
|
||||||
static final double COST_SUBQUERY = 1e5;
|
public static final double COST_SUBQUERY = 1e5;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cost to perform a cross join. Strongly encourages pushing down filters into join conditions, even if it means
|
* Cost to perform a cross join. Strongly encourages pushing down filters into join conditions, even if it means
|
||||||
* we need to add a subquery (this is higher than {@link #COST_SUBQUERY}).
|
* we need to add a subquery (this is higher than {@link #COST_SUBQUERY}).
|
||||||
*/
|
*/
|
||||||
static final double COST_JOIN_CROSS = 1e8;
|
public static final double COST_JOIN_CROSS = 1e8;
|
||||||
|
|
||||||
private CostEstimates()
|
private CostEstimates()
|
||||||
{
|
{
|
||||||
|
|
|
@ -260,6 +260,30 @@ public class PartialDruidQuery
|
||||||
if (selectProject == null) {
|
if (selectProject == null) {
|
||||||
theProject = newSelectProject;
|
theProject = newSelectProject;
|
||||||
} else {
|
} else {
|
||||||
|
return mergeProject(newSelectProject);
|
||||||
|
}
|
||||||
|
|
||||||
|
return new PartialDruidQuery(
|
||||||
|
builderSupplier,
|
||||||
|
scan,
|
||||||
|
whereFilter,
|
||||||
|
theProject,
|
||||||
|
aggregate,
|
||||||
|
aggregateProject,
|
||||||
|
havingFilter,
|
||||||
|
sort,
|
||||||
|
sortProject,
|
||||||
|
window,
|
||||||
|
windowProject
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
public PartialDruidQuery mergeProject(Project newSelectProject)
|
||||||
|
{
|
||||||
|
if (stage() != Stage.SELECT_PROJECT) {
|
||||||
|
throw new ISE("Expected partial query state to be [%s], but found [%s]", Stage.SELECT_PROJECT, stage());
|
||||||
|
}
|
||||||
|
Project theProject;
|
||||||
final List<RexNode> newProjectRexNodes = RelOptUtil.pushPastProject(
|
final List<RexNode> newProjectRexNodes = RelOptUtil.pushPastProject(
|
||||||
newSelectProject.getProjects(),
|
newSelectProject.getProjects(),
|
||||||
selectProject
|
selectProject
|
||||||
|
@ -278,8 +302,6 @@ public class PartialDruidQuery
|
||||||
);
|
);
|
||||||
theProject = (Project) relBuilder.build();
|
theProject = (Project) relBuilder.build();
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
return new PartialDruidQuery(
|
return new PartialDruidQuery(
|
||||||
builderSupplier,
|
builderSupplier,
|
||||||
scan,
|
scan,
|
||||||
|
|
|
@ -0,0 +1,101 @@
|
||||||
|
/*
|
||||||
|
* 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.rel.logical;
|
||||||
|
|
||||||
|
import org.apache.calcite.plan.RelOptCluster;
|
||||||
|
import org.apache.calcite.plan.RelOptCost;
|
||||||
|
import org.apache.calcite.plan.RelOptPlanner;
|
||||||
|
import org.apache.calcite.plan.RelTraitSet;
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
import org.apache.calcite.rel.RelWriter;
|
||||||
|
import org.apache.calcite.rel.core.Aggregate;
|
||||||
|
import org.apache.calcite.rel.core.AggregateCall;
|
||||||
|
import org.apache.calcite.rel.metadata.RelMetadataQuery;
|
||||||
|
import org.apache.calcite.util.ImmutableBitSet;
|
||||||
|
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||||
|
import org.apache.druid.sql.calcite.rel.CostEstimates;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link DruidLogicalNode} convention node for {@link Aggregate} plan node.
|
||||||
|
*/
|
||||||
|
public class DruidAggregate extends Aggregate implements DruidLogicalNode
|
||||||
|
{
|
||||||
|
private final PlannerContext plannerContext;
|
||||||
|
|
||||||
|
public DruidAggregate(
|
||||||
|
RelOptCluster cluster,
|
||||||
|
RelTraitSet traitSet,
|
||||||
|
RelNode input,
|
||||||
|
ImmutableBitSet groupSet,
|
||||||
|
List<ImmutableBitSet> groupSets,
|
||||||
|
List<AggregateCall> aggCalls,
|
||||||
|
PlannerContext plannerContext
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(cluster, traitSet, input, groupSet, groupSets, aggCalls);
|
||||||
|
assert getConvention() instanceof DruidLogicalConvention;
|
||||||
|
this.plannerContext = plannerContext;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq)
|
||||||
|
{
|
||||||
|
double rowCount = mq.getRowCount(this);
|
||||||
|
double cost = CostEstimates.COST_DIMENSION * getGroupSet().size();
|
||||||
|
for (AggregateCall aggregateCall : getAggCallList()) {
|
||||||
|
if (aggregateCall.hasFilter()) {
|
||||||
|
cost += CostEstimates.COST_AGGREGATION * CostEstimates.MULTIPLIER_FILTER;
|
||||||
|
} else {
|
||||||
|
cost += CostEstimates.COST_AGGREGATION;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (!plannerContext.getPlannerConfig().isUseApproximateCountDistinct() &&
|
||||||
|
getAggCallList().stream().anyMatch(AggregateCall::isDistinct)) {
|
||||||
|
return planner.getCostFactory().makeInfiniteCost();
|
||||||
|
}
|
||||||
|
return planner.getCostFactory().makeCost(rowCount, cost, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final Aggregate copy(
|
||||||
|
RelTraitSet traitSet,
|
||||||
|
RelNode input,
|
||||||
|
ImmutableBitSet groupSet,
|
||||||
|
List<ImmutableBitSet> groupSets,
|
||||||
|
List<AggregateCall> aggCalls
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return new DruidAggregate(getCluster(), traitSet, input, groupSet, groupSets, aggCalls, plannerContext);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelWriter explainTerms(RelWriter pw)
|
||||||
|
{
|
||||||
|
return super.explainTerms(pw).item("druid", "logical");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double estimateRowCount(RelMetadataQuery mq)
|
||||||
|
{
|
||||||
|
return mq.getRowCount(this);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,54 @@
|
||||||
|
/*
|
||||||
|
* 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.rel.logical;
|
||||||
|
|
||||||
|
import org.apache.calcite.plan.RelOptCluster;
|
||||||
|
import org.apache.calcite.plan.RelOptCost;
|
||||||
|
import org.apache.calcite.plan.RelOptPlanner;
|
||||||
|
import org.apache.calcite.plan.RelTraitSet;
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
import org.apache.calcite.rel.core.Filter;
|
||||||
|
import org.apache.calcite.rel.metadata.RelMetadataQuery;
|
||||||
|
import org.apache.calcite.rex.RexNode;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link DruidLogicalNode} convention node for {@link Filter} plan node.
|
||||||
|
*/
|
||||||
|
public class DruidFilter extends Filter implements DruidLogicalNode
|
||||||
|
{
|
||||||
|
|
||||||
|
public DruidFilter(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition)
|
||||||
|
{
|
||||||
|
super(cluster, traits, child, condition);
|
||||||
|
assert getConvention() instanceof DruidLogicalConvention;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition)
|
||||||
|
{
|
||||||
|
return new DruidFilter(getCluster(), getTraitSet(), input, condition);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq)
|
||||||
|
{
|
||||||
|
return planner.getCostFactory().makeCost(mq.getRowCount(this), 0, 0);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,93 @@
|
||||||
|
/*
|
||||||
|
* 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.rel.logical;
|
||||||
|
|
||||||
|
import org.apache.calcite.plan.Convention;
|
||||||
|
import org.apache.calcite.plan.ConventionTraitDef;
|
||||||
|
import org.apache.calcite.plan.RelOptPlanner;
|
||||||
|
import org.apache.calcite.plan.RelTrait;
|
||||||
|
import org.apache.calcite.plan.RelTraitDef;
|
||||||
|
import org.apache.calcite.plan.RelTraitSet;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A Calcite convention to produce {@link DruidLogicalNode} based DAG.
|
||||||
|
*/
|
||||||
|
public class DruidLogicalConvention implements Convention
|
||||||
|
{
|
||||||
|
|
||||||
|
private static final DruidLogicalConvention INSTANCE = new DruidLogicalConvention();
|
||||||
|
private static final String NAME = "DRUID_LOGICAL";
|
||||||
|
|
||||||
|
private DruidLogicalConvention()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DruidLogicalConvention instance()
|
||||||
|
{
|
||||||
|
return INSTANCE;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Class getInterface()
|
||||||
|
{
|
||||||
|
return DruidLogicalNode.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getName()
|
||||||
|
{
|
||||||
|
return NAME;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean canConvertConvention(Convention toConvention)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelTraitDef getTraitDef()
|
||||||
|
{
|
||||||
|
return ConventionTraitDef.INSTANCE;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean satisfies(RelTrait trait)
|
||||||
|
{
|
||||||
|
return trait.equals(this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void register(RelOptPlanner planner)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return NAME;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,30 @@
|
||||||
|
/*
|
||||||
|
* 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.rel.logical;
|
||||||
|
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An interface to mark {@link RelNode} as Druid physical nodes. These physical nodes look a lot same as their logical
|
||||||
|
* counterparts in Calcite, but they do follow a different costing model.
|
||||||
|
*/
|
||||||
|
public interface DruidLogicalNode extends RelNode
|
||||||
|
{
|
||||||
|
}
|
|
@ -0,0 +1,101 @@
|
||||||
|
/*
|
||||||
|
* 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.rel.logical;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
import org.apache.calcite.plan.RelOptCluster;
|
||||||
|
import org.apache.calcite.plan.RelOptCost;
|
||||||
|
import org.apache.calcite.plan.RelOptPlanner;
|
||||||
|
import org.apache.calcite.plan.RelTraitSet;
|
||||||
|
import org.apache.calcite.rel.RelCollationTraitDef;
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
import org.apache.calcite.rel.RelWriter;
|
||||||
|
import org.apache.calcite.rel.core.Project;
|
||||||
|
import org.apache.calcite.rel.metadata.RelMdCollation;
|
||||||
|
import org.apache.calcite.rel.metadata.RelMetadataQuery;
|
||||||
|
import org.apache.calcite.rel.type.RelDataType;
|
||||||
|
import org.apache.calcite.rex.RexNode;
|
||||||
|
import org.apache.calcite.sql.SqlKind;
|
||||||
|
import org.apache.calcite.sql.type.SqlTypeName;
|
||||||
|
import org.apache.druid.sql.calcite.rel.CostEstimates;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link DruidLogicalNode} convention node for {@link Project} plan node.
|
||||||
|
*/
|
||||||
|
public class DruidProject extends Project implements DruidLogicalNode
|
||||||
|
{
|
||||||
|
public DruidProject(
|
||||||
|
RelOptCluster cluster,
|
||||||
|
RelTraitSet traitSet,
|
||||||
|
RelNode input,
|
||||||
|
List<? extends RexNode> projects,
|
||||||
|
RelDataType rowType
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(cluster, traitSet, input, projects, rowType);
|
||||||
|
assert getConvention() instanceof DruidLogicalConvention;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq)
|
||||||
|
{
|
||||||
|
double cost = 0;
|
||||||
|
double rowCount = mq.getRowCount(getInput());
|
||||||
|
for (final RexNode rexNode : getProjects()) {
|
||||||
|
if (rexNode.isA(SqlKind.INPUT_REF)) {
|
||||||
|
cost += 0;
|
||||||
|
}
|
||||||
|
if (rexNode.getType().getSqlTypeName() == SqlTypeName.BOOLEAN || rexNode.isA(SqlKind.CAST)) {
|
||||||
|
cost += 0;
|
||||||
|
} else if (!rexNode.isA(ImmutableSet.of(SqlKind.INPUT_REF, SqlKind.LITERAL))) {
|
||||||
|
cost += CostEstimates.COST_EXPRESSION;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// adding atleast 1e-6 cost since zero cost is converted to a tiny cost by the planner which is (1 row, 1 cpu, 0 io)
|
||||||
|
// that becomes a significant cost in some cases.
|
||||||
|
return planner.getCostFactory().makeCost(0, Math.max(cost * rowCount, 1e-6), 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects, RelDataType rowType)
|
||||||
|
{
|
||||||
|
return new DruidProject(getCluster(), traitSet, input, exps, rowType);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelWriter explainTerms(RelWriter pw)
|
||||||
|
{
|
||||||
|
return super.explainTerms(pw).item("druid", "logical");
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DruidProject create(final RelNode input, final List<? extends RexNode> projects, RelDataType rowType)
|
||||||
|
{
|
||||||
|
final RelOptCluster cluster = input.getCluster();
|
||||||
|
final RelMetadataQuery mq = cluster.getMetadataQuery();
|
||||||
|
final RelTraitSet traitSet =
|
||||||
|
input.getTraitSet().replaceIfs(
|
||||||
|
RelCollationTraitDef.INSTANCE,
|
||||||
|
() -> RelMdCollation.project(mq, input, projects)
|
||||||
|
);
|
||||||
|
return new DruidProject(cluster, traitSet, input, projects, rowType);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,97 @@
|
||||||
|
/*
|
||||||
|
* 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.rel.logical;
|
||||||
|
|
||||||
|
import org.apache.calcite.plan.RelOptCluster;
|
||||||
|
import org.apache.calcite.plan.RelOptCost;
|
||||||
|
import org.apache.calcite.plan.RelOptPlanner;
|
||||||
|
import org.apache.calcite.plan.RelTraitSet;
|
||||||
|
import org.apache.calcite.rel.RelCollation;
|
||||||
|
import org.apache.calcite.rel.RelCollationTraitDef;
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
import org.apache.calcite.rel.RelWriter;
|
||||||
|
import org.apache.calcite.rel.core.Sort;
|
||||||
|
import org.apache.calcite.rel.metadata.RelMetadataQuery;
|
||||||
|
import org.apache.calcite.rex.RexNode;
|
||||||
|
import org.apache.druid.sql.calcite.planner.OffsetLimit;
|
||||||
|
import org.apache.druid.sql.calcite.rel.CostEstimates;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link DruidLogicalNode} convention node for {@link Sort} plan node.
|
||||||
|
*/
|
||||||
|
public class DruidSort extends Sort implements DruidLogicalNode
|
||||||
|
{
|
||||||
|
private DruidSort(
|
||||||
|
RelOptCluster cluster,
|
||||||
|
RelTraitSet traits,
|
||||||
|
RelNode input,
|
||||||
|
RelCollation collation,
|
||||||
|
RexNode offset,
|
||||||
|
RexNode fetch
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(cluster, traits, input, collation, offset, fetch);
|
||||||
|
assert getConvention() instanceof DruidLogicalConvention;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DruidSort create(RelNode input, RelCollation collation, RexNode offset, RexNode fetch)
|
||||||
|
{
|
||||||
|
RelOptCluster cluster = input.getCluster();
|
||||||
|
collation = RelCollationTraitDef.INSTANCE.canonize(collation);
|
||||||
|
RelTraitSet traitSet =
|
||||||
|
input.getTraitSet().replace(DruidLogicalConvention.instance()).replace(collation);
|
||||||
|
return new DruidSort(cluster, traitSet, input, collation, offset, fetch);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Sort copy(
|
||||||
|
RelTraitSet traitSet,
|
||||||
|
RelNode newInput,
|
||||||
|
RelCollation newCollation,
|
||||||
|
RexNode offset,
|
||||||
|
RexNode fetch
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return new DruidSort(getCluster(), traitSet, newInput, newCollation, offset, fetch);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq)
|
||||||
|
{
|
||||||
|
double cost = 0;
|
||||||
|
double rowCount = mq.getRowCount(this);
|
||||||
|
|
||||||
|
if (fetch != null) {
|
||||||
|
OffsetLimit offsetLimit = OffsetLimit.fromSort(this);
|
||||||
|
rowCount = Math.min(rowCount, offsetLimit.getLimit() - offsetLimit.getOffset());
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!getCollation().getFieldCollations().isEmpty() && fetch == null) {
|
||||||
|
cost = rowCount * CostEstimates.MULTIPLIER_ORDER_BY;
|
||||||
|
}
|
||||||
|
return planner.getCostFactory().makeCost(rowCount, cost, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelWriter explainTerms(RelWriter pw)
|
||||||
|
{
|
||||||
|
return super.explainTerms(pw).item("druid", "logical");
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,111 @@
|
||||||
|
/*
|
||||||
|
* 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.rel.logical;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import org.apache.calcite.plan.RelOptCluster;
|
||||||
|
import org.apache.calcite.plan.RelOptCost;
|
||||||
|
import org.apache.calcite.plan.RelOptPlanner;
|
||||||
|
import org.apache.calcite.plan.RelOptTable;
|
||||||
|
import org.apache.calcite.plan.RelTraitSet;
|
||||||
|
import org.apache.calcite.rel.RelCollationTraitDef;
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
import org.apache.calcite.rel.RelWriter;
|
||||||
|
import org.apache.calcite.rel.core.Project;
|
||||||
|
import org.apache.calcite.rel.core.TableScan;
|
||||||
|
import org.apache.calcite.rel.metadata.RelMetadataQuery;
|
||||||
|
import org.apache.calcite.rel.type.RelDataType;
|
||||||
|
import org.apache.calcite.schema.Table;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link DruidLogicalNode} convention node for {@link TableScan} plan node.
|
||||||
|
*/
|
||||||
|
public class DruidTableScan extends TableScan implements DruidLogicalNode
|
||||||
|
{
|
||||||
|
private final Project project;
|
||||||
|
|
||||||
|
public DruidTableScan(
|
||||||
|
RelOptCluster cluster,
|
||||||
|
RelTraitSet traitSet,
|
||||||
|
RelOptTable table,
|
||||||
|
Project project
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(cluster, traitSet, table);
|
||||||
|
this.project = project;
|
||||||
|
assert getConvention() instanceof DruidLogicalConvention;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs)
|
||||||
|
{
|
||||||
|
return new DruidTableScan(getCluster(), traitSet, table, project);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq)
|
||||||
|
{
|
||||||
|
return planner.getCostFactory().makeTinyCost();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double estimateRowCount(RelMetadataQuery mq)
|
||||||
|
{
|
||||||
|
return 1_000;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelWriter explainTerms(RelWriter pw)
|
||||||
|
{
|
||||||
|
if (project != null) {
|
||||||
|
project.explainTerms(pw);
|
||||||
|
}
|
||||||
|
return super.explainTerms(pw).item("druid", "logical");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelDataType deriveRowType()
|
||||||
|
{
|
||||||
|
if (project != null) {
|
||||||
|
return project.getRowType();
|
||||||
|
}
|
||||||
|
return super.deriveRowType();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Project getProject()
|
||||||
|
{
|
||||||
|
return project;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DruidTableScan create(RelOptCluster cluster, final RelOptTable relOptTable)
|
||||||
|
{
|
||||||
|
final Table table = relOptTable.unwrap(Table.class);
|
||||||
|
final RelTraitSet traitSet =
|
||||||
|
cluster.traitSet().replaceIfs(RelCollationTraitDef.INSTANCE, () -> {
|
||||||
|
if (table != null) {
|
||||||
|
return table.getStatistic().getCollations();
|
||||||
|
}
|
||||||
|
return ImmutableList.of();
|
||||||
|
});
|
||||||
|
return new DruidTableScan(cluster, traitSet, relOptTable, null);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,64 @@
|
||||||
|
/*
|
||||||
|
* 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.rel.logical;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import org.apache.calcite.plan.RelOptCluster;
|
||||||
|
import org.apache.calcite.plan.RelOptCost;
|
||||||
|
import org.apache.calcite.plan.RelOptPlanner;
|
||||||
|
import org.apache.calcite.plan.RelTraitSet;
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalValues;
|
||||||
|
import org.apache.calcite.rel.metadata.RelMetadataQuery;
|
||||||
|
import org.apache.calcite.rel.type.RelDataType;
|
||||||
|
import org.apache.calcite.rex.RexLiteral;
|
||||||
|
import org.apache.druid.sql.calcite.rel.CostEstimates;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link DruidLogicalNode} convention node for {@link LogicalValues} plan node.
|
||||||
|
*/
|
||||||
|
public class DruidValues extends LogicalValues implements DruidLogicalNode
|
||||||
|
{
|
||||||
|
|
||||||
|
public DruidValues(
|
||||||
|
RelOptCluster cluster,
|
||||||
|
RelTraitSet traitSet,
|
||||||
|
RelDataType rowType,
|
||||||
|
ImmutableList<ImmutableList<RexLiteral>> tuples
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(cluster, traitSet, rowType, tuples);
|
||||||
|
assert getConvention() instanceof DruidLogicalConvention;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs)
|
||||||
|
{
|
||||||
|
return new DruidValues(getCluster(), traitSet, getRowType(), tuples);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq)
|
||||||
|
{
|
||||||
|
return planner.getCostFactory().makeCost(CostEstimates.COST_BASE, 0, 0);
|
||||||
|
}
|
||||||
|
}
|
|
@ -92,7 +92,7 @@ public class DruidLogicalValuesRule extends RelOptRule
|
||||||
*/
|
*/
|
||||||
@Nullable
|
@Nullable
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
static Object getValueFromLiteral(RexLiteral literal, PlannerContext plannerContext)
|
public static Object getValueFromLiteral(RexLiteral literal, PlannerContext plannerContext)
|
||||||
{
|
{
|
||||||
switch (literal.getType().getSqlTypeName()) {
|
switch (literal.getType().getSqlTypeName()) {
|
||||||
case CHAR:
|
case CHAR:
|
||||||
|
|
|
@ -0,0 +1,339 @@
|
||||||
|
/*
|
||||||
|
* 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.logical;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import org.apache.calcite.plan.RelOptCluster;
|
||||||
|
import org.apache.calcite.plan.RelOptRule;
|
||||||
|
import org.apache.calcite.plan.RelOptRuleCall;
|
||||||
|
import org.apache.calcite.rel.RelCollations;
|
||||||
|
import org.apache.calcite.rel.core.Aggregate;
|
||||||
|
import org.apache.calcite.rel.core.AggregateCall;
|
||||||
|
import org.apache.calcite.rel.core.Project;
|
||||||
|
import org.apache.calcite.rel.core.RelFactories;
|
||||||
|
import org.apache.calcite.rel.type.RelDataType;
|
||||||
|
import org.apache.calcite.rel.type.RelDataTypeFactory;
|
||||||
|
import org.apache.calcite.rex.RexBuilder;
|
||||||
|
import org.apache.calcite.rex.RexCall;
|
||||||
|
import org.apache.calcite.rex.RexLiteral;
|
||||||
|
import org.apache.calcite.rex.RexNode;
|
||||||
|
import org.apache.calcite.sql.SqlKind;
|
||||||
|
import org.apache.calcite.sql.SqlPostfixOperator;
|
||||||
|
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
|
||||||
|
import org.apache.calcite.sql.type.SqlTypeName;
|
||||||
|
import org.apache.calcite.tools.RelBuilder;
|
||||||
|
import org.apache.calcite.tools.RelBuilderFactory;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A copy of {@link org.apache.calcite.rel.rules.AggregateCaseToFilterRule} except that it fixes a bug to eliminate
|
||||||
|
* left-over projects for converted aggregates to filter-aggregates. The elimination of left-over projects is necessary
|
||||||
|
* with the new planning since it determines the cost of the plan and hence determines which plan is going to get picked
|
||||||
|
* as the cheapest one.
|
||||||
|
* This fix will also be contributed upstream to Calcite project, and we can remove this rule once the fix is a part of
|
||||||
|
* the Calcite version we use.
|
||||||
|
*/
|
||||||
|
public class DruidAggregateCaseToFilterRule extends RelOptRule
|
||||||
|
{
|
||||||
|
public static final DruidAggregateCaseToFilterRule INSTANCE =
|
||||||
|
new DruidAggregateCaseToFilterRule(RelFactories.LOGICAL_BUILDER, null);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates an AggregateCaseToFilterRule.
|
||||||
|
*/
|
||||||
|
protected DruidAggregateCaseToFilterRule(
|
||||||
|
RelBuilderFactory relBuilderFactory,
|
||||||
|
String description
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(operand(Aggregate.class, operand(Project.class, any())),
|
||||||
|
relBuilderFactory, description
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean matches(final RelOptRuleCall call)
|
||||||
|
{
|
||||||
|
final Aggregate aggregate = call.rel(0);
|
||||||
|
final Project project = call.rel(1);
|
||||||
|
|
||||||
|
for (AggregateCall aggregateCall : aggregate.getAggCallList()) {
|
||||||
|
final int singleArg = soleArgument(aggregateCall);
|
||||||
|
if (singleArg >= 0
|
||||||
|
&& isThreeArgCase(project.getProjects().get(singleArg))) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onMatch(RelOptRuleCall call)
|
||||||
|
{
|
||||||
|
final Aggregate aggregate = call.rel(0);
|
||||||
|
final Project project = call.rel(1);
|
||||||
|
final List<AggregateCall> newCalls =
|
||||||
|
new ArrayList<>(aggregate.getAggCallList().size());
|
||||||
|
List<RexNode> newProjects;
|
||||||
|
|
||||||
|
// TODO : fix grouping columns
|
||||||
|
Set<Integer> groupUsedFields = new HashSet<>();
|
||||||
|
for (int fieldNumber : aggregate.getGroupSet()) {
|
||||||
|
groupUsedFields.add(fieldNumber);
|
||||||
|
}
|
||||||
|
|
||||||
|
List<RexNode> updatedProjects = new ArrayList<>();
|
||||||
|
for (int i = 0; i < project.getProjects().size(); i++) {
|
||||||
|
if (groupUsedFields.contains(i)) {
|
||||||
|
updatedProjects.add(project.getProjects().get(i));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
newProjects = updatedProjects;
|
||||||
|
|
||||||
|
for (AggregateCall aggregateCall : aggregate.getAggCallList()) {
|
||||||
|
AggregateCall newCall =
|
||||||
|
transform(aggregateCall, project, newProjects);
|
||||||
|
|
||||||
|
// Possibly CAST the new aggregator to an appropriate type.
|
||||||
|
newCalls.add(newCall);
|
||||||
|
}
|
||||||
|
final RelBuilder relBuilder = call.builder()
|
||||||
|
.push(project.getInput())
|
||||||
|
.project(newProjects);
|
||||||
|
|
||||||
|
final RelBuilder.GroupKey groupKey =
|
||||||
|
relBuilder.groupKey(
|
||||||
|
aggregate.getGroupSet(),
|
||||||
|
aggregate.getGroupSets()
|
||||||
|
);
|
||||||
|
|
||||||
|
relBuilder.aggregate(groupKey, newCalls)
|
||||||
|
.convert(aggregate.getRowType(), false);
|
||||||
|
|
||||||
|
call.transformTo(relBuilder.build());
|
||||||
|
call.getPlanner().setImportance(aggregate, 0.0);
|
||||||
|
}
|
||||||
|
|
||||||
|
private AggregateCall transform(AggregateCall aggregateCall, Project project, List<RexNode> newProjects)
|
||||||
|
{
|
||||||
|
final int singleArg = soleArgument(aggregateCall);
|
||||||
|
if (singleArg < 0) {
|
||||||
|
Set<Integer> newFields = new HashSet<>();
|
||||||
|
for (int fieldNumber : aggregateCall.getArgList()) {
|
||||||
|
newProjects.add(project.getProjects().get(fieldNumber));
|
||||||
|
newFields.add(newProjects.size() - 1);
|
||||||
|
}
|
||||||
|
int newFilterArg = -1;
|
||||||
|
if (aggregateCall.hasFilter()) {
|
||||||
|
newProjects.add(project.getProjects().get(aggregateCall.filterArg));
|
||||||
|
newFilterArg = newProjects.size() - 1;
|
||||||
|
}
|
||||||
|
return AggregateCall.create(aggregateCall.getAggregation(),
|
||||||
|
aggregateCall.isDistinct(),
|
||||||
|
aggregateCall.isApproximate(),
|
||||||
|
aggregateCall.ignoreNulls(),
|
||||||
|
new ArrayList<>(newFields),
|
||||||
|
newFilterArg,
|
||||||
|
aggregateCall.getCollation(),
|
||||||
|
aggregateCall.getType(),
|
||||||
|
aggregateCall.getName()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
final RexNode rexNode = project.getProjects().get(singleArg);
|
||||||
|
if (!isThreeArgCase(rexNode)) {
|
||||||
|
newProjects.add(rexNode);
|
||||||
|
int callArg = newProjects.size() - 1;
|
||||||
|
int newFilterArg = -1;
|
||||||
|
if (aggregateCall.hasFilter()) {
|
||||||
|
newProjects.add(project.getProjects().get(aggregateCall.filterArg));
|
||||||
|
newFilterArg = newProjects.size() - 1;
|
||||||
|
}
|
||||||
|
return AggregateCall.create(aggregateCall.getAggregation(),
|
||||||
|
aggregateCall.isDistinct(),
|
||||||
|
aggregateCall.isApproximate(),
|
||||||
|
aggregateCall.ignoreNulls(),
|
||||||
|
ImmutableList.of(callArg),
|
||||||
|
newFilterArg,
|
||||||
|
aggregateCall.getCollation(),
|
||||||
|
aggregateCall.getType(),
|
||||||
|
aggregateCall.getName()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
final RelOptCluster cluster = project.getCluster();
|
||||||
|
final RexBuilder rexBuilder = cluster.getRexBuilder();
|
||||||
|
final RexCall caseCall = (RexCall) rexNode;
|
||||||
|
|
||||||
|
// If one arg is null and the other is not, reverse them and set "flip",
|
||||||
|
// which negates the filter.
|
||||||
|
final boolean flip = RexLiteral.isNullLiteral(caseCall.operands.get(1))
|
||||||
|
&& !RexLiteral.isNullLiteral(caseCall.operands.get(2));
|
||||||
|
final RexNode arg1 = caseCall.operands.get(flip ? 2 : 1);
|
||||||
|
final RexNode arg2 = caseCall.operands.get(flip ? 1 : 2);
|
||||||
|
|
||||||
|
// Operand 1: Filter
|
||||||
|
final SqlPostfixOperator op =
|
||||||
|
flip ? SqlStdOperatorTable.IS_FALSE : SqlStdOperatorTable.IS_TRUE;
|
||||||
|
final RexNode filterFromCase =
|
||||||
|
rexBuilder.makeCall(op, caseCall.operands.get(0));
|
||||||
|
|
||||||
|
// Combine the CASE filter with an honest-to-goodness SQL FILTER, if the
|
||||||
|
// latter is present.
|
||||||
|
final RexNode filter;
|
||||||
|
if (aggregateCall.filterArg >= 0) {
|
||||||
|
filter = rexBuilder.makeCall(SqlStdOperatorTable.AND,
|
||||||
|
project.getProjects().get(aggregateCall.filterArg), filterFromCase
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
filter = filterFromCase;
|
||||||
|
}
|
||||||
|
|
||||||
|
final SqlKind kind = aggregateCall.getAggregation().getKind();
|
||||||
|
if (aggregateCall.isDistinct()) {
|
||||||
|
// Just one style supported:
|
||||||
|
// COUNT(DISTINCT CASE WHEN x = 'foo' THEN y END)
|
||||||
|
// =>
|
||||||
|
// COUNT(DISTINCT y) FILTER(WHERE x = 'foo')
|
||||||
|
|
||||||
|
if (kind == SqlKind.COUNT
|
||||||
|
&& RexLiteral.isNullLiteral(arg2)) {
|
||||||
|
newProjects.add(arg1);
|
||||||
|
newProjects.add(filter);
|
||||||
|
return AggregateCall.create(SqlStdOperatorTable.COUNT, true, false,
|
||||||
|
false, ImmutableList.of(newProjects.size() - 2),
|
||||||
|
newProjects.size() - 1, RelCollations.EMPTY,
|
||||||
|
aggregateCall.getType(), aggregateCall.getName()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
newProjects.add(rexNode);
|
||||||
|
int callArg = newProjects.size() - 1;
|
||||||
|
int newFilterArg = -1;
|
||||||
|
if (aggregateCall.hasFilter()) {
|
||||||
|
newProjects.add(project.getProjects().get(aggregateCall.filterArg));
|
||||||
|
newFilterArg = newProjects.size() - 1;
|
||||||
|
}
|
||||||
|
return AggregateCall.create(aggregateCall.getAggregation(),
|
||||||
|
aggregateCall.isDistinct(),
|
||||||
|
aggregateCall.isApproximate(),
|
||||||
|
aggregateCall.ignoreNulls(),
|
||||||
|
ImmutableList.of(callArg),
|
||||||
|
newFilterArg,
|
||||||
|
aggregateCall.getCollation(),
|
||||||
|
aggregateCall.getType(),
|
||||||
|
aggregateCall.getName()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Four styles supported:
|
||||||
|
//
|
||||||
|
// A1: AGG(CASE WHEN x = 'foo' THEN cnt END)
|
||||||
|
// => operands (x = 'foo', cnt, null)
|
||||||
|
// A2: SUM(CASE WHEN x = 'foo' THEN cnt ELSE 0 END)
|
||||||
|
// => operands (x = 'foo', cnt, 0); must be SUM
|
||||||
|
// B: SUM(CASE WHEN x = 'foo' THEN 1 ELSE 0 END)
|
||||||
|
// => operands (x = 'foo', 1, 0); must be SUM
|
||||||
|
// C: COUNT(CASE WHEN x = 'foo' THEN 'dummy' END)
|
||||||
|
// => operands (x = 'foo', 'dummy', null)
|
||||||
|
|
||||||
|
if (kind == SqlKind.COUNT // Case C
|
||||||
|
&& arg1.isA(SqlKind.LITERAL)
|
||||||
|
&& !RexLiteral.isNullLiteral(arg1)
|
||||||
|
&& RexLiteral.isNullLiteral(arg2)) {
|
||||||
|
newProjects.add(filter);
|
||||||
|
return AggregateCall.create(SqlStdOperatorTable.COUNT, false, false,
|
||||||
|
false, ImmutableList.of(), newProjects.size() - 1,
|
||||||
|
RelCollations.EMPTY, aggregateCall.getType(),
|
||||||
|
aggregateCall.getName()
|
||||||
|
);
|
||||||
|
} else if (kind == SqlKind.SUM // Case B
|
||||||
|
&& isIntLiteral(arg1) && RexLiteral.intValue(arg1) == 1
|
||||||
|
&& isIntLiteral(arg2) && RexLiteral.intValue(arg2) == 0) {
|
||||||
|
|
||||||
|
newProjects.add(filter);
|
||||||
|
final RelDataTypeFactory typeFactory = cluster.getTypeFactory();
|
||||||
|
final RelDataType dataType =
|
||||||
|
typeFactory.createTypeWithNullability(
|
||||||
|
typeFactory.createSqlType(SqlTypeName.BIGINT), false);
|
||||||
|
return AggregateCall.create(SqlStdOperatorTable.COUNT, false, false,
|
||||||
|
false, ImmutableList.of(), newProjects.size() - 1,
|
||||||
|
RelCollations.EMPTY, dataType, aggregateCall.getName()
|
||||||
|
);
|
||||||
|
} else if ((RexLiteral.isNullLiteral(arg2) // Case A1
|
||||||
|
&& aggregateCall.getAggregation().allowsFilter())
|
||||||
|
|| (kind == SqlKind.SUM // Case A2
|
||||||
|
&& isIntLiteral(arg2)
|
||||||
|
&& RexLiteral.intValue(arg2) == 0)) {
|
||||||
|
newProjects.add(arg1);
|
||||||
|
newProjects.add(filter);
|
||||||
|
return AggregateCall.create(aggregateCall.getAggregation(), false,
|
||||||
|
false, false, ImmutableList.of(newProjects.size() - 2),
|
||||||
|
newProjects.size() - 1, RelCollations.EMPTY,
|
||||||
|
aggregateCall.getType(), aggregateCall.getName()
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
newProjects.add(rexNode);
|
||||||
|
int callArg = newProjects.size() - 1;
|
||||||
|
int newFilterArg = -1;
|
||||||
|
if (aggregateCall.hasFilter()) {
|
||||||
|
newProjects.add(project.getProjects().get(aggregateCall.filterArg));
|
||||||
|
newFilterArg = newProjects.size() - 1;
|
||||||
|
}
|
||||||
|
return AggregateCall.create(aggregateCall.getAggregation(),
|
||||||
|
aggregateCall.isDistinct(),
|
||||||
|
aggregateCall.isApproximate(),
|
||||||
|
aggregateCall.ignoreNulls(),
|
||||||
|
ImmutableList.of(callArg),
|
||||||
|
newFilterArg,
|
||||||
|
aggregateCall.getCollation(),
|
||||||
|
aggregateCall.getType(),
|
||||||
|
aggregateCall.getName()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the argument, if an aggregate call has a single argument,
|
||||||
|
* otherwise -1.
|
||||||
|
*/
|
||||||
|
private static int soleArgument(AggregateCall aggregateCall)
|
||||||
|
{
|
||||||
|
return aggregateCall.getArgList().size() == 1
|
||||||
|
? aggregateCall.getArgList().get(0)
|
||||||
|
: -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static boolean isThreeArgCase(final RexNode rexNode)
|
||||||
|
{
|
||||||
|
return rexNode.getKind() == SqlKind.CASE
|
||||||
|
&& ((RexCall) rexNode).operands.size() == 3;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static boolean isIntLiteral(final RexNode rexNode)
|
||||||
|
{
|
||||||
|
return rexNode instanceof RexLiteral
|
||||||
|
&& SqlTypeName.INT_TYPES.contains(rexNode.getType().getSqlTypeName());
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,88 @@
|
||||||
|
/*
|
||||||
|
* 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.logical;
|
||||||
|
|
||||||
|
import org.apache.calcite.plan.RelTrait;
|
||||||
|
import org.apache.calcite.plan.RelTraitSet;
|
||||||
|
import org.apache.calcite.rel.RelCollation;
|
||||||
|
import org.apache.calcite.rel.RelCollationTraitDef;
|
||||||
|
import org.apache.calcite.rel.RelCollations;
|
||||||
|
import org.apache.calcite.rel.RelFieldCollation;
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
import org.apache.calcite.rel.convert.ConverterRule;
|
||||||
|
import org.apache.calcite.rel.core.Aggregate;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalAggregate;
|
||||||
|
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidAggregate;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link ConverterRule} to convert {@link Aggregate} to {@link DruidAggregate}
|
||||||
|
*/
|
||||||
|
public class DruidAggregateRule extends ConverterRule
|
||||||
|
{
|
||||||
|
private final PlannerContext plannerContext;
|
||||||
|
|
||||||
|
public DruidAggregateRule(
|
||||||
|
Class<? extends RelNode> clazz,
|
||||||
|
RelTrait in,
|
||||||
|
RelTrait out,
|
||||||
|
String descriptionPrefix,
|
||||||
|
PlannerContext plannerContext
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(clazz, in, out, descriptionPrefix);
|
||||||
|
this.plannerContext = plannerContext;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode convert(RelNode rel)
|
||||||
|
{
|
||||||
|
LogicalAggregate aggregate = (LogicalAggregate) rel;
|
||||||
|
RelTraitSet newTrait = deriveTraits(aggregate, aggregate.getTraitSet());
|
||||||
|
return new DruidAggregate(
|
||||||
|
aggregate.getCluster(),
|
||||||
|
newTrait,
|
||||||
|
convert(aggregate.getInput(), aggregate.getInput().getTraitSet().replace(DruidLogicalConvention.instance())),
|
||||||
|
aggregate.getGroupSet(),
|
||||||
|
aggregate.getGroupSets(),
|
||||||
|
aggregate.getAggCallList(),
|
||||||
|
plannerContext
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private RelTraitSet deriveTraits(Aggregate aggregate, RelTraitSet traits)
|
||||||
|
{
|
||||||
|
final RelCollation collation = traits.getTrait(RelCollationTraitDef.INSTANCE);
|
||||||
|
if ((collation == null || collation.getFieldCollations().isEmpty()) && aggregate.getGroupSets().size() == 1) {
|
||||||
|
// Druid sorts by grouping keys when grouping. Add the collation.
|
||||||
|
// Note: [aggregate.getGroupSets().size() == 1] above means that collation isn't added for GROUPING SETS.
|
||||||
|
final List<RelFieldCollation> sortFields = new ArrayList<>();
|
||||||
|
for (int i = 0; i < aggregate.getGroupCount(); i++) {
|
||||||
|
sortFields.add(new RelFieldCollation(i));
|
||||||
|
}
|
||||||
|
return traits.replace(DruidLogicalConvention.instance()).replace(RelCollations.of(sortFields));
|
||||||
|
}
|
||||||
|
return traits.replace(DruidLogicalConvention.instance());
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,62 @@
|
||||||
|
/*
|
||||||
|
* 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.logical;
|
||||||
|
|
||||||
|
import org.apache.calcite.plan.RelTrait;
|
||||||
|
import org.apache.calcite.plan.RelTraitSet;
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
import org.apache.calcite.rel.convert.ConverterRule;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalFilter;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidFilter;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link ConverterRule} to convert {@link org.apache.calcite.rel.core.Filter} to {@link DruidFilter}
|
||||||
|
*/
|
||||||
|
public class DruidFilterRule extends ConverterRule
|
||||||
|
{
|
||||||
|
|
||||||
|
public DruidFilterRule(
|
||||||
|
Class<? extends RelNode> clazz,
|
||||||
|
RelTrait in,
|
||||||
|
RelTrait out,
|
||||||
|
String descriptionPrefix
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(clazz, in, out, descriptionPrefix);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode convert(RelNode rel)
|
||||||
|
{
|
||||||
|
LogicalFilter filter = (LogicalFilter) rel;
|
||||||
|
RelTraitSet newTrait = filter.getTraitSet().replace(DruidLogicalConvention.instance());
|
||||||
|
return new DruidFilter(
|
||||||
|
filter.getCluster(),
|
||||||
|
newTrait,
|
||||||
|
convert(
|
||||||
|
filter.getInput(),
|
||||||
|
filter.getInput().getTraitSet()
|
||||||
|
.replace(DruidLogicalConvention.instance())
|
||||||
|
),
|
||||||
|
filter.getCondition()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,90 @@
|
||||||
|
/*
|
||||||
|
* 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.logical;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import org.apache.calcite.plan.Convention;
|
||||||
|
import org.apache.calcite.plan.RelOptRule;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalAggregate;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalFilter;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalProject;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalSort;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalTableScan;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalValues;
|
||||||
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
|
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
|
||||||
|
public class DruidLogicalRules
|
||||||
|
{
|
||||||
|
private final PlannerContext plannerContext;
|
||||||
|
|
||||||
|
public DruidLogicalRules(PlannerContext plannerContext)
|
||||||
|
{
|
||||||
|
this.plannerContext = plannerContext;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<RelOptRule> rules()
|
||||||
|
{
|
||||||
|
return new ArrayList<>(
|
||||||
|
ImmutableList.of(
|
||||||
|
new DruidTableScanRule(
|
||||||
|
RelOptRule.operand(LogicalTableScan.class, null, RelOptRule.any()),
|
||||||
|
StringUtils.format("%s", DruidTableScanRule.class.getSimpleName())
|
||||||
|
),
|
||||||
|
new DruidAggregateRule(
|
||||||
|
LogicalAggregate.class,
|
||||||
|
Convention.NONE,
|
||||||
|
DruidLogicalConvention.instance(),
|
||||||
|
DruidAggregateRule.class.getSimpleName(),
|
||||||
|
plannerContext
|
||||||
|
),
|
||||||
|
new DruidSortRule(
|
||||||
|
LogicalSort.class,
|
||||||
|
Convention.NONE,
|
||||||
|
DruidLogicalConvention.instance(),
|
||||||
|
DruidSortRule.class.getSimpleName()
|
||||||
|
),
|
||||||
|
new DruidProjectRule(
|
||||||
|
LogicalProject.class,
|
||||||
|
Convention.NONE,
|
||||||
|
DruidLogicalConvention.instance(),
|
||||||
|
DruidProjectRule.class.getSimpleName()
|
||||||
|
),
|
||||||
|
new DruidFilterRule(
|
||||||
|
LogicalFilter.class,
|
||||||
|
Convention.NONE,
|
||||||
|
DruidLogicalConvention.instance(),
|
||||||
|
DruidFilterRule.class.getSimpleName()
|
||||||
|
),
|
||||||
|
new DruidValuesRule(
|
||||||
|
LogicalValues.class,
|
||||||
|
Convention.NONE,
|
||||||
|
DruidLogicalConvention.instance(),
|
||||||
|
DruidValuesRule.class.getSimpleName()
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,59 @@
|
||||||
|
/*
|
||||||
|
* 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.logical;
|
||||||
|
|
||||||
|
import org.apache.calcite.plan.RelTrait;
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
import org.apache.calcite.rel.convert.ConverterRule;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalProject;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidProject;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link ConverterRule} to convert {@link org.apache.calcite.rel.core.Project} to {@link DruidProject}
|
||||||
|
*/
|
||||||
|
public class DruidProjectRule extends ConverterRule
|
||||||
|
{
|
||||||
|
|
||||||
|
public DruidProjectRule(
|
||||||
|
Class<? extends RelNode> clazz,
|
||||||
|
RelTrait in,
|
||||||
|
RelTrait out,
|
||||||
|
String descriptionPrefix
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(clazz, in, out, descriptionPrefix);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode convert(RelNode rel)
|
||||||
|
{
|
||||||
|
LogicalProject project = (LogicalProject) rel;
|
||||||
|
return DruidProject.create(
|
||||||
|
convert(
|
||||||
|
project.getInput(),
|
||||||
|
project.getInput().getTraitSet()
|
||||||
|
.replace(DruidLogicalConvention.instance())
|
||||||
|
),
|
||||||
|
project.getProjects(),
|
||||||
|
project.getRowType()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,60 @@
|
||||||
|
/*
|
||||||
|
* 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.logical;
|
||||||
|
|
||||||
|
import org.apache.calcite.plan.RelTrait;
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
import org.apache.calcite.rel.convert.ConverterRule;
|
||||||
|
import org.apache.calcite.rel.core.Sort;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalSort;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidSort;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link ConverterRule} to convert {@link Sort} to {@link DruidSort}
|
||||||
|
*/
|
||||||
|
public class DruidSortRule extends ConverterRule
|
||||||
|
{
|
||||||
|
|
||||||
|
public DruidSortRule(
|
||||||
|
Class<? extends RelNode> clazz,
|
||||||
|
RelTrait in,
|
||||||
|
RelTrait out,
|
||||||
|
String descriptionPrefix
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(clazz, in, out, descriptionPrefix);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode convert(RelNode rel)
|
||||||
|
{
|
||||||
|
LogicalSort sort = (LogicalSort) rel;
|
||||||
|
return DruidSort.create(
|
||||||
|
convert(
|
||||||
|
sort.getInput(),
|
||||||
|
sort.getInput().getTraitSet().replace(DruidLogicalConvention.instance())
|
||||||
|
),
|
||||||
|
sort.getCollation(),
|
||||||
|
sort.offset,
|
||||||
|
sort.fetch
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,54 @@
|
||||||
|
/*
|
||||||
|
* 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.logical;
|
||||||
|
|
||||||
|
import org.apache.calcite.plan.RelOptRule;
|
||||||
|
import org.apache.calcite.plan.RelOptRuleCall;
|
||||||
|
import org.apache.calcite.plan.RelOptRuleOperand;
|
||||||
|
import org.apache.calcite.plan.RelTraitSet;
|
||||||
|
import org.apache.calcite.rel.convert.ConverterRule;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalTableScan;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidTableScan;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link ConverterRule} to convert {@link org.apache.calcite.rel.core.TableScan} to {@link DruidTableScan}
|
||||||
|
*/
|
||||||
|
public class DruidTableScanRule extends RelOptRule
|
||||||
|
{
|
||||||
|
public DruidTableScanRule(RelOptRuleOperand operand, String description)
|
||||||
|
{
|
||||||
|
super(operand, description);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onMatch(RelOptRuleCall call)
|
||||||
|
{
|
||||||
|
LogicalTableScan tableScan = call.rel(0);
|
||||||
|
RelTraitSet newTrait = tableScan.getTraitSet().replace(DruidLogicalConvention.instance());
|
||||||
|
DruidTableScan druidTableScan = new DruidTableScan(
|
||||||
|
tableScan.getCluster(),
|
||||||
|
newTrait,
|
||||||
|
tableScan.getTable(),
|
||||||
|
null
|
||||||
|
);
|
||||||
|
call.transformTo(druidTableScan);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,58 @@
|
||||||
|
/*
|
||||||
|
* 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.logical;
|
||||||
|
|
||||||
|
import org.apache.calcite.plan.RelTrait;
|
||||||
|
import org.apache.calcite.plan.RelTraitSet;
|
||||||
|
import org.apache.calcite.rel.RelNode;
|
||||||
|
import org.apache.calcite.rel.convert.ConverterRule;
|
||||||
|
import org.apache.calcite.rel.logical.LogicalValues;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention;
|
||||||
|
import org.apache.druid.sql.calcite.rel.logical.DruidValues;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link ConverterRule} to convert {@link org.apache.calcite.rel.core.Values} to {@link DruidValues}
|
||||||
|
*/
|
||||||
|
public class DruidValuesRule extends ConverterRule
|
||||||
|
{
|
||||||
|
|
||||||
|
public DruidValuesRule(
|
||||||
|
Class<? extends RelNode> clazz,
|
||||||
|
RelTrait in,
|
||||||
|
RelTrait out,
|
||||||
|
String descriptionPrefix
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(clazz, in, out, descriptionPrefix);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RelNode convert(RelNode rel)
|
||||||
|
{
|
||||||
|
LogicalValues values = (LogicalValues) rel;
|
||||||
|
RelTraitSet newTrait = values.getTraitSet().replace(DruidLogicalConvention.instance());
|
||||||
|
return new DruidValues(
|
||||||
|
values.getCluster(),
|
||||||
|
newTrait,
|
||||||
|
values.getRowType(),
|
||||||
|
values.getTuples()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -858,6 +858,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
||||||
public class CalciteTestConfig implements QueryTestBuilder.QueryTestConfig
|
public class CalciteTestConfig implements QueryTestBuilder.QueryTestConfig
|
||||||
{
|
{
|
||||||
private boolean isRunningMSQ = false;
|
private boolean isRunningMSQ = false;
|
||||||
|
private Map<String, Object> baseQueryContext;
|
||||||
|
|
||||||
public CalciteTestConfig()
|
public CalciteTestConfig()
|
||||||
{
|
{
|
||||||
|
@ -868,6 +869,11 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
||||||
this.isRunningMSQ = isRunningMSQ;
|
this.isRunningMSQ = isRunningMSQ;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public CalciteTestConfig(Map<String, Object> baseQueryContext)
|
||||||
|
{
|
||||||
|
this.baseQueryContext = baseQueryContext;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public QueryLogHook queryLogHook()
|
public QueryLogHook queryLogHook()
|
||||||
{
|
{
|
||||||
|
@ -909,6 +915,12 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
||||||
{
|
{
|
||||||
return isRunningMSQ;
|
return isRunningMSQ;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, Object> baseQueryContext()
|
||||||
|
{
|
||||||
|
return baseQueryContext;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void assertResultsEquals(String sql, List<Object[]> expectedResults, List<Object[]> results)
|
public void assertResultsEquals(String sql, List<Object[]> expectedResults, List<Object[]> results)
|
||||||
|
|
|
@ -0,0 +1,425 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import org.apache.druid.query.QueryContexts;
|
||||||
|
import org.apache.druid.server.security.AuthConfig;
|
||||||
|
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||||
|
import org.apache.druid.sql.calcite.util.SqlTestFramework;
|
||||||
|
import org.junit.Ignore;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
public class DecoupledPlanningCalciteQueryTest extends CalciteQueryTest
|
||||||
|
{
|
||||||
|
private static final ImmutableMap<String, Object> CONTEXT_OVERRIDES = ImmutableMap.of(
|
||||||
|
PlannerConfig.CTX_NATIVE_QUERY_SQL_PLANNING_MODE, PlannerConfig.NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED,
|
||||||
|
QueryContexts.ENABLE_DEBUG, true
|
||||||
|
);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected QueryTestBuilder testBuilder()
|
||||||
|
{
|
||||||
|
return new QueryTestBuilder(
|
||||||
|
new CalciteTestConfig(CONTEXT_OVERRIDES)
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public SqlTestFramework.PlannerFixture plannerFixture(PlannerConfig plannerConfig, AuthConfig authConfig)
|
||||||
|
{
|
||||||
|
plannerConfig = plannerConfig.withOverrides(CONTEXT_OVERRIDES);
|
||||||
|
return queryFramework().plannerFixture(DecoupledPlanningCalciteQueryTest.this, plannerConfig, authConfig);
|
||||||
|
}
|
||||||
|
})
|
||||||
|
.cannotVectorize(cannotVectorize)
|
||||||
|
.skipVectorize(skipVectorize);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testGroupByWithSelectAndOrderByProjections()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testTopNWithSelectAndOrderByProjections()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionAllQueries()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionAllQueriesWithLimit()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionAllDifferentTablesWithMapping()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testJoinUnionAllDifferentTablesWithMapping()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionAllTablesColumnTypeMismatchFloatLong()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionAllTablesColumnTypeMismatchStringLong()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionAllTablesWhenMappingIsRequired()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionIsUnplannable()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionAllTablesWhenCastAndMappingIsRequired()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionAllSameTableTwice()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionAllSameTableTwiceWithSameMapping()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionAllSameTableTwiceWithDifferentMapping()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionAllSameTableThreeTimes()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnionAllSameTableThreeTimesWithSameMapping()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testSelfJoin()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testTwoExactCountDistincts()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testViewAndJoin()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testGroupByWithSortOnPostAggregationDefault()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testGroupByWithSortOnPostAggregationNoTopNConfig()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testGroupByWithSortOnPostAggregationNoTopNContext()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnplannableQueries()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnplannableTwoExactCountDistincts()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUnplannableExactCountDistinctOnSketch()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testExactCountDistinctUsingSubqueryOnUnionAllTables()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUseTimeFloorInsteadOfGranularityOnJoinResult()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testMinMaxAvgDailyCountWithLimit()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testExactCountDistinctOfSemiJoinResult()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testMaxSubqueryRows()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testExactCountDistinctUsingSubqueryWithWherePushDown()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUsingSubqueryAsFilterOnTwoColumns()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUsingSubqueryAsFilterWithInnerSort()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testUsingSubqueryWithLimit()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testPostAggWithTimeseries()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testPostAggWithTopN()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testRequireTimeConditionPositive()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void testRequireTimeConditionSemiJoinNegative()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testEmptyGroupWithOffsetDoesntInfiniteLoop()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testJoinWithTimeDimension()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testSubqueryTypeMismatchWithLiterals()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testTimeseriesQueryWithEmptyInlineDatasourceAndGranularity()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testGroupBySortPushDown()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testGroupingWithNullInFilter()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
@Test
|
||||||
|
public void testStringAggExpressionNonConstantSeparator()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testOrderByAlongWithInternalScanQuery()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testSortProjectAfterNestedGroupBy()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testOrderByAlongWithInternalScanQueryNoDistinct()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testNestedGroupBy()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testQueryWithSelectProjectAndIdentityProjectDoesNotRename()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testFilterOnCurrentTimestampWithIntervalArithmetic()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Ignore
|
||||||
|
public void testFilterOnCurrentTimestampOnView()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -21,6 +21,7 @@ package org.apache.druid.sql.calcite;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import org.apache.druid.query.Query;
|
import org.apache.druid.query.Query;
|
||||||
|
import org.apache.druid.query.QueryContexts;
|
||||||
import org.apache.druid.segment.column.RowSignature;
|
import org.apache.druid.segment.column.RowSignature;
|
||||||
import org.apache.druid.server.security.AuthConfig;
|
import org.apache.druid.server.security.AuthConfig;
|
||||||
import org.apache.druid.server.security.AuthenticationResult;
|
import org.apache.druid.server.security.AuthenticationResult;
|
||||||
|
@ -79,11 +80,13 @@ public class QueryTestBuilder
|
||||||
ResultsVerifier defaultResultsVerifier(List<Object[]> expectedResults, RowSignature expectedResultSignature);
|
ResultsVerifier defaultResultsVerifier(List<Object[]> expectedResults, RowSignature expectedResultSignature);
|
||||||
|
|
||||||
boolean isRunningMSQ();
|
boolean isRunningMSQ();
|
||||||
|
|
||||||
|
Map<String, Object> baseQueryContext();
|
||||||
}
|
}
|
||||||
|
|
||||||
protected final QueryTestConfig config;
|
protected final QueryTestConfig config;
|
||||||
protected PlannerConfig plannerConfig = BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT;
|
protected PlannerConfig plannerConfig = BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT;
|
||||||
protected Map<String, Object> queryContext = BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT;
|
protected Map<String, Object> queryContext;
|
||||||
protected List<SqlParameter> parameters = CalciteTestBase.DEFAULT_PARAMETERS;
|
protected List<SqlParameter> parameters = CalciteTestBase.DEFAULT_PARAMETERS;
|
||||||
protected String sql;
|
protected String sql;
|
||||||
protected AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
|
protected AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
|
||||||
|
@ -108,6 +111,12 @@ public class QueryTestBuilder
|
||||||
public QueryTestBuilder(final QueryTestConfig config)
|
public QueryTestBuilder(final QueryTestConfig config)
|
||||||
{
|
{
|
||||||
this.config = config;
|
this.config = config;
|
||||||
|
// Done to maintain backwards compat. So,
|
||||||
|
// 1. If no base context is provided in config, the queryContext is set to the default one
|
||||||
|
// 2. If some base context is provided in config, we set that context as the queryContext
|
||||||
|
// 3. If someone overrides the context, we merge the context with the empty/non-empty base context provided in the config
|
||||||
|
this.queryContext =
|
||||||
|
config.baseQueryContext() == null ? BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT : config.baseQueryContext();
|
||||||
}
|
}
|
||||||
|
|
||||||
public QueryTestBuilder plannerConfig(PlannerConfig plannerConfig)
|
public QueryTestBuilder plannerConfig(PlannerConfig plannerConfig)
|
||||||
|
@ -118,7 +127,7 @@ public class QueryTestBuilder
|
||||||
|
|
||||||
public QueryTestBuilder queryContext(Map<String, Object> queryContext)
|
public QueryTestBuilder queryContext(Map<String, Object> queryContext)
|
||||||
{
|
{
|
||||||
this.queryContext = queryContext;
|
this.queryContext = QueryContexts.override(config.baseQueryContext(), queryContext);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue