SQL: EXPLAIN improvements. (#4733)

* SQL: EXPLAIN improvements.

- Include query JSON in explain output.
- Fix a bug where semi-joins and nested groupBys were not fully explained.
- Fix a bug where limits were not included in "select" query explanations.

* Fix compile error.

* Fix compile error.

* Fix tests.
This commit is contained in:
Gian Merlino 2017-09-01 09:35:13 -07:00 committed by Fangjin Yang
parent 4c04083926
commit 34a03b8e6c
19 changed files with 383 additions and 179 deletions

View File

@ -25,8 +25,8 @@ import com.google.common.io.Files;
import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
import io.druid.benchmark.datagen.BenchmarkSchemas;
import io.druid.benchmark.datagen.SegmentGenerator;
import io.druid.java.util.common.Intervals;
import io.druid.data.input.Row;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
@ -118,7 +118,8 @@ public class SqlBenchmark
CalciteTests.createMockQueryLifecycleFactory(walker),
CalciteTests.createOperatorTable(),
CalciteTests.createExprMacroTable(),
plannerConfig
plannerConfig,
CalciteTests.getJsonMapper()
);
groupByQuery = GroupByQuery
.builder()

View File

@ -132,7 +132,8 @@ public class QuantileSqlAggregatorTest
CalciteTests.createMockQueryLifecycleFactory(walker),
operatorTable,
CalciteTests.createExprMacroTable(),
plannerConfig
plannerConfig,
CalciteTests.getJsonMapper()
);
}

View File

@ -19,7 +19,9 @@
package io.druid.sql.calcite.planner;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import io.druid.guice.annotations.Json;
import io.druid.math.expr.ExprMacroTable;
import io.druid.server.QueryLifecycleFactory;
import io.druid.sql.calcite.rel.QueryMaker;
@ -53,6 +55,7 @@ public class PlannerFactory
private final DruidOperatorTable operatorTable;
private final ExprMacroTable macroTable;
private final PlannerConfig plannerConfig;
private final ObjectMapper jsonMapper;
@Inject
public PlannerFactory(
@ -60,7 +63,8 @@ public class PlannerFactory
final QueryLifecycleFactory queryLifecycleFactory,
final DruidOperatorTable operatorTable,
final ExprMacroTable macroTable,
final PlannerConfig plannerConfig
final PlannerConfig plannerConfig,
final @Json ObjectMapper jsonMapper
)
{
this.druidSchema = druidSchema;
@ -68,13 +72,14 @@ public class PlannerFactory
this.operatorTable = operatorTable;
this.macroTable = macroTable;
this.plannerConfig = plannerConfig;
this.jsonMapper = jsonMapper;
}
public DruidPlanner createPlanner(final Map<String, Object> queryContext)
{
final SchemaPlus rootSchema = Calcites.createRootSchema(druidSchema);
final PlannerContext plannerContext = PlannerContext.create(operatorTable, macroTable, plannerConfig, queryContext);
final QueryMaker queryMaker = new QueryMaker(queryLifecycleFactory, plannerContext);
final QueryMaker queryMaker = new QueryMaker(queryLifecycleFactory, plannerContext, jsonMapper);
final FrameworkConfig frameworkConfig = Frameworks
.newConfigBuilder()
.parserConfig(PARSER_CONFIG)

View File

@ -219,7 +219,7 @@ public class Rules
rules.add(CaseFilteredAggregatorRule.instance());
// Druid-specific rules.
rules.add(new DruidTableScanRule(plannerContext, queryMaker));
rules.add(new DruidTableScanRule(queryMaker));
rules.add(new DruidFilterRule());
if (plannerConfig.getMaxSemiJoinRowsInMemory() > 0) {

View File

@ -19,48 +19,51 @@
package io.druid.sql.calcite.rel;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.QueryDataSource;
import io.druid.query.TableDataSource;
import io.druid.query.filter.DimFilter;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.interpreter.BindableConvention;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptCost;
import org.apache.calcite.plan.RelOptPlanner;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelWriter;
import org.apache.calcite.rel.metadata.RelMetadataQuery;
import org.apache.calcite.rel.type.RelDataType;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.List;
public class DruidNestedGroupBy extends DruidRel<DruidNestedGroupBy>
/**
* DruidRel that uses a "query" dataSource.
*/
public class DruidOuterQueryRel extends DruidRel<DruidOuterQueryRel>
{
private final DruidRel sourceRel;
private final RelNode sourceRel;
private final DruidQueryBuilder queryBuilder;
private DruidNestedGroupBy(
private DruidOuterQueryRel(
RelOptCluster cluster,
RelTraitSet traitSet,
DruidRel sourceRel,
DruidQueryBuilder queryBuilder
RelNode sourceRel,
DruidQueryBuilder queryBuilder,
QueryMaker queryMaker
)
{
super(cluster, traitSet, sourceRel.getQueryMaker());
super(cluster, traitSet, queryMaker);
this.sourceRel = sourceRel;
this.queryBuilder = queryBuilder;
if (sourceRel.getQueryBuilder().getGrouping() == null) {
throw new IllegalArgumentException("inner query must be groupBy");
}
if (queryBuilder.getGrouping() == null) {
throw new IllegalArgumentException("outer query must be groupBy");
}
}
public static DruidNestedGroupBy from(
public static DruidOuterQueryRel from(
final DruidRel sourceRel,
final DimFilter filter,
final Grouping grouping,
@ -68,21 +71,22 @@ public class DruidNestedGroupBy extends DruidRel<DruidNestedGroupBy>
final List<String> rowOrder
)
{
return new DruidNestedGroupBy(
return new DruidOuterQueryRel(
sourceRel.getCluster(),
sourceRel.getTraitSet(),
sourceRel,
DruidQueryBuilder.fullScan(
sourceRel.getOutputRowSignature(),
sourceRel.getCluster().getTypeFactory()
).withFilter(filter).withGrouping(grouping, rowType, rowOrder)
).withFilter(filter).withGrouping(grouping, rowType, rowOrder),
sourceRel.getQueryMaker()
);
}
@Override
public RowSignature getSourceRowSignature()
{
return sourceRel.getOutputRowSignature();
return ((DruidRel) sourceRel).getOutputRowSignature();
}
@Override
@ -94,7 +98,7 @@ public class DruidNestedGroupBy extends DruidRel<DruidNestedGroupBy>
@Override
public Sequence<Object[]> runQuery()
{
final QueryDataSource queryDataSource = sourceRel.asDataSource();
final QueryDataSource queryDataSource = ((DruidRel) sourceRel).asDataSource();
if (queryDataSource != null) {
return getQueryMaker().runQuery(
queryDataSource,
@ -106,26 +110,28 @@ public class DruidNestedGroupBy extends DruidRel<DruidNestedGroupBy>
}
@Override
public DruidNestedGroupBy withQueryBuilder(DruidQueryBuilder newQueryBuilder)
public DruidOuterQueryRel withQueryBuilder(final DruidQueryBuilder newQueryBuilder)
{
return new DruidNestedGroupBy(
return new DruidOuterQueryRel(
getCluster(),
getTraitSet().plusAll(newQueryBuilder.getRelTraits()),
sourceRel,
newQueryBuilder
newQueryBuilder,
getQueryMaker()
);
}
@Override
public int getQueryCount()
{
return 1 + sourceRel.getQueryCount();
return 1 + ((DruidRel) sourceRel).getQueryCount();
}
@Nullable
@Override
public QueryDataSource asDataSource()
{
final QueryDataSource queryDataSource = sourceRel.asDataSource();
final QueryDataSource queryDataSource = ((DruidRel) sourceRel).asDataSource();
if (queryDataSource == null) {
return null;
} else {
@ -134,33 +140,65 @@ public class DruidNestedGroupBy extends DruidRel<DruidNestedGroupBy>
}
@Override
public DruidNestedGroupBy asBindable()
public DruidOuterQueryRel asBindable()
{
return new DruidNestedGroupBy(
return new DruidOuterQueryRel(
getCluster(),
getTraitSet().plus(BindableConvention.INSTANCE),
sourceRel,
queryBuilder
queryBuilder,
getQueryMaker()
);
}
@Override
public DruidNestedGroupBy asDruidConvention()
public DruidOuterQueryRel asDruidConvention()
{
return new DruidNestedGroupBy(
return new DruidOuterQueryRel(
getCluster(),
getTraitSet().plus(DruidConvention.instance()),
sourceRel,
queryBuilder
RelOptRule.convert(sourceRel, DruidConvention.instance()),
queryBuilder,
getQueryMaker()
);
}
@Override
public List<RelNode> getInputs()
{
return ImmutableList.of(sourceRel);
}
@Override
public RelNode copy(final RelTraitSet traitSet, final List<RelNode> inputs)
{
return new DruidOuterQueryRel(
getCluster(),
traitSet,
Iterables.getOnlyElement(inputs),
getQueryBuilder(),
getQueryMaker()
);
}
@Override
public RelWriter explainTerms(RelWriter pw)
{
return pw
.item("sourceRel", sourceRel)
.item("queryBuilder", queryBuilder);
final TableDataSource dummyDataSource = new TableDataSource("__subquery__");
final String queryString;
try {
queryString = getQueryMaker()
.getJsonMapper()
.writeValueAsString(queryBuilder.toGroupByQuery(dummyDataSource, getPlannerContext()));
}
catch (IOException e) {
throw new RuntimeException(e);
}
return super.explainTerms(pw)
.input("innerQuery", sourceRel)
.item("query", queryString);
}
@Override
@ -172,6 +210,6 @@ public class DruidNestedGroupBy extends DruidRel<DruidNestedGroupBy>
@Override
public RelOptCost computeSelfCost(final RelOptPlanner planner, final RelMetadataQuery mq)
{
return sourceRel.computeSelfCost(planner, mq).multiplyBy(2.0);
return planner.getCostFactory().makeCost(mq.getRowCount(sourceRel), 0, 0);
}
}

View File

@ -29,6 +29,8 @@ import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.math.expr.ExprMacroTable;
import io.druid.query.DataSource;
import io.druid.query.Query;
import io.druid.query.QueryDataSource;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.filter.DimFilter;
@ -64,6 +66,7 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelDataTypeField;
import org.apache.calcite.sql.type.SqlTypeName;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
@ -337,6 +340,55 @@ public class DruidQueryBuilder
}
}
/**
* Return this query as some kind of Druid query. The returned query will either be {@link TopNQuery},
* {@link TimeseriesQuery}, {@link GroupByQuery}, or {@link SelectQuery}.
*
* @param dataSource data source to query
* @param plannerContext planner context
*
* @return Druid query
*/
public Query toQuery(
final DataSource dataSource,
final PlannerContext plannerContext
)
{
if (dataSource instanceof QueryDataSource) {
// If there is a subquery then the outer query must be a groupBy.
final GroupByQuery outerQuery = toGroupByQuery(dataSource, plannerContext);
if (outerQuery == null) {
// Bug in the planner rules. They shouldn't allow this to happen.
throw new IllegalStateException("Can't use QueryDataSource without an outer groupBy query!");
}
return outerQuery;
}
final TimeseriesQuery tsQuery = toTimeseriesQuery(dataSource, plannerContext);
if (tsQuery != null) {
return tsQuery;
}
final TopNQuery topNQuery = toTopNQuery(dataSource, plannerContext);
if (topNQuery != null) {
return topNQuery;
}
final GroupByQuery groupByQuery = toGroupByQuery(dataSource, plannerContext);
if (groupByQuery != null) {
return groupByQuery;
}
final SelectQuery selectQuery = toSelectQuery(dataSource, plannerContext);
if (selectQuery != null) {
return selectQuery;
}
throw new IllegalStateException("WTF?! Cannot build a query even though we planned it?");
}
/**
* Return this query as a Timeseries query, or null if this query is not compatible with Timeseries.
*
@ -345,6 +397,7 @@ public class DruidQueryBuilder
*
* @return query or null
*/
@Nullable
public TimeseriesQuery toTimeseriesQuery(
final DataSource dataSource,
final PlannerContext plannerContext
@ -417,6 +470,7 @@ public class DruidQueryBuilder
*
* @return query or null
*/
@Nullable
public TopNQuery toTopNQuery(
final DataSource dataSource,
final PlannerContext plannerContext
@ -491,6 +545,7 @@ public class DruidQueryBuilder
*
* @return query or null
*/
@Nullable
public GroupByQuery toGroupByQuery(
final DataSource dataSource,
final PlannerContext plannerContext
@ -525,6 +580,7 @@ public class DruidQueryBuilder
*
* @return query or null
*/
@Nullable
public SelectQuery toSelectQuery(
final DataSource dataSource,
final PlannerContext plannerContext
@ -536,6 +592,7 @@ public class DruidQueryBuilder
final Filtration filtration = Filtration.create(filter).optimize(sourceRowSignature);
final boolean descending;
final int threshold;
if (limitSpec != null) {
// Safe to assume limitSpec has zero or one entry; DruidSelectSortRule wouldn't push in anything else.
@ -548,8 +605,11 @@ public class DruidQueryBuilder
} else {
descending = false;
}
threshold = limitSpec.getLimit();
} else {
descending = false;
threshold = 0;
}
// We need to ask for dummy columns to prevent Select from returning all of them.
@ -576,6 +636,9 @@ public class DruidQueryBuilder
metrics.add(dummyColumn);
}
// Not used for actual queries (will be replaced by QueryMaker) but the threshold is important for the planner.
final PagingSpec pagingSpec = new PagingSpec(null, threshold);
return new SelectQuery(
dataSource,
filtration.getQuerySegmentSpec(),
@ -585,7 +648,7 @@ public class DruidQueryBuilder
ImmutableList.of(new DefaultDimensionSpec(dummyColumn, dummyColumn)),
metrics.stream().sorted().distinct().collect(Collectors.toList()),
getVirtualColumns(plannerContext.getExprMacroTable()),
new PagingSpec(null, 0) /* dummy -- will be replaced */,
pagingSpec,
plannerContext.getQueryContext()
);
}

View File

@ -20,13 +20,9 @@
package io.druid.sql.calcite.rel;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.guava.Sequence;
import io.druid.query.QueryDataSource;
import io.druid.query.groupby.GroupByQuery;
import io.druid.segment.VirtualColumns;
import io.druid.sql.calcite.filtration.Filtration;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.DruidTable;
import io.druid.sql.calcite.table.RowSignature;
import org.apache.calcite.interpreter.BindableConvention;
@ -40,14 +36,22 @@ import org.apache.calcite.rel.RelWriter;
import org.apache.calcite.rel.metadata.RelMetadataQuery;
import org.apache.calcite.rel.type.RelDataType;
import javax.annotation.Nullable;
import java.io.IOException;
/**
* DruidRel that uses a "table" dataSource.
*/
public class DruidQueryRel extends DruidRel<DruidQueryRel>
{
// Factors used for computing cost (see computeSelfCost). These are intended to encourage pushing down filters
// and limits through stacks of nested queries when possible.
private static final double COST_BASE = 1.0;
private static final double COST_PER_COLUMN = 0.001;
private static final double COST_FILTER_MULTIPLIER = 0.1;
private static final double COST_GROUPING_MULTIPLIER = 0.5;
private static final double COST_LIMIT_MULTIPLIER = 0.5;
private static final double COST_HAVING_MULTIPLIER = 5.0;
private final RelOptTable table;
private final DruidTable druidTable;
@ -75,7 +79,6 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
final RelOptCluster cluster,
final RelOptTable table,
final DruidTable druidTable,
final PlannerContext plannerContext,
final QueryMaker queryMaker
)
{
@ -89,6 +92,7 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
);
}
@Nullable
@Override
public QueryDataSource asDataSource()
{
@ -182,34 +186,18 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
@Override
public RelWriter explainTerms(final RelWriter pw)
{
pw.item("dataSource", druidTable.getDataSource());
if (queryBuilder != null) {
final Filtration filtration = Filtration.create(queryBuilder.getFilter()).optimize(getSourceRowSignature());
final VirtualColumns virtualColumns = queryBuilder.getVirtualColumns(getPlannerContext().getExprMacroTable());
if (!virtualColumns.isEmpty()) {
pw.item("virtualColumns", virtualColumns);
}
if (!filtration.getIntervals().equals(ImmutableList.of(Filtration.eternity()))) {
pw.item("intervals", filtration.getIntervals());
}
if (filtration.getDimFilter() != null) {
pw.item("filter", filtration.getDimFilter());
}
if (queryBuilder.getSelectProjection() != null) {
pw.item("selectProjection", queryBuilder.getSelectProjection());
}
if (queryBuilder.getGrouping() != null) {
pw.item("dimensions", queryBuilder.getGrouping().getDimensions());
pw.item("aggregations", queryBuilder.getGrouping().getAggregations());
}
if (queryBuilder.getHaving() != null) {
pw.item("having", queryBuilder.getHaving());
}
if (queryBuilder.getLimitSpec() != null) {
pw.item("limitSpec", queryBuilder.getLimitSpec());
}
final String queryString;
try {
queryString = getQueryMaker()
.getJsonMapper()
.writeValueAsString(queryBuilder.toQuery(druidTable.getDataSource(), getPlannerContext()));
}
return pw;
catch (IOException e) {
throw new RuntimeException(e);
}
return pw.item("query", queryString);
}
@Override
@ -217,18 +205,29 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
{
double cost = COST_BASE;
if (queryBuilder.getSelectProjection() != null) {
cost += COST_PER_COLUMN * queryBuilder.getSelectProjection().getVirtualColumns().size();
cost += COST_PER_COLUMN * queryBuilder.getSelectProjection().getDirectColumns().size();
}
if (queryBuilder.getFilter() != null) {
cost *= COST_FILTER_MULTIPLIER;
}
if (queryBuilder.getGrouping() != null) {
cost *= COST_GROUPING_MULTIPLIER;
cost += COST_PER_COLUMN * queryBuilder.getGrouping().getAggregatorFactories().size();
cost += COST_PER_COLUMN * queryBuilder.getGrouping().getPostAggregators().size();
}
if (queryBuilder.getLimitSpec() != null) {
if (queryBuilder.getLimitSpec() != null && queryBuilder.getLimitSpec().getLimit() < Integer.MAX_VALUE) {
cost *= COST_LIMIT_MULTIPLIER;
}
if (queryBuilder.getHaving() != null) {
cost *= COST_HAVING_MULTIPLIER;
}
return planner.getCostFactory().makeCost(cost, 0, 0);
}
}

View File

@ -35,11 +35,13 @@ import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.AbstractRelNode;
import javax.annotation.Nullable;
public abstract class DruidRel<T extends DruidRel> extends AbstractRelNode implements BindableRel
{
private final QueryMaker queryMaker;
public DruidRel(RelOptCluster cluster, RelTraitSet traitSet, QueryMaker queryMaker)
protected DruidRel(RelOptCluster cluster, RelTraitSet traitSet, QueryMaker queryMaker)
{
super(cluster, traitSet);
this.queryMaker = queryMaker;
@ -74,6 +76,7 @@ public abstract class DruidRel<T extends DruidRel> extends AbstractRelNode imple
*
* @return query dataSource, or null if it is known in advance that this rel will yield an empty result set.
*/
@Nullable
public abstract QueryDataSource asDataSource();
public abstract T asBindable();

View File

@ -20,6 +20,7 @@
package io.druid.sql.calcite.rel;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import io.druid.java.util.common.StringUtils;
@ -28,6 +29,7 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.QueryDataSource;
import io.druid.query.ResourceLimitExceededException;
import io.druid.query.TableDataSource;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DimFilter;
@ -42,19 +44,26 @@ import org.apache.calcite.interpreter.BindableConvention;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptCost;
import org.apache.calcite.plan.RelOptPlanner;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelWriter;
import org.apache.calcite.rel.metadata.RelMetadataQuery;
import org.apache.calcite.rel.type.RelDataType;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
/**
* DruidRel that has a main query, and also a subquery "right" that is used to filter the main query.
*/
public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
{
private final DruidRel<?> left;
private final DruidRel<?> right;
private final RelNode right;
private final List<DruidExpression> leftExpressions;
private final List<Integer> rightKeys;
private final int maxSemiJoinRowsInMemory;
@ -62,14 +71,15 @@ public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
private DruidSemiJoin(
final RelOptCluster cluster,
final RelTraitSet traitSet,
final DruidRel left,
final DruidRel right,
final DruidRel<?> left,
final RelNode right,
final List<DruidExpression> leftExpressions,
final List<Integer> rightKeys,
final int maxSemiJoinRowsInMemory
final int maxSemiJoinRowsInMemory,
final QueryMaker queryMaker
)
{
super(cluster, traitSet, left.getQueryMaker());
super(cluster, traitSet, queryMaker);
this.left = left;
this.right = right;
this.leftExpressions = ImmutableList.copyOf(leftExpressions);
@ -114,7 +124,8 @@ public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
right,
listBuilder.build(),
rightKeys,
plannerContext.getPlannerConfig().getMaxSemiJoinRowsInMemory()
plannerContext.getPlannerConfig().getMaxSemiJoinRowsInMemory(),
left.getQueryMaker()
);
}
@ -140,10 +151,12 @@ public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
right,
leftExpressions,
rightKeys,
maxSemiJoinRowsInMemory
maxSemiJoinRowsInMemory,
getQueryMaker()
);
}
@Nullable
@Override
public QueryDataSource asDataSource()
{
@ -158,10 +171,11 @@ public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
getCluster(),
getTraitSet().replace(BindableConvention.INSTANCE),
left,
right,
RelOptRule.convert(right, BindableConvention.INSTANCE),
leftExpressions,
rightKeys,
maxSemiJoinRowsInMemory
maxSemiJoinRowsInMemory,
getQueryMaker()
);
}
@ -172,17 +186,18 @@ public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
getCluster(),
getTraitSet().replace(DruidConvention.instance()),
left,
right,
RelOptRule.convert(right, DruidConvention.instance()),
leftExpressions,
rightKeys,
maxSemiJoinRowsInMemory
maxSemiJoinRowsInMemory,
getQueryMaker()
);
}
@Override
public int getQueryCount()
{
return left.getQueryCount() + right.getQueryCount();
return ((DruidRel) left).getQueryCount() + ((DruidRel) right).getQueryCount();
}
@Override
@ -202,14 +217,47 @@ public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
return left.getRowType();
}
@Override
public List<RelNode> getInputs()
{
return ImmutableList.of(right);
}
@Override
public RelNode copy(final RelTraitSet traitSet, final List<RelNode> inputs)
{
return new DruidSemiJoin(
getCluster(),
getTraitSet(),
left,
Iterables.getOnlyElement(inputs),
leftExpressions,
rightKeys,
maxSemiJoinRowsInMemory,
getQueryMaker()
);
}
@Override
public RelWriter explainTerms(RelWriter pw)
{
return pw
.item("leftExpressions", leftExpressions)
.item("leftQuery", left.getQueryBuilder())
.item("rightKeys", rightKeys)
.item("rightQuery", right.getQueryBuilder());
final TableDataSource dummyDataSource = new TableDataSource("__subquery__");
final String queryString;
try {
queryString = getQueryMaker()
.getJsonMapper()
.writeValueAsString(left.getQueryBuilder().toQuery(dummyDataSource, getPlannerContext()));
}
catch (IOException e) {
throw new RuntimeException(e);
}
return super.explainTerms(pw)
.input("right", right)
.item("query", queryString)
.item("leftExpressions", leftExpressions)
.item("rightKeys", rightKeys);
}
@Override
@ -224,9 +272,11 @@ public class DruidSemiJoin extends DruidRel<DruidSemiJoin>
*/
private DruidRel<?> getLeftRelWithFilter()
{
final DruidRel<?> druidRight = (DruidRel) this.right;
// Build list of acceptable values from right side.
final Set<List<String>> valuess = Sets.newHashSet();
final List<DimFilter> filters = right.runQuery().accumulate(
final List<DimFilter> filters = druidRight.runQuery().accumulate(
new ArrayList<>(),
new Accumulator<List<DimFilter>, Object[]>()
{

View File

@ -19,6 +19,7 @@
package io.druid.sql.calcite.rel;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
@ -32,7 +33,6 @@ import io.druid.java.util.common.guava.Sequences;
import io.druid.math.expr.Evals;
import io.druid.query.DataSource;
import io.druid.query.Query;
import io.druid.query.QueryDataSource;
import io.druid.query.Result;
import io.druid.query.groupby.GroupByQuery;
import io.druid.query.select.EventHolder;
@ -68,14 +68,17 @@ public class QueryMaker
{
private final QueryLifecycleFactory queryLifecycleFactory;
private final PlannerContext plannerContext;
private final ObjectMapper jsonMapper;
public QueryMaker(
final QueryLifecycleFactory queryLifecycleFactory,
final PlannerContext plannerContext
final PlannerContext plannerContext,
final ObjectMapper jsonMapper
)
{
this.queryLifecycleFactory = queryLifecycleFactory;
this.plannerContext = plannerContext;
this.jsonMapper = jsonMapper;
}
public PlannerContext getPlannerContext()
@ -83,42 +86,29 @@ public class QueryMaker
return plannerContext;
}
public ObjectMapper getJsonMapper()
{
return jsonMapper;
}
public Sequence<Object[]> runQuery(
final DataSource dataSource,
final DruidQueryBuilder queryBuilder
)
{
if (dataSource instanceof QueryDataSource) {
final GroupByQuery outerQuery = queryBuilder.toGroupByQuery(dataSource, plannerContext);
if (outerQuery == null) {
// Bug in the planner rules. They shouldn't allow this to happen.
throw new IllegalStateException("Can't use QueryDataSource without an outer groupBy query!");
}
final Query query = queryBuilder.toQuery(dataSource, plannerContext);
return executeGroupBy(queryBuilder, outerQuery);
if (query instanceof TimeseriesQuery) {
return executeTimeseries(queryBuilder, (TimeseriesQuery) query);
} else if (query instanceof TopNQuery) {
return executeTopN(queryBuilder, (TopNQuery) query);
} else if (query instanceof GroupByQuery) {
return executeGroupBy(queryBuilder, (GroupByQuery) query);
} else if (query instanceof SelectQuery) {
return executeSelect(queryBuilder, (SelectQuery) query);
} else {
throw new ISE("Cannot run query of class[%s]", query.getClass().getName());
}
final TimeseriesQuery tsQuery = queryBuilder.toTimeseriesQuery(dataSource, plannerContext);
if (tsQuery != null) {
return executeTimeseries(queryBuilder, tsQuery);
}
final TopNQuery topNQuery = queryBuilder.toTopNQuery(dataSource, plannerContext);
if (topNQuery != null) {
return executeTopN(queryBuilder, topNQuery);
}
final GroupByQuery groupByQuery = queryBuilder.toGroupByQuery(dataSource, plannerContext);
if (groupByQuery != null) {
return executeGroupBy(queryBuilder, groupByQuery);
}
final SelectQuery selectQuery = queryBuilder.toSelectQuery(dataSource, plannerContext);
if (selectQuery != null) {
return executeSelect(queryBuilder, selectQuery);
}
throw new IllegalStateException("WTF?! Cannot execute query even though we planned it?");
}
private Sequence<Object[]> executeSelect(

View File

@ -33,18 +33,22 @@ public class DruidFilterRule extends RelOptRule
super(operand(Filter.class, operand(DruidRel.class, none())));
}
@Override
public boolean matches(final RelOptRuleCall call)
{
final DruidRel druidRel = call.rel(1);
return druidRel.getQueryBuilder().getFilter() == null
&& druidRel.getQueryBuilder().getSelectProjection() == null
&& druidRel.getQueryBuilder().getGrouping() == null;
}
@Override
public void onMatch(RelOptRuleCall call)
{
final Filter filter = call.rel(0);
final DruidRel druidRel = call.rel(1);
if (druidRel.getQueryBuilder().getFilter() != null
|| druidRel.getQueryBuilder().getSelectProjection() != null
|| druidRel.getQueryBuilder().getGrouping() != null) {
return;
}
final DimFilter dimFilter = Expressions.toFilter(
druidRel.getPlannerContext(),
druidRel.getSourceRowSignature(),
@ -53,8 +57,7 @@ public class DruidFilterRule extends RelOptRule
if (dimFilter != null) {
call.transformTo(
druidRel.withQueryBuilder(
druidRel.getQueryBuilder()
.withFilter(dimFilter)
druidRel.getQueryBuilder().withFilter(dimFilter)
)
);
}

View File

@ -19,7 +19,6 @@
package io.druid.sql.calcite.rule;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.rel.DruidQueryRel;
import io.druid.sql.calcite.rel.QueryMaker;
import io.druid.sql.calcite.table.DruidTable;
@ -30,16 +29,11 @@ import org.apache.calcite.rel.logical.LogicalTableScan;
public class DruidTableScanRule extends RelOptRule
{
private final PlannerContext plannerContext;
private final QueryMaker queryMaker;
public DruidTableScanRule(
final PlannerContext plannerContext,
final QueryMaker queryMaker
)
public DruidTableScanRule(final QueryMaker queryMaker)
{
super(operand(LogicalTableScan.class, any()));
this.plannerContext = plannerContext;
this.queryMaker = queryMaker;
}
@ -51,7 +45,7 @@ public class DruidTableScanRule extends RelOptRule
final DruidTable druidTable = table.unwrap(DruidTable.class);
if (druidTable != null) {
call.transformTo(
DruidQueryRel.fullScan(scan.getCluster(), table, druidTable, plannerContext, queryMaker)
DruidQueryRel.fullScan(scan.getCluster(), table, druidTable, queryMaker)
);
}
}

View File

@ -61,7 +61,7 @@ import io.druid.sql.calcite.expression.SimpleExtraction;
import io.druid.sql.calcite.filtration.Filtration;
import io.druid.sql.calcite.planner.Calcites;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.rel.DruidNestedGroupBy;
import io.druid.sql.calcite.rel.DruidOuterQueryRel;
import io.druid.sql.calcite.rel.DruidRel;
import io.druid.sql.calcite.rel.Grouping;
import io.druid.sql.calcite.table.RowSignature;
@ -115,7 +115,7 @@ public class GroupByRules
{
private DruidAggregateRule()
{
super(operand(Aggregate.class, operand(DruidRel.class, none())));
super(operand(Aggregate.class, operand(DruidRel.class, any())));
}
@Override
@ -142,7 +142,7 @@ public class GroupByRules
{
private DruidAggregateProjectRule()
{
super(operand(Aggregate.class, operand(Project.class, operand(DruidRel.class, none()))));
super(operand(Aggregate.class, operand(Project.class, operand(DruidRel.class, any()))));
}
@Override
@ -171,7 +171,7 @@ public class GroupByRules
{
private DruidAggregateProjectFilterRule()
{
super(operand(Aggregate.class, operand(Project.class, operand(Filter.class, operand(DruidRel.class, none())))));
super(operand(Aggregate.class, operand(Project.class, operand(Filter.class, operand(DruidRel.class, any())))));
}
@Override
@ -207,7 +207,7 @@ public class GroupByRules
{
private DruidGroupByPostAggregationRule()
{
super(operand(Project.class, operand(DruidRel.class, none())));
super(operand(Project.class, operand(DruidRel.class, any())));
}
@Override
@ -233,7 +233,7 @@ public class GroupByRules
{
private DruidGroupByHavingRule()
{
super(operand(Filter.class, operand(DruidRel.class, none())));
super(operand(Filter.class, operand(DruidRel.class, any())));
}
@Override
@ -259,7 +259,7 @@ public class GroupByRules
{
private DruidGroupByLimitRule()
{
super(operand(Sort.class, operand(DruidRel.class, none())));
super(operand(Sort.class, operand(DruidRel.class, any())));
}
@Override
@ -443,7 +443,7 @@ public class GroupByRules
if (isNestedQuery) {
// Nested groupBy.
return DruidNestedGroupBy.from(druidRel, filter, grouping, aggregate.getRowType(), rowOrder);
return DruidOuterQueryRel.from(druidRel, filter, grouping, aggregate.getRowType(), rowOrder);
} else {
// groupBy on a base dataSource or semiJoin.
return druidRel.withQueryBuilder(
@ -586,8 +586,7 @@ public class GroupByRules
if (dimFilter != null) {
return druidRel.withQueryBuilder(
druidRel.getQueryBuilder()
.withHaving(dimFilter)
druidRel.getQueryBuilder().withHaving(dimFilter)
);
} else {
return null;

View File

@ -158,8 +158,7 @@ public class SelectRules
(orderBys.size() == 1 && orderBys.get(0).getDimension().equals(Column.TIME_COLUMN_NAME))) {
call.transformTo(
druidRel.withQueryBuilder(
druidRel.getQueryBuilder()
.withLimitSpec(limitSpec)
druidRel.getQueryBuilder().withLimitSpec(limitSpec)
)
);
}

View File

@ -126,7 +126,8 @@ public class DruidAvaticaHandlerTest
CalciteTests.createMockQueryLifecycleFactory(walker),
operatorTable,
macroTable,
plannerConfig
plannerConfig,
CalciteTests.getJsonMapper()
),
AVATICA_CONFIG
);
@ -257,7 +258,7 @@ public class DruidAvaticaHandlerTest
ImmutableList.of(
ImmutableMap.of(
"PLAN",
"DruidQueryRel(dataSource=[foo], dimensions=[[]], aggregations=[[Aggregation{virtualColumns=[], aggregatorFactories=[CountAggregatorFactory{name='a0'}], postAggregator=null}]])\n"
"DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"context\":{\"skipEmptyBuckets\":true}}])\n"
)
),
getRows(resultSet)
@ -565,7 +566,8 @@ public class DruidAvaticaHandlerTest
CalciteTests.createMockQueryLifecycleFactory(walker),
operatorTable,
macroTable,
plannerConfig
plannerConfig,
CalciteTests.getJsonMapper()
),
smallFrameConfig
)

View File

@ -70,7 +70,8 @@ public class DruidStatementTest
CalciteTests.createMockQueryLifecycleFactory(walker),
operatorTable,
macroTable,
plannerConfig
plannerConfig,
CalciteTests.getJsonMapper()
);
}

View File

@ -418,7 +418,7 @@ public class CalciteQueryTest
ImmutableList.of(),
ImmutableList.of(
new Object[]{
"DruidQueryRel(dataSource=[foo])\n"
"DruidQueryRel(query=[{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dummy\",\"outputName\":\"dummy\",\"outputType\":\"STRING\"}],\"metrics\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"virtualColumns\":[],\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":0,\"fromNext\":true},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807}}])\n"
}
)
);
@ -642,8 +642,8 @@ public class CalciteQueryTest
new Object[]{
"BindableProject(dim1=[$9], dim10=[$2], dim2=[$3])\n"
+ " BindableJoin(condition=[=($9, $3)], joinType=[inner])\n"
+ " DruidQueryRel(dataSource=[foo])\n"
+ " DruidQueryRel(dataSource=[foo], filter=[!dim1 = ])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dummy\",\"outputName\":\"dummy\",\"outputType\":\"STRING\"}],\"metrics\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"virtualColumns\":[],\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":0,\"fromNext\":true},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807}}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dummy\",\"outputName\":\"dummy\",\"outputType\":\"STRING\"}],\"metrics\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"virtualColumns\":[],\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":0,\"fromNext\":true},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807}}])\n"
}
)
);
@ -1602,11 +1602,17 @@ public class CalciteQueryTest
ImmutableList.of(),
ImmutableList.of(
new Object[]{
"DruidQueryRel(dataSource=[foo], "
+ "filter=[(dim2 = a && !substring(0, 1)(dim1) = z)], "
+ "dimensions=[[]], "
+ "aggregations=[[Aggregation{virtualColumns=[], "
+ "aggregatorFactories=[CountAggregatorFactory{name='a0'}], postAggregator=null}]])\n"
"DruidQueryRel(query=[{"
+ "\"queryType\":\"timeseries\","
+ "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},"
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
+ "\"descending\":false,"
+ "\"virtualColumns\":[],"
+ "\"filter\":{\"type\":\"and\",\"fields\":[{\"type\":\"selector\",\"dimension\":\"dim2\",\"value\":\"a\",\"extractionFn\":null},{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"z\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}}]},"
+ "\"granularity\":{\"type\":\"all\"},"
+ "\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],"
+ "\"postAggregations\":[],"
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"}}])\n"
}
)
);
@ -3082,8 +3088,8 @@ public class CalciteQueryTest
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(DIMS(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("dim1", "d1")
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("dim2", "d1")
))
.setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
@ -3092,12 +3098,12 @@ public class CalciteQueryTest
)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(DIMS(new DefaultDimensionSpec("d0", "d0")))
.setDimensions(DIMS(new DefaultDimensionSpec("d1", "d0")))
.setAggregatorSpecs(AGGS(
new LongSumAggregatorFactory("a0", "a0"),
new FilteredAggregatorFactory(
new CountAggregatorFactory("a1"),
NOT(SELECTOR("d1", "", null))
NOT(SELECTOR("d0", "", null))
)
))
.setContext(QUERY_CONTEXT_DEFAULT)
@ -3238,6 +3244,32 @@ public class CalciteQueryTest
);
}
@Test
public void testExplainDoubleNestedGroupBy() throws Exception
{
testQuery(
"EXPLAIN PLAN FOR SELECT SUM(cnt), COUNT(*) FROM (\n"
+ " SELECT dim2, SUM(t1.cnt) cnt FROM (\n"
+ " SELECT\n"
+ " dim1,\n"
+ " dim2,\n"
+ " COUNT(*) cnt\n"
+ " FROM druid.foo\n"
+ " GROUP BY dim1, dim2\n"
+ " ) t1\n"
+ " GROUP BY dim2\n"
+ ") t2",
ImmutableList.of(),
ImmutableList.of(
new Object[]{
"DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"longSum\",\"name\":\"a0\",\"fieldName\":\"a0\",\"expression\":null},{\"type\":\"count\",\"name\":\"a1\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n"
+ " DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"d1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[{\"type\":\"longSum\",\"name\":\"a0\",\"fieldName\":\"a0\",\"expression\":null}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"},{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d1\",\"outputType\":\"STRING\"}],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n"
}
)
);
}
@Test
public void testExactCountDistinctUsingSubquery() throws Exception
{
@ -3392,9 +3424,6 @@ public class CalciteQueryTest
@Test
public void testExactCountDistinctOfSemiJoinResult() throws Exception
{
// TODO(gianm): Would be good to have existing filters like IN able to work on expressions?
// TODO(gianm): Would be good to have expression filters optimize for the case where they hit just one column.
testQuery(
"SELECT COUNT(*)\n"
+ "FROM (\n"
@ -3449,6 +3478,29 @@ public class CalciteQueryTest
);
}
@Test
public void testExplainExactCountDistinctOfSemiJoinResult() throws Exception
{
testQuery(
"EXPLAIN PLAN FOR SELECT COUNT(*)\n"
+ "FROM (\n"
+ " SELECT DISTINCT dim2\n"
+ " FROM druid.foo\n"
+ " WHERE SUBSTRING(dim2, 1, 1) IN (\n"
+ " SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n"
+ " )\n"
+ ")",
ImmutableList.of(),
ImmutableList.of(
new Object[]{
"DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n"
+ " DruidSemiJoin(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}], leftExpressions=[[DruidExpression{simpleExtraction=null, expression='substring(\"dim2\", 0, 1)'}]], rightKeys=[[0]])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n"
}
)
);
}
@Test
public void testExactCountDistinctUsingSubqueryWithWherePushDown() throws Exception
{
@ -3596,16 +3648,13 @@ public class CalciteQueryTest
.setGranularity(Granularities.ALL)
.setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
.setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
.setPostAggregatorSpecs(ImmutableList.of(
EXPRESSION_POST_AGG("p0", "CAST(\"a0\", 'STRING')")
))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(DIMS(new DefaultDimensionSpec("p0", "d0")))
.setDimensions(DIMS(new DefaultDimensionSpec("a0", "d0")))
.setAggregatorSpecs(AGGS(
new CountAggregatorFactory("a0")
))
@ -5462,7 +5511,8 @@ public class CalciteQueryTest
CalciteTests.createMockQueryLifecycleFactory(walker),
operatorTable,
macroTable,
plannerConfig
plannerConfig,
CalciteTests.getJsonMapper()
);
viewManager.createView(

View File

@ -84,7 +84,8 @@ public class SqlResourceTest
CalciteTests.createMockQueryLifecycleFactory(walker),
operatorTable,
macroTable,
plannerConfig
plannerConfig,
CalciteTests.getJsonMapper()
)
);
}
@ -187,7 +188,7 @@ public class SqlResourceTest
ImmutableList.of(
ImmutableMap.<String, Object>of(
"PLAN",
"DruidQueryRel(dataSource=[foo], dimensions=[[]], aggregations=[[Aggregation{virtualColumns=[], aggregatorFactories=[CountAggregatorFactory{name='a0'}], postAggregator=null}]])\n"
"DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"context\":{\"skipEmptyBuckets\":true}}])\n"
)
),
rows

View File

@ -316,6 +316,11 @@ public class CalciteTests
);
}
public static ObjectMapper getJsonMapper()
{
return INJECTOR.getInstance(Key.get(ObjectMapper.class, Json.class));
}
public static SpecificSegmentsQuerySegmentWalker createMockWalker(final File tmpDir)
{
final QueryableIndex index1 = IndexBuilder.create()