diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java index 75887bcbec14..6f2a97b24823 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java @@ -20,10 +20,12 @@ package org.apache.druid.sql.calcite.planner; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.QueryContexts; import org.joda.time.DateTimeZone; +import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -428,4 +430,28 @@ public PlannerConfig build() return config; } } + + public Map getNonDefaultAsQueryContext() + { + Map overrides = new HashMap<>(); + PlannerConfig def = new PlannerConfig(); + if (def.useApproximateCountDistinct != useApproximateCountDistinct) { + overrides.put( + CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT, + String.valueOf(useApproximateCountDistinct) + ); + } + if (def.useGroupingSetForExactDistinct != useGroupingSetForExactDistinct) { + overrides.put( + CTX_KEY_USE_GROUPING_SET_FOR_EXACT_DISTINCT, + String.valueOf(useGroupingSetForExactDistinct) + ); + } + + PlannerConfig newConfig = PlannerConfig.builder().withOverrides(overrides).build(); + if (!equals(newConfig)) { + throw new IAE("Some configs are not handled in this method or not persistable as QueryContext keys!\nold: %s\nnew: %s", this, newConfig); + } + return overrides; + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java index 8a7bf4a6f742..9f15d3822866 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java @@ -588,7 +588,11 @@ protected PlannerResult planWithDruidConvention() throws ValidationException .plus(rootQueryRel.collation), parameterized ); + handlerContext.hook().captureDruidRel(druidRel); + + Hook.JAVA_PLAN.run(druidRel); + if (explain != null) { return planExplanation(possiblyLimitedRoot, druidRel, true); } else { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java index ac6c08547494..798b0bb407bd 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java @@ -19,14 +19,19 @@ package org.apache.druid.sql.calcite.rel; +import com.google.common.collect.Iterables; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.plan.volcano.RelSubset; import org.apache.calcite.rel.AbstractRelNode; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelShuttleImpl; import org.apache.calcite.rel.RelWriter; import org.apache.druid.server.QueryResponse; import org.apache.druid.sql.calcite.planner.PlannerContext; import javax.annotation.Nullable; + import java.util.Set; public abstract class DruidRel> extends AbstractRelNode @@ -122,4 +127,31 @@ protected Object clone() throws CloneNotSupportedException * Get the set of names of table datasources read by this DruidRel */ public abstract Set getDataSourceNames(); + + public final RelNode unwrapLogicalPlan() + { + return accept(new LogicalPlanUnwrapperShuttle()); + } + + private static class LogicalPlanUnwrapperShuttle extends RelShuttleImpl + { + @Override + public RelNode visit(RelNode other) + { + return super.visit(visitNode(other)); + } + + private RelNode visitNode(RelNode other) + { + if (other instanceof RelSubset) { + final RelSubset subset = (RelSubset) other; + return visitNode(Iterables.getFirst(subset.getRels(), null)); + } + if (other instanceof DruidRel) { + DruidRel druidRel = (DruidRel) other; + return druidRel.getPartialDruidQuery().leafRel(); + } + return other; + } + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidJoinRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidJoinRule.java index ded383cbefba..1a50ae1fbcd5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidJoinRule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidJoinRule.java @@ -54,7 +54,6 @@ public DruidJoinRule(Class clazz, RelTrait in, RelTrait out, // reject the query in case the anaysis detected any issues throw InvalidSqlInput.exception(analysis.errorStr); } - return new DruidJoin( join.getCluster(), newTrait, @@ -67,7 +66,7 @@ public DruidJoinRule(Class clazz, RelTrait in, RelTrait out, join.getRight(), DruidLogicalConvention.instance() ), - join.getCondition(), + analysis.getConditionWithUnsupportedSubConditionsIgnored(join.getCluster().getRexBuilder()), join.getVariablesSet(), join.getJoinType() ); diff --git a/sql/src/test/java/org/apache/druid/quidem/DruidConnectionExtras.java b/sql/src/test/java/org/apache/druid/quidem/DruidConnectionExtras.java index 75bdd4280fab..2b75f82122c1 100644 --- a/sql/src/test/java/org/apache/druid/quidem/DruidConnectionExtras.java +++ b/sql/src/test/java/org/apache/druid/quidem/DruidConnectionExtras.java @@ -25,7 +25,7 @@ public interface DruidConnectionExtras { ObjectMapper getObjectMapper(); - class DruidConnectionExtrasImpl implements DruidConnectionExtras + public class DruidConnectionExtrasImpl implements DruidConnectionExtras { private final ObjectMapper objectMapper; diff --git a/sql/src/test/java/org/apache/druid/quidem/DruidQTestInfo.java b/sql/src/test/java/org/apache/druid/quidem/DruidQTestInfo.java new file mode 100644 index 000000000000..9d655da3448a --- /dev/null +++ b/sql/src/test/java/org/apache/druid/quidem/DruidQTestInfo.java @@ -0,0 +1,41 @@ +/* + * 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.quidem; + +import java.io.File; + +public class DruidQTestInfo +{ + public final File caseDir; + public final String testName; + public final String comment; + + public DruidQTestInfo(File caseDir, String testName, String comment) + { + this.caseDir = caseDir; + this.testName = testName; + this.comment = comment; + } + + public File getIQFile() + { + return new File(caseDir, testName + ".iq"); + } +} diff --git a/sql/src/test/java/org/apache/druid/quidem/DruidQuidemCommandHandler.java b/sql/src/test/java/org/apache/druid/quidem/DruidQuidemCommandHandler.java index f6577e3903fe..8c2b99d80b6b 100644 --- a/sql/src/test/java/org/apache/druid/quidem/DruidQuidemCommandHandler.java +++ b/sql/src/test/java/org/apache/druid/quidem/DruidQuidemCommandHandler.java @@ -33,6 +33,7 @@ import org.apache.calcite.util.Util; import org.apache.druid.query.Query; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; +import org.apache.druid.sql.calcite.rel.DruidRel; import org.apache.druid.sql.calcite.util.QueryLogHook; import java.sql.ResultSet; @@ -171,6 +172,9 @@ protected final void executeExplain(Context x) } for (RelNode node : logged) { + if (node instanceof DruidRel) { + node = ((DruidRel) node).unwrapLogicalPlan(); + } String str = RelOptUtil.dumpPlan("", node, SqlExplainFormat.TEXT, SqlExplainLevel.EXPPLAN_ATTRIBUTES); x.echo(ImmutableList.of(str)); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 94d2249add3c..b991e8c230ea 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -106,6 +106,7 @@ import org.junit.Assert; import org.junit.internal.matchers.ThrowableMessageMatcher; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Named; import org.junit.jupiter.api.extension.RegisterExtension; import javax.annotation.Nullable; @@ -933,6 +934,12 @@ public Map baseQueryContext() { return baseQueryContext; } + + @Override + public SqlTestFramework queryFramework() + { + return BaseCalciteQueryTest.this.queryFramework(); + } } public enum ResultMatchMode @@ -1273,51 +1280,51 @@ public static Object[] provideQueryContexts() { return new Object[] { // default behavior - QUERY_CONTEXT_DEFAULT, + Named.of("default", QUERY_CONTEXT_DEFAULT), // all rewrites enabled - new ImmutableMap.Builder() + Named.of("all_enabled", new ImmutableMap.Builder() .putAll(QUERY_CONTEXT_DEFAULT) .put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, true) .put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, true) .put(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, true) - .build(), + .build()), // filter-on-value-column rewrites disabled, everything else enabled - new ImmutableMap.Builder() + Named.of("filter-on-value-column_disabled", new ImmutableMap.Builder() .putAll(QUERY_CONTEXT_DEFAULT) .put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, false) .put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, true) .put(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, true) - .build(), + .build()), // filter rewrites fully disabled, join-to-filter enabled - new ImmutableMap.Builder() + Named.of("join-to-filter", new ImmutableMap.Builder() .putAll(QUERY_CONTEXT_DEFAULT) .put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, false) .put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, false) .put(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, true) - .build(), + .build()), // filter rewrites disabled, but value column filters still set to true // (it should be ignored and this should // behave the same as the previous context) - new ImmutableMap.Builder() + Named.of("filter-rewrites-disabled", new ImmutableMap.Builder() .putAll(QUERY_CONTEXT_DEFAULT) .put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, true) .put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, false) .put(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, true) - .build(), + .build()), // filter rewrites fully enabled, join-to-filter disabled - new ImmutableMap.Builder() + Named.of("filter-rewrites", new ImmutableMap.Builder() .putAll(QUERY_CONTEXT_DEFAULT) .put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, true) .put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, true) .put(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, false) - .build(), + .build()), // all rewrites disabled - new ImmutableMap.Builder() + Named.of("all_disabled", new ImmutableMap.Builder() .putAll(QUERY_CONTEXT_DEFAULT) .put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, false) .put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, false) .put(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, false) - .build(), + .build()), }; } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index a822d8b8f101..f1ddeb9ccfbd 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -83,7 +83,7 @@ import org.apache.druid.segment.virtual.ListFilteredVirtualColumn; import org.apache.druid.server.QueryLifecycle; import org.apache.druid.server.security.Access; -import org.apache.druid.sql.calcite.DecoupledTestConfig.NativeQueryIgnore; +import org.apache.druid.sql.calcite.DecoupledTestConfig.QuidemTestCaseReason; import org.apache.druid.sql.calcite.NotYetSupported.Modes; import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.MinTopNThreshold; import org.apache.druid.sql.calcite.expression.DruidExpression; @@ -107,6 +107,7 @@ import java.util.Map; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assume.assumeFalse; import static org.junit.jupiter.api.Assertions.assertThrows; public class CalciteJoinQueryTest extends BaseCalciteQueryTest @@ -179,12 +180,11 @@ public void testInnerJoinWithLimitAndAlias() ); } - // Adjust topN threshold, so that the topN engine keeps only 1 slot for aggregates, which should be enough // to compute the query with limit 1. @SqlTestFrameworkConfig.MinTopNThreshold(1) @Test - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN, separateDefaultModeTest = true) public void testExactTopNOnInnerJoinWithLimit() { Map context = new HashMap<>(QUERY_CONTEXT_DEFAULT); @@ -492,7 +492,7 @@ public void testJoinWithLimitBeforeJoining() } @Test - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS, separateDefaultModeTest = true) public void testJoinOnTimeseriesWithFloorOnTime() { // Cannot vectorize JOIN operator. @@ -547,7 +547,7 @@ public void testJoinOnTimeseriesWithFloorOnTime() } @Test - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS, separateDefaultModeTest = true) public void testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime() { // Cannot vectorize JOIN operator. @@ -614,7 +614,7 @@ public void testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime() } @Test - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS, separateDefaultModeTest = true) public void testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues() { // Cannot vectorize JOIN operator. @@ -1529,7 +1529,7 @@ public void testManyManyInnerJoinOnManyManyLookup(Map queryConte ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.FINALIZING_FIELD_ACCESS) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.FINALIZING_FIELD_ACCESS) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinQueryOfLookup(Map queryContext) @@ -1609,7 +1609,7 @@ public void testTimeColumnAggregationsOnLookups(Map queryContext } } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.DEFINETLY_WORSE_PLAN) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.DEFINETLY_WORSE_PLAN) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinQueryOfLookupRemovable(Map queryContext) @@ -1648,7 +1648,6 @@ public void testInnerJoinQueryOfLookupRemovable(Map queryContext ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinTwoLookupsToTableUsingNumericColumn(Map queryContext) @@ -1976,7 +1975,7 @@ public void testWhereInSelectNullFromLookup() } @Test - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS) public void testCommaJoinLeftFunction() { testQuery( @@ -2151,7 +2150,7 @@ public void testJoinTableLookupTableMismatchedTypesWithoutComma(Map queryContext) @@ -2280,7 +2279,7 @@ public void testInnerJoinMismatchedTypes(Map queryContext) ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinLeftFunction(Map queryContext) @@ -2733,7 +2732,7 @@ public void testNotInAggregationSubquery(Map queryContext) ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testUsingSubqueryWithExtractionFns(Map queryContext) @@ -2937,7 +2936,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter(Map ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_LEFT_DIRECT_ACCESS) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_LEFT_DIRECT_ACCESS, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess(Map queryContext) @@ -3049,7 +3048,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithOuterWhere(Map ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_LEFT_DIRECT_ACCESS) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_LEFT_DIRECT_ACCESS, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess(Map queryContext) @@ -3151,7 +3150,7 @@ public void testLeftJoinOnTwoInlineDataSources(Map queryContext) ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_LEFT_DIRECT_ACCESS) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_LEFT_DIRECT_ACCESS, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess(Map queryContext) @@ -3253,7 +3252,7 @@ public void testInnerJoinOnTwoInlineDataSourcesWithOuterWhere(Map queryContext) @@ -3355,9 +3354,9 @@ public void testInnerJoinOnTwoInlineDataSources(Map queryContext ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN, separateDefaultModeTest = true) public void testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources(Map queryContext) { cannotVectorize(); @@ -3440,7 +3439,7 @@ public void testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources(Map queryContext) @@ -3591,6 +3590,10 @@ public void testLeftJoinSubqueryWithNullKeyFilter(Map queryConte // This test case was originally added in https://github.com/apache/druid/pull/11434 with a note about this. Assumptions.assumeFalse(NullHandling.replaceWithDefault() && QueryContext.of(queryContext).getEnableJoinFilterRewrite()); + assumeFalse( + "join condition not support in decoupled mode", testBuilder().isDecoupledMode() && NullHandling.replaceWithDefault() + ); + // Cannot vectorize due to 'concat' expression. cannotVectorize(); @@ -3670,9 +3673,9 @@ public void testLeftJoinSubqueryWithNullKeyFilter(Map queryConte ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN) public void testLeftJoinSubqueryWithSelectorFilter(Map queryContext) { // Cannot vectorize due to 'concat' expression. @@ -3876,7 +3879,6 @@ public void testJoinWithExplicitIsNotDistinctFromCondition(Map q ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinSubqueryWithSelectorFilter(Map queryContext) @@ -4152,7 +4154,6 @@ public void testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery(Map queryContext) @@ -5152,9 +5153,9 @@ public void testCountOnSemiJoinSingleColumn(Map queryContext) ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN) public void testTopNOnStringWithNonSortedOrUniqueDictionary(Map queryContext) { testQuery( @@ -5193,9 +5194,9 @@ public void testTopNOnStringWithNonSortedOrUniqueDictionary(Map ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN) public void testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim(Map queryContext) { @@ -5234,7 +5235,7 @@ public void testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim(Map queryContext) @@ -5291,7 +5292,7 @@ public void testVirtualColumnOnMVFilterJoinExpression(Map queryC ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.DEFINETLY_WORSE_PLAN) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.DEFINETLY_WORSE_PLAN) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testVirtualColumnOnMVFilterMultiJoinExpression(Map queryContext) @@ -5652,6 +5653,8 @@ public void testPlanWithInFilterMoreThanInSubQueryThreshold() @ParameterizedTest(name = "{0}") public void testRegressionFilteredAggregatorsSubqueryJoins(Map queryContext) { + assumeFalse("not support in decoupled mode", testBuilder().isDecoupledMode() && NullHandling.replaceWithDefault()); + cannotVectorize(); testQuery( "select\n" + @@ -6027,7 +6030,7 @@ public void testJoinsWithThreeConditions() } @Test - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS, separateDefaultModeTest = true) public void testJoinWithInputRefCondition() { cannotVectorize(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index b5a32f4301cb..33e2d3e00d07 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -112,7 +112,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.join.JoinType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; -import org.apache.druid.sql.calcite.DecoupledTestConfig.NativeQueryIgnore; +import org.apache.druid.sql.calcite.DecoupledTestConfig.QuidemTestCaseReason; import org.apache.druid.sql.calcite.NotYetSupported.Modes; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.filtration.Filtration; @@ -6955,7 +6955,7 @@ public void testApproxCountDistinctBuiltin() ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.AGG_COL_EXCHANGE) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.AGG_COL_EXCHANGE, separateDefaultModeTest = true) @Test public void testExactCountDistinctWithGroupingAndOtherAggregators() { @@ -7010,7 +7010,7 @@ public void testExactCountDistinctWithGroupingAndOtherAggregators() ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.AGG_COL_EXCHANGE) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.AGG_COL_EXCHANGE, separateDefaultModeTest = true) @Test public void testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin() { @@ -8209,7 +8209,7 @@ public void testRegexpLikeFilter() ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.AGG_COL_EXCHANGE) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.AGG_COL_EXCHANGE) @Test public void testGroupBySortPushDown() { @@ -8305,7 +8305,7 @@ public void testGroupByLimitPushDownWithHavingOnLong() ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.IMPROVED_PLAN) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IMPROVED_PLAN, separateDefaultModeTest = true) @Test public void testGroupByLimitPushdownExtraction() { @@ -8752,7 +8752,7 @@ public void testGroupByFloor() ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.SLIGHTLY_WORSE_PLAN) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_PLAN, separateDefaultModeTest = true) @SqlTestFrameworkConfig.NumMergeBuffers(3) @Test public void testQueryWithSelectProjectAndIdentityProjectDoesNotRename() @@ -10594,7 +10594,7 @@ public void testGroupByTimeAndOtherDimension() ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.IMPROVED_PLAN) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IMPROVED_PLAN, separateDefaultModeTest = true) @Test public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim() { @@ -12729,7 +12729,7 @@ public void testNvlColumns() ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.IMPROVED_PLAN) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IMPROVED_PLAN) @Test public void testGroupByWithLiteralInSubqueryGrouping() { @@ -12918,7 +12918,7 @@ public void testQueryContextOuterLimit() ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.IMPROVED_PLAN) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IMPROVED_PLAN) @Test public void testRepeatedIdenticalVirtualExpressionGrouping() { @@ -14493,7 +14493,7 @@ public void testGreatestFunctionForStringWithIsNull() ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.AGGREGATE_REMOVE_NOT_FIRED) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.AGGREGATE_REMOVE_NOT_FIRED, separateDefaultModeTest = true) @Test public void testSubqueryTypeMismatchWithLiterals() { @@ -15213,7 +15213,7 @@ public void testScanAndSortCanGetSchemaFromScanQuery() .run(); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.SLIGHTLY_WORSE_PLAN) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_PLAN, separateDefaultModeTest = true) @Test public void testWindowingWithScanAndSort() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledExtension.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledExtension.java index ec1e64df95c3..ec3d25632a63 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledExtension.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledExtension.java @@ -20,14 +20,20 @@ package org.apache.druid.sql.calcite; import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.query.QueryContexts; +import org.apache.druid.quidem.DruidQTestInfo; +import org.apache.druid.quidem.ProjectPathUtils; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.sql.calcite.BaseCalciteQueryTest.CalciteTestConfig; import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.util.SqlTestFramework; -import org.junit.jupiter.api.extension.Extension; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; -public class DecoupledExtension implements Extension +import java.io.File; + +public class DecoupledExtension implements BeforeEachCallback { private BaseCalciteQueryTest baseTest; @@ -36,6 +42,15 @@ public DecoupledExtension(BaseCalciteQueryTest baseTest) this.baseTest = baseTest; } + private File qCaseDir; + + @Override + public void beforeEach(ExtensionContext context) throws Exception + { + Class testClass = context.getTestClass().get(); + qCaseDir = ProjectPathUtils.getPathFromProjectRoot("sql/src/test/quidem/" + testClass.getName()); + } + private static final ImmutableMap CONTEXT_OVERRIDES = ImmutableMap.builder() .putAll(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT) .put(PlannerConfig.CTX_NATIVE_QUERY_SQL_PLANNING_MODE, PlannerConfig.NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED) @@ -47,8 +62,11 @@ public QueryTestBuilder testBuilder() DecoupledTestConfig decTestConfig = BaseCalciteQueryTest.queryFrameworkRule .getAnnotation(DecoupledTestConfig.class); + boolean runQuidem = (decTestConfig != null && decTestConfig.quidemReason().isPresent()); + CalciteTestConfig testConfig = baseTest.new CalciteTestConfig(CONTEXT_OVERRIDES) { + @Override public SqlTestFramework.PlannerFixture plannerFixture(PlannerConfig plannerConfig, AuthConfig authConfig) { @@ -56,16 +74,36 @@ public SqlTestFramework.PlannerFixture plannerFixture(PlannerConfig plannerConfi return baseTest.queryFramework().plannerFixture(plannerConfig, authConfig); } + + @Override + public DruidQTestInfo getQTestInfo() + { + if (runQuidem) { + final String testName; + if (decTestConfig.separateDefaultModeTest()) { + if (NullHandling.sqlCompatible()) { + testName = BaseCalciteQueryTest.queryFrameworkRule.testName() + "@NullHandling=sql"; + } else { + testName = BaseCalciteQueryTest.queryFrameworkRule.testName() + "@NullHandling=default"; + } + } else { + testName = BaseCalciteQueryTest.queryFrameworkRule.testName(); + } + return new DruidQTestInfo( + qCaseDir, + testName, + "quidem testcase reason: " + decTestConfig.quidemReason() + ); + } else { + return null; + } + } }; QueryTestBuilder builder = new QueryTestBuilder(testConfig) .cannotVectorize(baseTest.cannotVectorize) .skipVectorize(baseTest.skipVectorize); - if (decTestConfig != null && decTestConfig.nativeQueryIgnore().isPresent()) { - builder.verifyNativeQueries(x -> false); - } - return builder; } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteJoinQueryTest.java index dba11b35ffeb..d8abfbb20753 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteJoinQueryTest.java @@ -19,7 +19,6 @@ package org.apache.druid.sql.calcite; -import org.apache.druid.sql.calcite.DisableUnless.DisableUnlessRule; import org.apache.druid.sql.calcite.NotYetSupported.NotYetSupportedProcessor; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.extension.RegisterExtension; @@ -32,9 +31,6 @@ @ExtendWith(NotYetSupportedProcessor.class) public class DecoupledPlanningCalciteJoinQueryTest extends CalciteJoinQueryTest { - @RegisterExtension - public DisableUnlessRule sqlCompatOnly = DisableUnless.SQL_COMPATIBLE; - @RegisterExtension DecoupledExtension decoupledExtension = new DecoupledExtension(this); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java index 511db82b76b7..9bd57e10afb4 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java @@ -22,9 +22,6 @@ import org.apache.calcite.rel.rules.CoreRules; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; -import org.apache.druid.query.scan.ScanQuery; -import org.apache.druid.query.timeseries.TimeseriesQuery; - import java.lang.annotation.ElementType; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; @@ -42,9 +39,9 @@ * * The value of this field should describe the root cause of the difference. */ - NativeQueryIgnore nativeQueryIgnore() default NativeQueryIgnore.NONE; + QuidemTestCaseReason quidemReason() default QuidemTestCaseReason.NONE; - enum NativeQueryIgnore + enum QuidemTestCaseReason { NONE, /** @@ -68,16 +65,6 @@ enum NativeQueryIgnore * Worse plan; may loose vectorization; but no extra queries */ SLIGHTLY_WORSE_PLAN, - /** - * {@link TimeseriesQuery} to {@link ScanQuery} change. - * - * Not yet sure if this is improvement; or some issue - */ - TS_TO_SCAN, - /** - * GroupBy doesn't sorted?! - */ - GBY_DOESNT_SORT, /** * Equvivalent plan. * @@ -107,6 +94,7 @@ public boolean isPresent() { return this != NONE; } - }; + } + boolean separateDefaultModeTest() default false; } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/QTestCase.java b/sql/src/test/java/org/apache/druid/sql/calcite/QTestCase.java new file mode 100644 index 000000000000..888f8ecc573e --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/QTestCase.java @@ -0,0 +1,120 @@ +/* + * 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.hash.HashCode; +import com.google.common.hash.Hashing; +import com.google.common.io.Files; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.quidem.DruidQTestInfo; +import org.apache.druid.quidem.DruidQuidemTestBase; +import org.apache.druid.quidem.DruidQuidemTestBase.DruidQuidemRunner; +import org.apache.druid.sql.calcite.QueryTestRunner.QueryRunStep; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +public class QTestCase +{ + private StringBuffer sb; + private DruidQTestInfo testInfo; + + public QTestCase(DruidQTestInfo testInfo) + { + this.testInfo = testInfo; + sb = new StringBuffer(); + sb.append("# " + testInfo.comment); + sb.append("\n"); + } + + public void println(String str) + { + sb.append(str); + sb.append("\n"); + } + + public QueryRunStep toRunner() + { + return new QueryRunStep(null) + { + + @Override + public void run() + { + try { + if (DruidQuidemRunner.isOverwrite()) { + writeCaseTo(testInfo.getIQFile()); + } else { + isValidTestCaseFile(testInfo.getIQFile()); + } + + DruidQuidemRunner runner = new DruidQuidemTestBase.DruidQuidemRunner(); + runner.run(testInfo.getIQFile()); + } + catch (Exception e) { + throw new RuntimeException("Error running quidem test", e); + } + } + }; + } + + protected void isValidTestCaseFile(File iqFile) + { + if (!iqFile.exists()) { + throw new IllegalStateException("testcase doesn't exists; run with (-Dquidem.overwrite) : " + iqFile); + } + try { + String header = makeHeader(); + String testCaseFirstLine = Files.asCharSource(iqFile, StandardCharsets.UTF_8).readFirstLine(); + if (!header.equals(testCaseFirstLine)) { + throw new IllegalStateException( + "backing quidem testcase doesn't match test - run with (-Dquidem.overwrite) : " + iqFile + ); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + private String makeHeader() + { + HashCode hash = Hashing.crc32().hashBytes(sb.toString().getBytes(StandardCharsets.UTF_8)); + return StringUtils.format("# %s case-crc:%s", testInfo.testName, hash); + + } + + public void writeCaseTo(File file) throws IOException + { + FileUtils.mkdirp(file.getParentFile()); + try (FileOutputStream fos = new FileOutputStream(file)) { + fos.write(makeHeader().getBytes(StandardCharsets.UTF_8)); + fos.write('\n'); + fos.write(sb.toString().getBytes(StandardCharsets.UTF_8)); + } + catch (IOException e) { + throw new RuntimeException("Error writing testcase to: " + file, e); + } + } + +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestBuilder.java b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestBuilder.java index ff7e9b5a6bac..9484fd3d892c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestBuilder.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestBuilder.java @@ -23,6 +23,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; +import org.apache.druid.quidem.DruidQTestInfo; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; @@ -34,6 +35,7 @@ import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.calcite.util.SqlTestFramework; import org.apache.druid.sql.calcite.util.SqlTestFramework.PlannerFixture; import org.apache.druid.sql.http.SqlParameter; import java.util.ArrayList; @@ -75,6 +77,13 @@ public interface QueryTestConfig boolean isRunningMSQ(); Map baseQueryContext(); + + default DruidQTestInfo getQTestInfo() + { + return null; + } + + SqlTestFramework queryFramework(); } protected final QueryTestConfig config; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java index df14aeb105a0..1a5830b05d31 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java @@ -34,6 +34,7 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; +import org.apache.druid.quidem.DruidQTestInfo; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.sql.DirectStatement; @@ -56,6 +57,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -637,6 +639,33 @@ public void verify() public QueryTestRunner(QueryTestBuilder builder) { QueryTestConfig config = builder.config; + DruidQTestInfo iqTestInfo = config.getQTestInfo(); + if (iqTestInfo != null) { + QTestCase qt = new QTestCase(iqTestInfo); + Map queryContext = builder.getQueryContext(); + for (Entry entry : queryContext.entrySet()) { + qt.println(StringUtils.format("!set %s %s", entry.getKey(), entry.getValue())); + } + Map queryContext1 = builder.plannerConfig.getNonDefaultAsQueryContext(); + for (Entry entry : queryContext1.entrySet()) { + qt.println(StringUtils.format("!set %s %s", entry.getKey(), entry.getValue())); + } + + qt.println("!set outputformat mysql"); + qt.println("!use " + builder.config.queryFramework().getDruidTestURI()); + + qt.println(builder.sql + ";"); + if (builder.expectedResults != null) { + qt.println("!ok"); + } + qt.println("!logicalPlan"); + qt.println("!druidPlan"); + if (builder.expectedQueries != null) { + qt.println("!nativePlan"); + } + runSteps.add(qt.toRunner()); + return; + } if (builder.expectedResultsVerifier == null && builder.expectedResults != null) { builder.expectedResultsVerifier = config.defaultResultsVerifier( builder.expectedResults, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java index d320f4985a8d..ee75e73d52c3 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java @@ -19,10 +19,13 @@ package org.apache.druid.sql.calcite; +import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import com.google.common.collect.ImmutableSet; +import org.apache.curator.shaded.com.google.common.collect.Sets; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.topn.TopNQueryConfig; @@ -30,6 +33,7 @@ import org.apache.druid.sql.calcite.util.SqlTestFramework; import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplier; import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSupplier; +import org.apache.http.client.utils.URIBuilder; import org.junit.jupiter.api.extension.AfterAllCallback; import org.junit.jupiter.api.extension.BeforeEachCallback; import org.junit.jupiter.api.extension.ExtensionContext; @@ -45,14 +49,19 @@ import java.lang.annotation.Target; import java.lang.reflect.Constructor; import java.lang.reflect.Method; +import java.net.URI; +import java.net.URISyntaxException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Objects; import java.util.Set; import java.util.function.Function; +import java.util.regex.Pattern; import java.util.stream.Collectors; /** @@ -146,6 +155,13 @@ public Class fromString(String name) throws Ex Class value(); } + private static final Set KNOWN_CONFIG_KEYS = ImmutableSet.builder() + .add(NumMergeBuffers.class.getSimpleName()) + .add(MinTopNThreshold.class.getSimpleName()) + .add(ResultCache.class.getSimpleName()) + .add(ComponentSupplier.class.getSimpleName()) + .build(); + public final int numMergeBuffers; public final int minTopNThreshold; public final ResultCacheMode resultCache; @@ -166,6 +182,7 @@ public SqlTestFrameworkConfig(List annotations) public SqlTestFrameworkConfig(Map queryParams) { + validateConfigKeys(queryParams.keySet()); try { numMergeBuffers = NumMergeBuffers.PROCESSOR.fromMap(queryParams); minTopNThreshold = MinTopNThreshold.PROCESSOR.fromMap(queryParams); @@ -177,6 +194,15 @@ public SqlTestFrameworkConfig(Map queryParams) } } + private void validateConfigKeys(Set keySet) + { + Set diff = Sets.difference(keySet, KNOWN_CONFIG_KEYS); + if (diff.isEmpty()) { + return; + } + throw new IAE("Invalid configuration key(s) specified [%s]; valid options are [%s]", diff, KNOWN_CONFIG_KEYS); + } + @Override public int hashCode() { @@ -244,6 +270,7 @@ public static class Rule implements AfterAllCallback, BeforeEachCallback SqlTestFrameworkConfigStore configStore = new SqlTestFrameworkConfigStore(); private SqlTestFrameworkConfig config; private Method method; + private String testName; @Override public void afterAll(ExtensionContext context) @@ -259,12 +286,31 @@ public void beforeEach(ExtensionContext context) private void setConfig(ExtensionContext context) { + testName = buildTestCaseName(context); method = context.getTestMethod().get(); Class testClass = context.getTestClass().get(); List annotations = collectAnnotations(testClass, method); config = new SqlTestFrameworkConfig(annotations); } + /** + * Returns a string identifying the testcase. + * + * + */ + public String buildTestCaseName(ExtensionContext context) + { + List names = new ArrayList(); + Pattern pattern = Pattern.compile("\\([^\\)]*\\)"); + // this will add all name pieces - except the "last" which would be the + // Class level name + do { + names.add(0, pattern.matcher(context.getDisplayName()).replaceAll("")); + context = context.getParent().get(); + } while (context.getTestMethod().isPresent()); + return Joiner.on("@").join(names); + } + public SqlTestFrameworkConfig getConfig() { return config; @@ -282,7 +328,7 @@ public T getAnnotation(Class annotationType) public String testName() { - return method.getName(); + return testName; } } @@ -292,8 +338,8 @@ public static class ConfigurationInstance ConfigurationInstance(SqlTestFrameworkConfig config, QueryComponentSupplier testHost) { - SqlTestFramework.Builder builder = new SqlTestFramework.Builder(testHost) + .withConfig(config) .catalogResolver(testHost.createCatalogResolver()) .minTopNThreshold(config.minTopNThreshold) .mergeBufferCount(config.numMergeBuffers) @@ -321,6 +367,38 @@ public void close() } } + public URI getDruidTestURI() + { + try { + Map params = getNonDefaultMap(); + URIBuilder ub = new URIBuilder("druidtest:///"); + for (Entry entry : params.entrySet()) { + ub.setParameter(entry.getKey(), entry.getValue()); + } + ub.setPath("///"); + return ub.build(); + } + catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + + private Map getNonDefaultMap() + { + Map map = new HashMap<>(); + SqlTestFrameworkConfig def = new SqlTestFrameworkConfig(Collections.emptyList()); + if (def.numMergeBuffers != numMergeBuffers) { + map.put("NumMergeBuffers", String.valueOf(numMergeBuffers)); + } + if (def.minTopNThreshold != minTopNThreshold) { + map.put("MinTopNThreshold", String.valueOf(minTopNThreshold)); + } + if (!equals(new SqlTestFrameworkConfig(map))) { + throw new IAE("Can't reproduce config via map!"); + } + return map; + } + abstract static class ConfigOptionProcessor { final Class annotationClass; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfigTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfigTest.java index 7618ccf67ea4..175dbb1649d4 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfigTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfigTest.java @@ -19,7 +19,9 @@ package org.apache.druid.sql.calcite; +import com.google.common.collect.ImmutableMap; import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.MinTopNThreshold; import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.NumMergeBuffers; import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.ResultCache; @@ -30,6 +32,7 @@ import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; public class SqlTestFrameworkConfigTest { @@ -118,4 +121,21 @@ public void testAnnotationsAtTestLevel() throws Exception assertEquals(1, config.numMergeBuffers); assertEquals(ResultCacheMode.DISABLED, config.resultCache); } + + @Test + public void testInvalidConfigKeySpecified() + { + ImmutableMap configMap = ImmutableMap.builder() + .put("nonExistent", "someValue") + .build(); + IAE e = assertThrows( + IAE.class, + () -> new SqlTestFrameworkConfig(configMap) + ); + assertEquals( + "Invalid configuration key(s) specified [[nonExistent]]; valid options are [[NumMergeBuffers, MinTopNThreshold, ResultCache, ComponentSupplier]]", + e.getMessage() + ); + } + } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java index f0d6431615db..c64888414b62 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java @@ -50,6 +50,7 @@ import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.calcite.SqlTestFrameworkConfig; import org.apache.druid.sql.calcite.TempDirProducer; import org.apache.druid.sql.calcite.aggregation.SqlAggregationModule; import org.apache.druid.sql.calcite.planner.CalciteRulesManager; @@ -70,6 +71,7 @@ import java.io.Closeable; import java.io.IOException; +import java.net.URI; import java.util.ArrayList; import java.util.List; import java.util.Properties; @@ -403,6 +405,7 @@ public static class Builder private int mergeBufferCount; private CatalogResolver catalogResolver = CatalogResolver.NULL_RESOLVER; private List overrideModules = new ArrayList<>(); + private SqlTestFrameworkConfig config; public Builder(QueryComponentSupplier componentSupplier) { @@ -437,6 +440,12 @@ public SqlTestFramework build() { return new SqlTestFramework(this); } + + public Builder withConfig(SqlTestFrameworkConfig config) + { + this.config = config; + return this; + } } /** @@ -692,4 +701,9 @@ public void close() throw new RE(e); } } + + public URI getDruidTestURI() + { + return builder.config.getDruidTestURI(); + } } diff --git a/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/decoupled.iq b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/decoupled.iq index be52c7c4c65b..049d8d9d4d7e 100644 --- a/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/decoupled.iq +++ b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/decoupled.iq @@ -1,5 +1,5 @@ !set plannerStrategy DECOUPLED -!use druidtest://?numMergeBuffers=3 +!use druidtest://?NumMergeBuffers=3 !set outputformat mysql select cityName, count(case when delta > 0 then channel end) as cnt, count(1) as aall diff --git a/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq index e1ae27eef0d2..b48f587db246 100644 --- a/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq +++ b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq @@ -1,4 +1,4 @@ -!use druidtest://?numMergeBuffers=3 +!use druidtest://?NumMergeBuffers=3 !set outputformat mysql with v as ( diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinLeftFunction.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinLeftFunction.iq new file mode 100644 index 000000000000..19a189db3708 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinLeftFunction.iq @@ -0,0 +1,90 @@ +# testCommaJoinLeftFunction case-crc:30bb2791 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT foo.dim1, foo.dim2, l.k, l.v +FROM foo, lookup.lookyloo l +WHERE SUBSTRING(foo.dim2, 1, 1) = l.k +; ++------+------+---+----+ +| dim1 | dim2 | k | v | ++------+------+---+----+ +| | a | a | xa | +| 1 | a | a | xa | +| def | abc | a | xa | ++------+------+---+----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2], $f2=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], $f2=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testExactTopNOnInnerJoinWithLimit@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testExactTopNOnInnerJoinWithLimit@NullHandling=default.iq new file mode 100644 index 000000000000..a784c1fd3d03 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testExactTopNOnInnerJoinWithLimit@NullHandling=default.iq @@ -0,0 +1,115 @@ +# testExactTopNOnInnerJoinWithLimit@NullHandling=default case-crc:c3f897f8 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set useApproximateTopN false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:///?MinTopNThreshold=1 +select f1."dim4", sum("m1") from numfoo f1 inner join ( + select "dim4" from numfoo where dim4 <> 'a' group by 1 +) f2 on f1."dim4" = f2."dim4" group by 1 limit 1; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| b | 15.0 | ++------+--------+ +(1 row) + +!ok +LogicalSort(fetch=[1]) + LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim4=[$4], m1=[$14]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalAggregate(group=[{4}]) + LogicalFilter(condition=[<>($4, 'a')]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidSort(fetch=[1], druid=[logical]) + DruidAggregate(group=[{0}], EXPR$1=[SUM($1)], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim4=[$4], m1=[$14], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidAggregate(group=[{4}], druid=[logical]) + DruidFilter(condition=[<>($4, 'a')]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim4", + "value" : "a" + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim4", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0._d0\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "dimension", + "ordering" : { + "type" : "lexicographic" + } + }, + "threshold" : 1, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "doubleSum", + "name" : "a0", + "fieldName" : "m1" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..aa340691cc2c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default.iq @@ -0,0 +1,163 @@ +# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default case-crc:7916e17e +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1 +; ++------+ +| dim1 | ++------+ +| 10.1 | ++------+ +(1 row) + +!ok +LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalAggregate(group=[{1}]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidAggregate(group=[{1}], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "__time", + "outputName" : "d0", + "outputType" : "LONG" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default.iq new file mode 100644 index 000000000000..6c43fc799d24 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default.iq @@ -0,0 +1,163 @@ +# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default case-crc:a79ba14e +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1 +; ++------+ +| dim1 | ++------+ +| 10.1 | ++------+ +(1 row) + +!ok +LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalAggregate(group=[{1}]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidAggregate(group=[{1}], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "__time", + "outputName" : "d0", + "outputType" : "LONG" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default.iq new file mode 100644 index 000000000000..276e69eb7c1d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default.iq @@ -0,0 +1,160 @@ +# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default case-crc:d168257e +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1 +; ++------+ +| dim1 | ++------+ +| 10.1 | ++------+ +(1 row) + +!ok +LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalAggregate(group=[{1}]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidAggregate(group=[{1}], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "__time", + "outputName" : "d0", + "outputType" : "LONG" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..6e3914f40ffe --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,163 @@ +# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default case-crc:dc96fa7f +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1 +; ++------+ +| dim1 | ++------+ +| 10.1 | ++------+ +(1 row) + +!ok +LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalAggregate(group=[{1}]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidAggregate(group=[{1}], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "__time", + "outputName" : "d0", + "outputType" : "LONG" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 000000000000..f478bc1b7006 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,163 @@ +# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default case-crc:7936703f +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1 +; ++------+ +| dim1 | ++------+ +| 10.1 | ++------+ +(1 row) + +!ok +LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalAggregate(group=[{1}]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidAggregate(group=[{1}], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "__time", + "outputName" : "d0", + "outputType" : "LONG" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default.iq new file mode 100644 index 000000000000..f053d1aaf892 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,163 @@ +# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default case-crc:b5081d1c +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1 +; ++------+ +| dim1 | ++------+ +| 10.1 | ++------+ +(1 row) + +!ok +LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalAggregate(group=[{1}]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidAggregate(group=[{1}], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "__time", + "outputName" : "d0", + "outputType" : "LONG" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default.iq new file mode 100644 index 000000000000..968c7766fbd8 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default.iq @@ -0,0 +1,163 @@ +# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default case-crc:c0c596f8 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1 +; ++------+ +| dim1 | ++------+ +| 10.1 | ++------+ +(1 row) + +!ok +LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalAggregate(group=[{1}]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidAggregate(group=[{1}], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "__time", + "outputName" : "d0", + "outputType" : "LONG" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..8a2cf66e10cf --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=default.iq @@ -0,0 +1,90 @@ +# testInnerJoinCastLeft@all_disabled@NullHandling=default case-crc:532afc53 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT foo.m1, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k +; ++----+---+---+ +| m1 | k | v | ++----+---+---+ ++----+---+---+ +(0 rows) + +!ok +LogicalProject(m1=[$0], k=[$2], v=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"m1\", 'STRING')", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "m1", "v0" ], + "legacy" : false, + "columnTypes" : [ "FLOAT", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.k", "j0.v", "m1" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=default.iq new file mode 100644 index 000000000000..1bd3e860324a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=default.iq @@ -0,0 +1,90 @@ +# testInnerJoinCastLeft@all_enabled@NullHandling=default case-crc:615b6635 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT foo.m1, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k +; ++----+---+---+ +| m1 | k | v | ++----+---+---+ ++----+---+---+ +(0 rows) + +!ok +LogicalProject(m1=[$0], k=[$2], v=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"m1\", 'STRING')", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "m1", "v0" ], + "legacy" : false, + "columnTypes" : [ "FLOAT", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.k", "j0.v", "m1" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=default.iq new file mode 100644 index 000000000000..c91afac87da7 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=default.iq @@ -0,0 +1,87 @@ +# testInnerJoinCastLeft@default@NullHandling=default case-crc:fda58dbf +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +SELECT foo.m1, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k +; ++----+---+---+ +| m1 | k | v | ++----+---+---+ ++----+---+---+ +(0 rows) + +!ok +LogicalProject(m1=[$0], k=[$2], v=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"m1\", 'STRING')", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "m1", "v0" ], + "legacy" : false, + "columnTypes" : [ "FLOAT", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.k", "j0.v", "m1" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..d03641f46e32 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,90 @@ +# testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default case-crc:951d4374 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT foo.m1, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k +; ++----+---+---+ +| m1 | k | v | ++----+---+---+ ++----+---+---+ +(0 rows) + +!ok +LogicalProject(m1=[$0], k=[$2], v=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"m1\", 'STRING')", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "m1", "v0" ], + "legacy" : false, + "columnTypes" : [ "FLOAT", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.k", "j0.v", "m1" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 000000000000..39346aec7909 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,90 @@ +# testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default case-crc:403f05e1 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT foo.m1, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k +; ++----+---+---+ +| m1 | k | v | ++----+---+---+ ++----+---+---+ +(0 rows) + +!ok +LogicalProject(m1=[$0], k=[$2], v=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"m1\", 'STRING')", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "m1", "v0" ], + "legacy" : false, + "columnTypes" : [ "FLOAT", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.k", "j0.v", "m1" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=default.iq new file mode 100644 index 000000000000..74fb0a1e3f51 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,90 @@ +# testInnerJoinCastLeft@filter-rewrites@NullHandling=default case-crc:9b73e9cb +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT foo.m1, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k +; ++----+---+---+ +| m1 | k | v | ++----+---+---+ ++----+---+---+ +(0 rows) + +!ok +LogicalProject(m1=[$0], k=[$2], v=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"m1\", 'STRING')", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "m1", "v0" ], + "legacy" : false, + "columnTypes" : [ "FLOAT", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.k", "j0.v", "m1" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=default.iq new file mode 100644 index 000000000000..4cd32a7d9361 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=default.iq @@ -0,0 +1,90 @@ +# testInnerJoinCastLeft@join-to-filter@NullHandling=default case-crc:032c945b +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT foo.m1, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k +; ++----+---+---+ +| m1 | k | v | ++----+---+---+ ++----+---+---+ +(0 rows) + +!ok +LogicalProject(m1=[$0], k=[$2], v=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"m1\", 'STRING')", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "m1", "v0" ], + "legacy" : false, + "columnTypes" : [ "FLOAT", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.k", "j0.v", "m1" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_disabled.iq new file mode 100644 index 000000000000..fbdc5c4f6153 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_disabled.iq @@ -0,0 +1,93 @@ +# testInnerJoinLeftFunction@all_disabled case-crc:635a6dda +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT foo.dim1, foo.dim2, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k +; ++------+------+---+----+ +| dim1 | dim2 | k | v | ++------+------+---+----+ +| | a | a | xa | +| 1 | a | a | xa | +| def | abc | a | xa | ++------+------+---+----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_enabled.iq new file mode 100644 index 000000000000..15c92578854c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_enabled.iq @@ -0,0 +1,93 @@ +# testInnerJoinLeftFunction@all_enabled case-crc:7d493ec9 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT foo.dim1, foo.dim2, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k +; ++------+------+---+----+ +| dim1 | dim2 | k | v | ++------+------+---+----+ +| | a | a | xa | +| 1 | a | a | xa | +| def | abc | a | xa | ++------+------+---+----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@default.iq new file mode 100644 index 000000000000..a50455563bcb --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@default.iq @@ -0,0 +1,90 @@ +# testInnerJoinLeftFunction@default case-crc:00943b4a +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +SELECT foo.dim1, foo.dim2, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k +; ++------+------+---+----+ +| dim1 | dim2 | k | v | ++------+------+---+----+ +| | a | a | xa | +| 1 | a | a | xa | +| def | abc | a | xa | ++------+------+---+----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-on-value-column_disabled.iq new file mode 100644 index 000000000000..d4873e34d87f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-on-value-column_disabled.iq @@ -0,0 +1,93 @@ +# testInnerJoinLeftFunction@filter-on-value-column_disabled case-crc:70beb233 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT foo.dim1, foo.dim2, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k +; ++------+------+---+----+ +| dim1 | dim2 | k | v | ++------+------+---+----+ +| | a | a | xa | +| 1 | a | a | xa | +| def | abc | a | xa | ++------+------+---+----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites-disabled.iq new file mode 100644 index 000000000000..af34a46f2d65 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites-disabled.iq @@ -0,0 +1,93 @@ +# testInnerJoinLeftFunction@filter-rewrites-disabled case-crc:da4555b7 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT foo.dim1, foo.dim2, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k +; ++------+------+---+----+ +| dim1 | dim2 | k | v | ++------+------+---+----+ +| | a | a | xa | +| 1 | a | a | xa | +| def | abc | a | xa | ++------+------+---+----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites.iq new file mode 100644 index 000000000000..03fd1fea1489 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites.iq @@ -0,0 +1,93 @@ +# testInnerJoinLeftFunction@filter-rewrites case-crc:22eb8e52 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT foo.dim1, foo.dim2, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k +; ++------+------+---+----+ +| dim1 | dim2 | k | v | ++------+------+---+----+ +| | a | a | xa | +| 1 | a | a | xa | +| def | abc | a | xa | ++------+------+---+----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@join-to-filter.iq new file mode 100644 index 000000000000..f4791a842f69 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@join-to-filter.iq @@ -0,0 +1,93 @@ +# testInnerJoinLeftFunction@join-to-filter case-crc:90b51ec9 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT foo.dim1, foo.dim2, l.k, l.v +FROM foo +INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k +; ++------+------+---+----+ +| dim1 | dim2 | k | v | ++------+------+---+----+ +| | a | a | xa | +| 1 | a | a | xa | +| def | abc | a | xa | ++------+------+---+----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..bfe4a888f9ed --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default case-crc:31aaba25 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq new file mode 100644 index 000000000000..f05512baf05f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default case-crc:22a7ed95 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq new file mode 100644 index 000000000000..cf4014bd8d58 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq @@ -0,0 +1,130 @@ +# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default case-crc:fe77c11d +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..91309ae20b7b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default case-crc:c115dcb0 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 000000000000..e3f639e243a1 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default case-crc:466c2dda +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq new file mode 100644 index 000000000000..b3c0ab2c0f9b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default case-crc:8b226734 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq new file mode 100644 index 000000000000..892aa79987d6 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default case-crc:36ee0be3 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..c2010de170dc --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default case-crc:0c7abe39 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq new file mode 100644 index 000000000000..8e3ac0a20c62 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default case-crc:e167e948 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq new file mode 100644 index 000000000000..9a984f60c234 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq @@ -0,0 +1,130 @@ +# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default case-crc:ca2c56e9 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..eeb19b299173 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default case-crc:012eaa97 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 000000000000..943df21f27d0 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default case-crc:39af5216 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq new file mode 100644 index 000000000000..0108161cc01a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default case-crc:ec74956b +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq new file mode 100644 index 000000000000..4069023b634e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default case-crc:080e87b6 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq new file mode 100644 index 000000000000..dcd8f81055f7 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq @@ -0,0 +1,113 @@ +# testInnerJoinQueryOfLookup@all_disabled case-crc:1357fef3 +# quidem testcase reason: FINALIZING_FIELD_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.v, t1.v +FROM foo +INNER JOIN + (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 + ON foo.dim2 = t1.k; ++------+------+------+------+ +| dim1 | dim2 | v | v | ++------+------+------+------+ +| | a | xabc | xabc | +| 1 | a | xabc | xabc | ++------+------+------+------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) + LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) + DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "k", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "aggregations" : [ { + "type" : "stringAny", + "name" : "a0:a", + "fieldName" : "v", + "maxStringBytes" : 10, + "aggregateMultipleValues" : true + } ], + "postAggregations" : [ { + "type" : "finalizingFieldAccess", + "name" : "a0", + "fieldName" : "a0:a" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.a0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq new file mode 100644 index 000000000000..227cd5975ed9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq @@ -0,0 +1,113 @@ +# testInnerJoinQueryOfLookup@all_enabled case-crc:33c35499 +# quidem testcase reason: FINALIZING_FIELD_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.v, t1.v +FROM foo +INNER JOIN + (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 + ON foo.dim2 = t1.k; ++------+------+------+------+ +| dim1 | dim2 | v | v | ++------+------+------+------+ +| | a | xabc | xabc | +| 1 | a | xabc | xabc | ++------+------+------+------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) + LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) + DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "k", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "aggregations" : [ { + "type" : "stringAny", + "name" : "a0:a", + "fieldName" : "v", + "maxStringBytes" : 10, + "aggregateMultipleValues" : true + } ], + "postAggregations" : [ { + "type" : "finalizingFieldAccess", + "name" : "a0", + "fieldName" : "a0:a" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.a0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq new file mode 100644 index 000000000000..cdf3d768f81b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq @@ -0,0 +1,110 @@ +# testInnerJoinQueryOfLookup@default case-crc:fdc22455 +# quidem testcase reason: FINALIZING_FIELD_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.v, t1.v +FROM foo +INNER JOIN + (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 + ON foo.dim2 = t1.k; ++------+------+------+------+ +| dim1 | dim2 | v | v | ++------+------+------+------+ +| | a | xabc | xabc | +| 1 | a | xabc | xabc | ++------+------+------+------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) + LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) + DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "k", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "aggregations" : [ { + "type" : "stringAny", + "name" : "a0:a", + "fieldName" : "v", + "maxStringBytes" : 10, + "aggregateMultipleValues" : true + } ], + "postAggregations" : [ { + "type" : "finalizingFieldAccess", + "name" : "a0", + "fieldName" : "a0:a" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.a0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq new file mode 100644 index 000000000000..ba934275555a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq @@ -0,0 +1,113 @@ +# testInnerJoinQueryOfLookup@filter-on-value-column_disabled case-crc:223bb3ea +# quidem testcase reason: FINALIZING_FIELD_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.v, t1.v +FROM foo +INNER JOIN + (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 + ON foo.dim2 = t1.k; ++------+------+------+------+ +| dim1 | dim2 | v | v | ++------+------+------+------+ +| | a | xabc | xabc | +| 1 | a | xabc | xabc | ++------+------+------+------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) + LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) + DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "k", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "aggregations" : [ { + "type" : "stringAny", + "name" : "a0:a", + "fieldName" : "v", + "maxStringBytes" : 10, + "aggregateMultipleValues" : true + } ], + "postAggregations" : [ { + "type" : "finalizingFieldAccess", + "name" : "a0", + "fieldName" : "a0:a" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.a0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq new file mode 100644 index 000000000000..ce93b8d50fe3 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq @@ -0,0 +1,113 @@ +# testInnerJoinQueryOfLookup@filter-rewrites-disabled case-crc:90a7efad +# quidem testcase reason: FINALIZING_FIELD_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.v, t1.v +FROM foo +INNER JOIN + (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 + ON foo.dim2 = t1.k; ++------+------+------+------+ +| dim1 | dim2 | v | v | ++------+------+------+------+ +| | a | xabc | xabc | +| 1 | a | xabc | xabc | ++------+------+------+------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) + LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) + DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "k", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "aggregations" : [ { + "type" : "stringAny", + "name" : "a0:a", + "fieldName" : "v", + "maxStringBytes" : 10, + "aggregateMultipleValues" : true + } ], + "postAggregations" : [ { + "type" : "finalizingFieldAccess", + "name" : "a0", + "fieldName" : "a0:a" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.a0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq new file mode 100644 index 000000000000..3fde71a60692 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq @@ -0,0 +1,113 @@ +# testInnerJoinQueryOfLookup@filter-rewrites case-crc:41899e3c +# quidem testcase reason: FINALIZING_FIELD_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.v, t1.v +FROM foo +INNER JOIN + (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 + ON foo.dim2 = t1.k; ++------+------+------+------+ +| dim1 | dim2 | v | v | ++------+------+------+------+ +| | a | xabc | xabc | +| 1 | a | xabc | xabc | ++------+------+------+------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) + LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) + DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "k", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "aggregations" : [ { + "type" : "stringAny", + "name" : "a0:a", + "fieldName" : "v", + "maxStringBytes" : 10, + "aggregateMultipleValues" : true + } ], + "postAggregations" : [ { + "type" : "finalizingFieldAccess", + "name" : "a0", + "fieldName" : "a0:a" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.a0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq new file mode 100644 index 000000000000..9c7807a4f67e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq @@ -0,0 +1,113 @@ +# testInnerJoinQueryOfLookup@join-to-filter case-crc:9a602cc7 +# quidem testcase reason: FINALIZING_FIELD_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.v, t1.v +FROM foo +INNER JOIN + (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 + ON foo.dim2 = t1.k; ++------+------+------+------+ +| dim1 | dim2 | v | v | ++------+------+------+------+ +| | a | xabc | xabc | +| 1 | a | xabc | xabc | ++------+------+------+------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) + LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) + DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "k", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "aggregations" : [ { + "type" : "stringAny", + "name" : "a0:a", + "fieldName" : "v", + "maxStringBytes" : 10, + "aggregateMultipleValues" : true + } ], + "postAggregations" : [ { + "type" : "finalizingFieldAccess", + "name" : "a0", + "fieldName" : "a0:a" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.a0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_disabled.iq new file mode 100644 index 000000000000..a1153a61a6c9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_disabled.iq @@ -0,0 +1,96 @@ +# testInnerJoinQueryOfLookupRemovable@all_disabled case-crc:25b83d15 +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.sk +FROM foo +INNER JOIN + (SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1 + ON foo.dim2 = t1.k; ++------+------+-----+ +| dim1 | dim2 | sk | ++------+------+-----+ +| | a | xa | +| 1 | a | xa | +| def | abc | xab | ++------+------+-----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], sk=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"v\", 0, 3)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_enabled.iq new file mode 100644 index 000000000000..e5a886338fad --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_enabled.iq @@ -0,0 +1,96 @@ +# testInnerJoinQueryOfLookupRemovable@all_enabled case-crc:c65c79d6 +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.sk +FROM foo +INNER JOIN + (SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1 + ON foo.dim2 = t1.k; ++------+------+-----+ +| dim1 | dim2 | sk | ++------+------+-----+ +| | a | xa | +| 1 | a | xa | +| def | abc | xab | ++------+------+-----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], sk=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"v\", 0, 3)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@default.iq new file mode 100644 index 000000000000..c2df57120deb --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@default.iq @@ -0,0 +1,93 @@ +# testInnerJoinQueryOfLookupRemovable@default case-crc:c025c7ff +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.sk +FROM foo +INNER JOIN + (SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1 + ON foo.dim2 = t1.k; ++------+------+-----+ +| dim1 | dim2 | sk | ++------+------+-----+ +| | a | xa | +| 1 | a | xa | +| def | abc | xab | ++------+------+-----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], sk=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"v\", 0, 3)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled.iq new file mode 100644 index 000000000000..3712e2f8bfeb --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled.iq @@ -0,0 +1,96 @@ +# testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled case-crc:f30ce07a +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.sk +FROM foo +INNER JOIN + (SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1 + ON foo.dim2 = t1.k; ++------+------+-----+ +| dim1 | dim2 | sk | ++------+------+-----+ +| | a | xa | +| 1 | a | xa | +| def | abc | xab | ++------+------+-----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], sk=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"v\", 0, 3)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled.iq new file mode 100644 index 000000000000..1dfd6567a787 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled.iq @@ -0,0 +1,96 @@ +# testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled case-crc:5385c408 +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.sk +FROM foo +INNER JOIN + (SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1 + ON foo.dim2 = t1.k; ++------+------+-----+ +| dim1 | dim2 | sk | ++------+------+-----+ +| | a | xa | +| 1 | a | xa | +| def | abc | xab | ++------+------+-----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], sk=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"v\", 0, 3)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites.iq new file mode 100644 index 000000000000..dcc17c4ec78a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites.iq @@ -0,0 +1,96 @@ +# testInnerJoinQueryOfLookupRemovable@filter-rewrites case-crc:a5134bce +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.sk +FROM foo +INNER JOIN + (SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1 + ON foo.dim2 = t1.k; ++------+------+-----+ +| dim1 | dim2 | sk | ++------+------+-----+ +| | a | xa | +| 1 | a | xa | +| def | abc | xab | ++------+------+-----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], sk=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"v\", 0, 3)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@join-to-filter.iq new file mode 100644 index 000000000000..908086df3c8e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@join-to-filter.iq @@ -0,0 +1,96 @@ +# testInnerJoinQueryOfLookupRemovable@join-to-filter case-crc:20d8db5e +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, t1.sk +FROM foo +INNER JOIN + (SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1 + ON foo.dim2 = t1.k; ++------+------+-----+ +| dim1 | dim2 | sk | ++------+------+-----+ +| | a | xa | +| 1 | a | xa | +| def | abc | xab | ++------+------+-----+ +(3 rows) + +!ok +LogicalProject(dim1=[$0], dim2=[$1], sk=[$3]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"v\", 0, 3)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "dim2", "j0.v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=default.iq new file mode 100644 index 000000000000..016c05bcaf46 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=default.iq @@ -0,0 +1,155 @@ +# testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=default case-crc:a9e23ec5 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT CAST(__time AS BIGINT), m1, ANY_VALUE(dim3, 100) FROM foo WHERE (CAST(TIME_FLOOR(__time, 'PT1H') AS BIGINT) + 1, m1) IN + ( + SELECT CAST(TIME_FLOOR(__time, 'PT1H') AS BIGINT) + 1 AS t1, MIN(m1) AS t2 FROM foo WHERE dim3 = 'b' + AND __time BETWEEN '1994-04-29 00:00:00' AND '2020-01-11 00:00:00' GROUP BY 1 + ) +GROUP BY 1, 2 +; ++--------------+-----+--------+ +| EXPR$0 | m1 | EXPR$2 | ++--------------+-----+--------+ +| 946684800000 | 1.0 | [a, b] | +| 946771200000 | 2.0 | [b, c] | ++--------------+-----+--------+ +(2 rows) + +!ok +LogicalAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3)]) + LogicalProject(EXPR$0=[CAST($0):BIGINT NOT NULL], m1=[$2], dim3=[$1], $f3=[100]) + LogicalJoin(condition=[AND(=($3, $4), =($2, $5))], joinType=[inner]) + LogicalProject(__time=[$0], dim3=[$3], m1=[$5], $f3=[+(CAST(TIME_FLOOR($0, 'PT1H')):BIGINT NOT NULL, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}], t2=[MIN($1)]) + LogicalProject(t1=[+(CAST(TIME_FLOOR($0, 'PT1H')):BIGINT NOT NULL, 1)], m1=[$5]) + LogicalFilter(condition=[AND(=($3, 'b'), SEARCH($0, Sarg[[1994-04-29 00:00:00:TIMESTAMP(3)..2020-01-11 00:00:00:TIMESTAMP(3)]]:TIMESTAMP(3)))]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3)], druid=[logical]) + DruidProject(EXPR$0=[CAST($0):BIGINT NOT NULL], m1=[$2], dim3=[$1], $f3=[100], druid=[logical]) + DruidJoin(condition=[AND(=($3, $4), =($2, $5))], joinType=[inner]) + DruidProject(__time=[$0], dim3=[$3], m1=[$5], $f3=[+(CAST(TIME_FLOOR($0, 'PT1H')):BIGINT NOT NULL, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], t2=[MIN($1)], druid=[logical]) + DruidProject(t1=[+(CAST(TIME_FLOOR($0, 'PT1H')):BIGINT NOT NULL, 1)], m1=[$5], druid=[logical]) + DruidFilter(condition=[AND(=($3, 'b'), SEARCH($0, Sarg[[1994-04-29 00:00:00:TIMESTAMP(3)..2020-01-11 00:00:00:TIMESTAMP(3)]]:TIMESTAMP(3)))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "(timestamp_floor(\"__time\",'PT1H',null,'UTC') + 1)", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "dim3", "m1", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1994-04-29T00:00:00.000Z/2020-01-11T00:00:00.001Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "(timestamp_floor(\"__time\",'PT1H',null,'UTC') + 1)", + "outputType" : "LONG" + } ], + "filter" : { + "type" : "selector", + "dimension" : "dim3", + "value" : "b" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "LONG" + } ], + "aggregations" : [ { + "type" : "floatMin", + "name" : "a0", + "fieldName" : "m1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "((\"v0\" == \"j0.d0\") && (\"m1\" == \"j0.a0\"))", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "__time", + "outputName" : "d0", + "outputType" : "LONG" + }, { + "type" : "default", + "dimension" : "m1", + "outputName" : "d1", + "outputType" : "FLOAT" + } ], + "aggregations" : [ { + "type" : "stringAny", + "name" : "a0", + "fieldName" : "dim3", + "maxStringBytes" : 100, + "aggregateMultipleValues" : true + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=default.iq new file mode 100644 index 000000000000..90db281aa10f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=default.iq @@ -0,0 +1,155 @@ +# testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=default case-crc:b873fcbb +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT CAST(__time AS BIGINT), m1, ANY_VALUE(dim3, 100, false) FROM foo WHERE (CAST(TIME_FLOOR(__time, 'PT1H') AS BIGINT) + 1, m1) IN + ( + SELECT CAST(TIME_FLOOR(__time, 'PT1H') AS BIGINT) + 1 AS t1, MIN(m1) AS t2 FROM foo WHERE dim3 = 'b' + AND __time BETWEEN '1994-04-29 00:00:00' AND '2020-01-11 00:00:00' GROUP BY 1 + ) +GROUP BY 1, 2 +; ++--------------+-----+--------+ +| EXPR$0 | m1 | EXPR$2 | ++--------------+-----+--------+ +| 946684800000 | 1.0 | a | +| 946771200000 | 2.0 | b | ++--------------+-----+--------+ +(2 rows) + +!ok +LogicalAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3, $4)]) + LogicalProject(EXPR$0=[CAST($0):BIGINT NOT NULL], m1=[$2], dim3=[$1], $f3=[100], $f4=[false]) + LogicalJoin(condition=[AND(=($3, $4), =($2, $5))], joinType=[inner]) + LogicalProject(__time=[$0], dim3=[$3], m1=[$5], $f3=[+(CAST(TIME_FLOOR($0, 'PT1H')):BIGINT NOT NULL, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}], t2=[MIN($1)]) + LogicalProject(t1=[+(CAST(TIME_FLOOR($0, 'PT1H')):BIGINT NOT NULL, 1)], m1=[$5]) + LogicalFilter(condition=[AND(=($3, 'b'), SEARCH($0, Sarg[[1994-04-29 00:00:00:TIMESTAMP(3)..2020-01-11 00:00:00:TIMESTAMP(3)]]:TIMESTAMP(3)))]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3, $4)], druid=[logical]) + DruidProject(EXPR$0=[CAST($0):BIGINT NOT NULL], m1=[$2], dim3=[$1], $f3=[100], $f4=[false], druid=[logical]) + DruidJoin(condition=[AND(=($3, $4), =($2, $5))], joinType=[inner]) + DruidProject(__time=[$0], dim3=[$3], m1=[$5], $f3=[+(CAST(TIME_FLOOR($0, 'PT1H')):BIGINT NOT NULL, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], t2=[MIN($1)], druid=[logical]) + DruidProject(t1=[+(CAST(TIME_FLOOR($0, 'PT1H')):BIGINT NOT NULL, 1)], m1=[$5], druid=[logical]) + DruidFilter(condition=[AND(=($3, 'b'), SEARCH($0, Sarg[[1994-04-29 00:00:00:TIMESTAMP(3)..2020-01-11 00:00:00:TIMESTAMP(3)]]:TIMESTAMP(3)))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "(timestamp_floor(\"__time\",'PT1H',null,'UTC') + 1)", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "dim3", "m1", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1994-04-29T00:00:00.000Z/2020-01-11T00:00:00.001Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "(timestamp_floor(\"__time\",'PT1H',null,'UTC') + 1)", + "outputType" : "LONG" + } ], + "filter" : { + "type" : "selector", + "dimension" : "dim3", + "value" : "b" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "LONG" + } ], + "aggregations" : [ { + "type" : "floatMin", + "name" : "a0", + "fieldName" : "m1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "((\"v0\" == \"j0.d0\") && (\"m1\" == \"j0.a0\"))", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "__time", + "outputName" : "d0", + "outputType" : "LONG" + }, { + "type" : "default", + "dimension" : "m1", + "outputName" : "d1", + "outputType" : "FLOAT" + } ], + "aggregations" : [ { + "type" : "stringAny", + "name" : "a0", + "fieldName" : "dim3", + "maxStringBytes" : 100, + "aggregateMultipleValues" : false + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=default.iq new file mode 100644 index 000000000000..d5379c6190b3 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=default.iq @@ -0,0 +1,138 @@ +# testJoinOnTimeseriesWithFloorOnTime@NullHandling=default case-crc:76e87aaa +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT CAST(__time AS BIGINT), m1, ANY_VALUE(dim3, 100, true) FROM foo WHERE (TIME_FLOOR(__time, 'PT1H'), m1) IN + ( + SELECT TIME_FLOOR(__time, 'PT1H') AS t1, MIN(m1) AS t2 FROM foo WHERE dim3 = 'b' + AND __time BETWEEN '1994-04-29 00:00:00' AND '2020-01-11 00:00:00' GROUP BY 1 + ) +GROUP BY 1, 2 +; ++--------------+-----+--------+ +| EXPR$0 | m1 | EXPR$2 | ++--------------+-----+--------+ +| 946684800000 | 1.0 | [a, b] | +| 946771200000 | 2.0 | [b, c] | ++--------------+-----+--------+ +(2 rows) + +!ok +LogicalAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3, $4)]) + LogicalProject(EXPR$0=[CAST($0):BIGINT NOT NULL], m1=[$2], dim3=[$1], $f3=[100], $f4=[true]) + LogicalJoin(condition=[AND(=($3, $4), =($2, $5))], joinType=[inner]) + LogicalProject(__time=[$0], dim3=[$3], m1=[$5], $f3=[TIME_FLOOR($0, 'PT1H')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}], t2=[MIN($1)]) + LogicalProject(t1=[TIME_FLOOR($0, 'PT1H')], m1=[$5]) + LogicalFilter(condition=[AND(=($3, 'b'), SEARCH($0, Sarg[[1994-04-29 00:00:00:TIMESTAMP(3)..2020-01-11 00:00:00:TIMESTAMP(3)]]:TIMESTAMP(3)))]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3, $4)], druid=[logical]) + DruidProject(EXPR$0=[CAST($0):BIGINT NOT NULL], m1=[$2], dim3=[$1], $f3=[100], $f4=[true], druid=[logical]) + DruidJoin(condition=[AND(=($3, $4), =($2, $5))], joinType=[inner]) + DruidProject(__time=[$0], dim3=[$3], m1=[$5], $f3=[TIME_FLOOR($0, 'PT1H')], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], t2=[MIN($1)], druid=[logical]) + DruidProject(t1=[TIME_FLOOR($0, 'PT1H')], m1=[$5], druid=[logical]) + DruidFilter(condition=[AND(=($3, 'b'), SEARCH($0, Sarg[[1994-04-29 00:00:00:TIMESTAMP(3)..2020-01-11 00:00:00:TIMESTAMP(3)]]:TIMESTAMP(3)))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "timestamp_floor(\"__time\",'PT1H',null,'UTC')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "dim3", "m1", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1994-04-29T00:00:00.000Z/2020-01-11T00:00:00.001Z" ] + }, + "filter" : { + "type" : "selector", + "dimension" : "dim3", + "value" : "b" + }, + "granularity" : "HOUR", + "aggregations" : [ { + "type" : "floatMin", + "name" : "a0", + "fieldName" : "m1" + } ] + } + }, + "rightPrefix" : "j0.", + "condition" : "((\"v0\" == \"j0.d0\") && (\"m1\" == \"j0.a0\"))", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "__time", + "outputName" : "d0", + "outputType" : "LONG" + }, { + "type" : "default", + "dimension" : "m1", + "outputName" : "d1", + "outputType" : "FLOAT" + } ], + "aggregations" : [ { + "type" : "stringAny", + "name" : "a0", + "fieldName" : "dim3", + "maxStringBytes" : 100, + "aggregateMultipleValues" : true + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=default.iq new file mode 100644 index 000000000000..594acb5b77f5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=default.iq @@ -0,0 +1,126 @@ +# testJoinWithInputRefCondition@NullHandling=default case-crc:a8c0f240 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) FILTER (WHERE FLOOR(100) NOT IN (SELECT m1 FROM foo)) FROM foo; ++--------+ +| EXPR$0 | ++--------+ +| 6 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT() FILTER $0]) + LogicalProject($f0=[IS NULL($2)]) + LogicalJoin(condition=[=(CAST(FLOOR(100)):FLOAT NOT NULL, $1)], joinType=[left]) + LogicalProject(DUMMY=[0]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{5}], i=[LITERAL_AGG(true)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT() FILTER $0], druid=[logical]) + DruidProject($f0=[IS NULL($2)], druid=[logical]) + DruidJoin(condition=[=(CAST(FLOOR(100)):FLOAT NOT NULL, $1)], joinType=[left]) + DruidProject(DUMMY=[0], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{5}], i=[LITERAL_AGG(true)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "0", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "m1", + "outputName" : "d0", + "outputType" : "FLOAT" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "a0", + "expression" : "1", + "outputType" : "LONG" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(CAST(floor(100), 'DOUBLE') == \"j0.d0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "a0" + }, + "filter" : { + "type" : "selector", + "dimension" : "j0.a0", + "value" : null + }, + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..1149a9fc4daa --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default case-crc:5934672b +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq new file mode 100644 index 000000000000..9d204e2afb59 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default case-crc:2305f2ac +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq new file mode 100644 index 000000000000..ed2870a68fb9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq @@ -0,0 +1,130 @@ +# testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default case-crc:fb3f7888 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..674b934fe1f2 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default case-crc:914692f2 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 000000000000..707bfcb3dbc8 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default case-crc:8033cbb3 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq new file mode 100644 index 000000000000..421e08d29c1d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default case-crc:82d2e4f7 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq new file mode 100644 index 000000000000..11fa38a41524 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default case-crc:6381fa96 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..68149ec8f71e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -0,0 +1,139 @@ +# testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=default case-crc:5df88307 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' AND "__time" >= '1999' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=default.iq new file mode 100644 index 000000000000..db1534db083f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -0,0 +1,139 @@ +# testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=default case-crc:e1a970a4 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' AND "__time" >= '1999' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=default.iq new file mode 100644 index 000000000000..7845b7176b2f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=default.iq @@ -0,0 +1,136 @@ +# testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=default case-crc:235e0aba +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' AND "__time" >= '1999' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..446296eaf106 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,139 @@ +# testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default case-crc:46643c1d +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' AND "__time" >= '1999' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 000000000000..c59700399fdf --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,139 @@ +# testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default case-crc:fe4a2072 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' AND "__time" >= '1999' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq new file mode 100644 index 000000000000..b59bafc8b1fa --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,139 @@ +# testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=default case-crc:a5ec6a25 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' AND "__time" >= '1999' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=default.iq new file mode 100644 index 000000000000..dff78591ed13 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -0,0 +1,139 @@ +# testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=default case-crc:78302b3a +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' AND "__time" >= '1999' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1' +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR]) + LogicalFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical]) + DruidFilter(condition=[AND(=($1, '10.1'), >=($0, 1999-01-01 00:00:00))]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "1999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..5c59c87be249 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default case-crc:f40ac966 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq new file mode 100644 index 000000000000..1b77b8f5f771 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default case-crc:b06e5180 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq new file mode 100644 index 000000000000..7272c8143be5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq @@ -0,0 +1,130 @@ +# testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default case-crc:113b89ee +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..ee0d9adb8c2b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default case-crc:1c4df7a3 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 000000000000..a770dd20136a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default case-crc:c8b860e5 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq new file mode 100644 index 000000000000..a73e657f0a39 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default case-crc:47dc2525 +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq new file mode 100644 index 000000000000..310fcfb9ab20 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -0,0 +1,133 @@ +# testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default case-crc:28f1233a +# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableJoinLeftTableScanDirect true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +with abc as +( + SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1' +) +SELECT t1.dim1, t1."__time" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 +; ++------+---------------------+ +| dim1 | __time | ++------+---------------------+ +| 10.1 | 2000-01-02 00:00:00 | ++------+---------------------+ +(1 row) + +!ok +LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1]) + LogicalJoin(condition=[=($0, $2)], joinType=[left]) + LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[=($1, '10.1')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[left]) + DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[=($1, '10.1')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "__time", "v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "10.1" + }, + "columns" : [ "dim1" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.dim1\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "_v0", + "expression" : "'10.1'", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "__time", "_v0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq new file mode 100644 index 000000000000..0fdaa3152dd1 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq @@ -0,0 +1,100 @@ +# testLeftJoinSubqueryWithSelectorFilter@all_disabled case-crc:6ecea68d +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set computeInnerJoinCostAsFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k +FROM foo +LEFT JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k +WHERE l1.k = 'abc' +; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "concat(\"k\",'')", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.d0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled.iq new file mode 100644 index 000000000000..0df67a503b43 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled.iq @@ -0,0 +1,100 @@ +# testLeftJoinSubqueryWithSelectorFilter@all_enabled case-crc:1781f405 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set computeInnerJoinCostAsFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k +FROM foo +LEFT JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k +WHERE l1.k = 'abc' +; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "concat(\"k\",'')", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.d0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default.iq new file mode 100644 index 000000000000..bec4d44eeb22 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default.iq @@ -0,0 +1,97 @@ +# testLeftJoinSubqueryWithSelectorFilter@default case-crc:63cb4668 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set computeInnerJoinCostAsFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k +FROM foo +LEFT JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k +WHERE l1.k = 'abc' +; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "concat(\"k\",'')", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.d0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled.iq new file mode 100644 index 000000000000..f0e508b60ebd --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled.iq @@ -0,0 +1,100 @@ +# testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled case-crc:a3446cc7 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set computeInnerJoinCostAsFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k +FROM foo +LEFT JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k +WHERE l1.k = 'abc' +; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "concat(\"k\",'')", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.d0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled.iq new file mode 100644 index 000000000000..c5a872721836 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled.iq @@ -0,0 +1,100 @@ +# testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled case-crc:c34594f7 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set computeInnerJoinCostAsFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k +FROM foo +LEFT JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k +WHERE l1.k = 'abc' +; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "concat(\"k\",'')", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.d0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites.iq new file mode 100644 index 000000000000..52e0867d25c4 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites.iq @@ -0,0 +1,100 @@ +# testLeftJoinSubqueryWithSelectorFilter@filter-rewrites case-crc:cc6beb91 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set computeInnerJoinCostAsFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k +FROM foo +LEFT JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k +WHERE l1.k = 'abc' +; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "concat(\"k\",'')", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.d0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter.iq new file mode 100644 index 000000000000..f223813ea87b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter.iq @@ -0,0 +1,100 @@ +# testLeftJoinSubqueryWithSelectorFilter@join-to-filter case-crc:93aa5a3b +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set computeInnerJoinCostAsFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k +FROM foo +LEFT JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k +WHERE l1.k = 'abc' +; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "concat(\"k\",'')", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.d0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@all_disabled.iq new file mode 100644 index 000000000000..b6d4ee823411 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@all_disabled.iq @@ -0,0 +1,87 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionary@all_disabled case-crc:c879b2d7 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 2 LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| a | 9 | +| b | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "numeric", + "metric" : "a0" + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@all_enabled.iq new file mode 100644 index 000000000000..18a6f61af924 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@all_enabled.iq @@ -0,0 +1,87 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionary@all_enabled case-crc:9ce24a96 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 2 LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| a | 9 | +| b | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "numeric", + "metric" : "a0" + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@default.iq new file mode 100644 index 000000000000..c6ca4654cd03 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@default.iq @@ -0,0 +1,84 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionary@default case-crc:a6504917 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 2 LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| a | 9 | +| b | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "numeric", + "metric" : "a0" + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@filter-on-value-column_disabled.iq new file mode 100644 index 000000000000..2cea2b163451 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@filter-on-value-column_disabled.iq @@ -0,0 +1,87 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionary@filter-on-value-column_disabled case-crc:0a77a16f +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 2 LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| a | 9 | +| b | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "numeric", + "metric" : "a0" + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@filter-rewrites-disabled.iq new file mode 100644 index 000000000000..99ba2ecc6d6c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@filter-rewrites-disabled.iq @@ -0,0 +1,87 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionary@filter-rewrites-disabled case-crc:493a622a +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 2 LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| a | 9 | +| b | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "numeric", + "metric" : "a0" + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@filter-rewrites.iq new file mode 100644 index 000000000000..ff02227c2605 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@filter-rewrites.iq @@ -0,0 +1,87 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionary@filter-rewrites case-crc:62c4653d +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 2 LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| a | 9 | +| b | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "numeric", + "metric" : "a0" + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@join-to-filter.iq new file mode 100644 index 000000000000..ef541bc46f4e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionary@join-to-filter.iq @@ -0,0 +1,87 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionary@join-to-filter case-crc:ade7f733 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 2 LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| a | 9 | +| b | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "numeric", + "metric" : "a0" + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@all_disabled.iq new file mode 100644 index 000000000000..f5003b573a61 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@all_disabled.iq @@ -0,0 +1,89 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@all_disabled case-crc:7385b5ed +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 1 DESC LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| b | 9 | +| a | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$0], dir0=[DESC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$0], dir0=[DESC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "dimension", + "ordering" : { + "type" : "lexicographic" + } + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@all_enabled.iq new file mode 100644 index 000000000000..9493848bdfd1 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@all_enabled.iq @@ -0,0 +1,89 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@all_enabled case-crc:f1d6d218 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 1 DESC LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| b | 9 | +| a | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$0], dir0=[DESC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$0], dir0=[DESC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "dimension", + "ordering" : { + "type" : "lexicographic" + } + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@default.iq new file mode 100644 index 000000000000..bb7efc406924 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@default.iq @@ -0,0 +1,86 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@default case-crc:94e7e664 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 1 DESC LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| b | 9 | +| a | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$0], dir0=[DESC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$0], dir0=[DESC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "dimension", + "ordering" : { + "type" : "lexicographic" + } + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@filter-on-value-column_disabled.iq new file mode 100644 index 000000000000..97ca3bc0e06a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@filter-on-value-column_disabled.iq @@ -0,0 +1,89 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@filter-on-value-column_disabled case-crc:6270c3ac +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 1 DESC LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| b | 9 | +| a | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$0], dir0=[DESC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$0], dir0=[DESC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "dimension", + "ordering" : { + "type" : "lexicographic" + } + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@filter-rewrites-disabled.iq new file mode 100644 index 000000000000..dc582354c4c9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@filter-rewrites-disabled.iq @@ -0,0 +1,89 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@filter-rewrites-disabled case-crc:424109c4 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 1 DESC LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| b | 9 | +| a | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$0], dir0=[DESC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$0], dir0=[DESC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "dimension", + "ordering" : { + "type" : "lexicographic" + } + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@filter-rewrites.iq new file mode 100644 index 000000000000..29cdbfc32010 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@filter-rewrites.iq @@ -0,0 +1,89 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@filter-rewrites case-crc:d0ac6d14 +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 1 DESC LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| b | 9 | +| a | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$0], dir0=[DESC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$0], dir0=[DESC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "dimension", + "ordering" : { + "type" : "lexicographic" + } + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@join-to-filter.iq new file mode 100644 index 000000000000..3d953d2e325c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@join-to-filter.iq @@ -0,0 +1,89 @@ +# testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim@join-to-filter case-crc:1a92bbad +# quidem testcase reason: EQUIV_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT druid.broadcast.dim4, COUNT(*) +FROM druid.numfoo +INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4 +GROUP BY 1 ORDER BY 1 DESC LIMIT 4; ++------+--------+ +| dim4 | EXPR$1 | ++------+--------+ +| b | 9 | +| a | 9 | ++------+--------+ +(2 rows) + +!ok +LogicalSort(sort0=[$0], dir0=[DESC], fetch=[4]) + LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim4=[$4]) + LogicalTableScan(table=[[druid, broadcast]]) + +!logicalPlan +DruidSort(sort0=[$0], dir0=[DESC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim4=[$4], druid=[logical]) + DruidTableScan(table=[[druid, broadcast]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "numfoo" + }, + "right" : { + "type" : "globalTable", + "name" : "broadcast" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim4\" == \"j0.dim4\")", + "joinType" : "INNER" + }, + "dimension" : { + "type" : "default", + "dimension" : "j0.dim4", + "outputName" : "d0", + "outputType" : "STRING" + }, + "metric" : { + "type" : "inverted", + "metric" : { + "type" : "dimension", + "ordering" : { + "type" : "lexicographic" + } + } + }, + "threshold" : 4, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..6a8f185c2b65 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=default.iq @@ -0,0 +1,139 @@ +# testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=default case-crc:f2ca92ca +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT dim2, COUNT(*) FROM druid.foo WHERE substring(dim2, 1, 1) IN (SELECT substring(dim1, 1, 1) FROM druid.foo WHERE dim1 <> '')group by dim2; ++------+--------+ +| dim2 | EXPR$1 | ++------+--------+ +| a | 2 | +| abc | 1 | ++------+--------+ +(2 rows) + +!ok +LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(EXPR$0=[SUBSTRING($1, 1, 1)]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(EXPR$0=[SUBSTRING($1, 1, 1)], druid=[logical]) + DruidFilter(condition=[<>($1, '')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : null + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=default.iq new file mode 100644 index 000000000000..c17ac58287a1 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=default.iq @@ -0,0 +1,139 @@ +# testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=default case-crc:2261e315 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT dim2, COUNT(*) FROM druid.foo WHERE substring(dim2, 1, 1) IN (SELECT substring(dim1, 1, 1) FROM druid.foo WHERE dim1 <> '')group by dim2; ++------+--------+ +| dim2 | EXPR$1 | ++------+--------+ +| a | 2 | +| abc | 1 | ++------+--------+ +(2 rows) + +!ok +LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(EXPR$0=[SUBSTRING($1, 1, 1)]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(EXPR$0=[SUBSTRING($1, 1, 1)], druid=[logical]) + DruidFilter(condition=[<>($1, '')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : null + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=default.iq new file mode 100644 index 000000000000..6f276a7f689c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=default.iq @@ -0,0 +1,136 @@ +# testUsingSubqueryWithExtractionFns@default@NullHandling=default case-crc:e8b174d8 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +SELECT dim2, COUNT(*) FROM druid.foo WHERE substring(dim2, 1, 1) IN (SELECT substring(dim1, 1, 1) FROM druid.foo WHERE dim1 <> '')group by dim2; ++------+--------+ +| dim2 | EXPR$1 | ++------+--------+ +| a | 2 | +| abc | 1 | ++------+--------+ +(2 rows) + +!ok +LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(EXPR$0=[SUBSTRING($1, 1, 1)]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(EXPR$0=[SUBSTRING($1, 1, 1)], druid=[logical]) + DruidFilter(condition=[<>($1, '')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : null + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 000000000000..f1cd9c88b5e6 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,139 @@ +# testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=default case-crc:c71da62e +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT dim2, COUNT(*) FROM druid.foo WHERE substring(dim2, 1, 1) IN (SELECT substring(dim1, 1, 1) FROM druid.foo WHERE dim1 <> '')group by dim2; ++------+--------+ +| dim2 | EXPR$1 | ++------+--------+ +| a | 2 | +| abc | 1 | ++------+--------+ +(2 rows) + +!ok +LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(EXPR$0=[SUBSTRING($1, 1, 1)]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(EXPR$0=[SUBSTRING($1, 1, 1)], druid=[logical]) + DruidFilter(condition=[<>($1, '')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : null + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 000000000000..2d8c1e7202f2 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,139 @@ +# testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=default case-crc:ea2bb732 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT dim2, COUNT(*) FROM druid.foo WHERE substring(dim2, 1, 1) IN (SELECT substring(dim1, 1, 1) FROM druid.foo WHERE dim1 <> '')group by dim2; ++------+--------+ +| dim2 | EXPR$1 | ++------+--------+ +| a | 2 | +| abc | 1 | ++------+--------+ +(2 rows) + +!ok +LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(EXPR$0=[SUBSTRING($1, 1, 1)]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(EXPR$0=[SUBSTRING($1, 1, 1)], druid=[logical]) + DruidFilter(condition=[<>($1, '')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : null + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=default.iq new file mode 100644 index 000000000000..938e2c4a6772 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,139 @@ +# testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=default case-crc:7443f782 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT dim2, COUNT(*) FROM druid.foo WHERE substring(dim2, 1, 1) IN (SELECT substring(dim1, 1, 1) FROM druid.foo WHERE dim1 <> '')group by dim2; ++------+--------+ +| dim2 | EXPR$1 | ++------+--------+ +| a | 2 | +| abc | 1 | ++------+--------+ +(2 rows) + +!ok +LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(EXPR$0=[SUBSTRING($1, 1, 1)]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(EXPR$0=[SUBSTRING($1, 1, 1)], druid=[logical]) + DruidFilter(condition=[<>($1, '')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : null + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=default.iq new file mode 100644 index 000000000000..78aaa620f516 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=default.iq @@ -0,0 +1,139 @@ +# testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=default case-crc:6178ece2 +# quidem testcase reason: JOIN_FILTER_LOCATIONS +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT dim2, COUNT(*) FROM druid.foo WHERE substring(dim2, 1, 1) IN (SELECT substring(dim1, 1, 1) FROM druid.foo WHERE dim1 <> '')group by dim2; ++------+--------+ +| dim2 | EXPR$1 | ++------+--------+ +| a | 2 | +| abc | 1 | ++------+--------+ +(2 rows) + +!ok +LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}]) + LogicalProject(EXPR$0=[SUBSTRING($1, 1, 1)]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim2=[$2], $f1=[SUBSTRING($2, 1, 1)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(EXPR$0=[SUBSTRING($1, 1, 1)], druid=[logical]) + DruidFilter(condition=[<>($1, '')]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "substring(\"dim2\", 0, 1)", + "outputType" : "STRING" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim2", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : null + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "extraction", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_disabled.iq new file mode 100644 index 000000000000..bc97a36fb27a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_disabled.iq @@ -0,0 +1,126 @@ +# testVirtualColumnOnMVFilterJoinExpression@all_disabled case-crc:7a32e94a +# quidem testcase reason: SLIGHTLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN druid.numfoo as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim3", "j0.dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_enabled.iq new file mode 100644 index 000000000000..9c43e9625d7c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_enabled.iq @@ -0,0 +1,126 @@ +# testVirtualColumnOnMVFilterJoinExpression@all_enabled case-crc:91fa3751 +# quidem testcase reason: SLIGHTLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN druid.numfoo as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim3", "j0.dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@default.iq new file mode 100644 index 000000000000..6988e39d1e8f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@default.iq @@ -0,0 +1,123 @@ +# testVirtualColumnOnMVFilterJoinExpression@default case-crc:3387da5c +# quidem testcase reason: SLIGHTLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN druid.numfoo as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim3", "j0.dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled.iq new file mode 100644 index 000000000000..d6cc278554c2 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled.iq @@ -0,0 +1,126 @@ +# testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled case-crc:2116c5d2 +# quidem testcase reason: SLIGHTLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN druid.numfoo as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim3", "j0.dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled.iq new file mode 100644 index 000000000000..4a6831e10ea4 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled.iq @@ -0,0 +1,126 @@ +# testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled case-crc:93e6503a +# quidem testcase reason: SLIGHTLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN druid.numfoo as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim3", "j0.dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites.iq new file mode 100644 index 000000000000..8cb747f268c6 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites.iq @@ -0,0 +1,126 @@ +# testVirtualColumnOnMVFilterJoinExpression@filter-rewrites case-crc:f4ddf549 +# quidem testcase reason: SLIGHTLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN druid.numfoo as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim3", "j0.dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@join-to-filter.iq new file mode 100644 index 000000000000..c92c1686a148 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@join-to-filter.iq @@ -0,0 +1,126 @@ +# testVirtualColumnOnMVFilterJoinExpression@join-to-filter case-crc:f3a7b2f1 +# quidem testcase reason: SLIGHTLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN druid.numfoo as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim3", "j0.dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_disabled.iq new file mode 100644 index 000000000000..6115a839e4a6 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_disabled.iq @@ -0,0 +1,202 @@ +# testVirtualColumnOnMVFilterMultiJoinExpression@all_disabled case-crc:9c935b4c +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN (SELECT foo3.dim3 FROM druid.numfoo as foo3 INNER JOIN druid.numfoo as foo4 ON MV_FILTER_ONLY(foo3.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo4.dim3, ARRAY['a'])) as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "_v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "_v0", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0._v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "_j0.dim3", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_enabled.iq new file mode 100644 index 000000000000..1e4e0aa6acc6 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_enabled.iq @@ -0,0 +1,202 @@ +# testVirtualColumnOnMVFilterMultiJoinExpression@all_enabled case-crc:812de429 +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN (SELECT foo3.dim3 FROM druid.numfoo as foo3 INNER JOIN druid.numfoo as foo4 ON MV_FILTER_ONLY(foo3.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo4.dim3, ARRAY['a'])) as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "_v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "_v0", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0._v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "_j0.dim3", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@default.iq new file mode 100644 index 000000000000..2f175178b09c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@default.iq @@ -0,0 +1,199 @@ +# testVirtualColumnOnMVFilterMultiJoinExpression@default case-crc:a431c313 +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN (SELECT foo3.dim3 FROM druid.numfoo as foo3 INNER JOIN druid.numfoo as foo4 ON MV_FILTER_ONLY(foo3.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo4.dim3, ARRAY['a'])) as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "_v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "_v0", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0._v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "_j0.dim3", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-on-value-column_disabled.iq new file mode 100644 index 000000000000..f4295f4464ce --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-on-value-column_disabled.iq @@ -0,0 +1,202 @@ +# testVirtualColumnOnMVFilterMultiJoinExpression@filter-on-value-column_disabled case-crc:2429103a +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN (SELECT foo3.dim3 FROM druid.numfoo as foo3 INNER JOIN druid.numfoo as foo4 ON MV_FILTER_ONLY(foo3.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo4.dim3, ARRAY['a'])) as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "_v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "_v0", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0._v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "_j0.dim3", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites-disabled.iq new file mode 100644 index 000000000000..ec8a00bf4043 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites-disabled.iq @@ -0,0 +1,202 @@ +# testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites-disabled case-crc:5dbbd67c +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN (SELECT foo3.dim3 FROM druid.numfoo as foo3 INNER JOIN druid.numfoo as foo4 ON MV_FILTER_ONLY(foo3.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo4.dim3, ARRAY['a'])) as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "_v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "_v0", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0._v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "_j0.dim3", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites.iq new file mode 100644 index 000000000000..b4c1225b6adf --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites.iq @@ -0,0 +1,202 @@ +# testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites case-crc:0da69cb7 +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite true +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN (SELECT foo3.dim3 FROM druid.numfoo as foo3 INNER JOIN druid.numfoo as foo4 ON MV_FILTER_ONLY(foo3.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo4.dim3, ARRAY['a'])) as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "_v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "_v0", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0._v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "_j0.dim3", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@join-to-filter.iq new file mode 100644 index 000000000000..b8aa7edbf717 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@join-to-filter.iq @@ -0,0 +1,202 @@ +# testVirtualColumnOnMVFilterMultiJoinExpression@join-to-filter case-crc:3e1879da +# quidem testcase reason: DEFINETLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set maxScatterGatherBytes 9223372036854775807 +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set enableJoinFilterRewrite false +!set outputformat mysql +!use druidtest:/// +SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN (SELECT foo3.dim3 FROM druid.numfoo as foo3 INNER JOIN druid.numfoo as foo4 ON MV_FILTER_ONLY(foo3.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo4.dim3, ARRAY['a'])) as foo2 ON MV_FILTER_ONLY(foo1.dim3, ARRAY['a']) = MV_FILTER_ONLY(foo2.dim3, ARRAY['a']) +; ++-----------+-----------+ +| dim3 | dim3 | ++-----------+-----------+ +| ["a","b"] | ["a","b"] | ++-----------+-----------+ +(1 row) + +!ok +LogicalProject(dim3=[$0], dim30=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject(dim3=[$0], $f1=[MV_FILTER_ONLY($0, ARRAY('a'))], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim3=[$3], $f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidProject($f17=[MV_FILTER_ONLY($3, ARRAY('a'))], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim3", "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "legacy" : false, + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"v0\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "mv-filtered", + "name" : "_v0", + "delegate" : { + "type" : "default", + "dimension" : "dim3", + "outputName" : "dim3", + "outputType" : "STRING" + }, + "values" : [ "a" ], + "isAllowList" : true + } ], + "resultFormat" : "compactedList", + "columns" : [ "_v0", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0._v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "_j0.dim3", "dim3" ], + "legacy" : false, + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testExactCountDistinctWithGroupingAndOtherAggregators@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testExactCountDistinctWithGroupingAndOtherAggregators@NullHandling=default.iq new file mode 100644 index 000000000000..96e1908cbf2e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testExactCountDistinctWithGroupingAndOtherAggregators@NullHandling=default.iq @@ -0,0 +1,108 @@ +# testExactCountDistinctWithGroupingAndOtherAggregators@NullHandling=default case-crc:80416865 +# quidem testcase reason: AGG_COL_EXCHANGE +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set useApproximateCountDistinct false +!set outputformat mysql +!use druidtest:/// +SELECT dim2, SUM(cnt), COUNT(distinct dim1) FROM druid.foo GROUP BY dim2; ++------+--------+--------+ +| dim2 | EXPR$1 | EXPR$2 | ++------+--------+--------+ +| | 3 | 3 | +| a | 2 | 1 | +| abc | 1 | 1 | ++------+--------+--------+ +(3 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[SUM($2)], EXPR$2=[COUNT($0)]) + LogicalAggregate(group=[{1, 2}], EXPR$1=[SUM($4)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[SUM($2)], EXPR$2=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{1, 2}], EXPR$1=[SUM($4)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "longSum", + "name" : "a0", + "fieldName" : "cnt" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "d1", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "longSum", + "name" : "_a0", + "fieldName" : "a0" + }, { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a1" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "d0", + "value" : null + } + }, + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testExactCountDistinctWithGroupingAndOtherAggregators@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testExactCountDistinctWithGroupingAndOtherAggregators@NullHandling=sql.iq new file mode 100644 index 000000000000..591792ade346 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testExactCountDistinctWithGroupingAndOtherAggregators@NullHandling=sql.iq @@ -0,0 +1,108 @@ +# testExactCountDistinctWithGroupingAndOtherAggregators@NullHandling=sql case-crc:80416865 +# quidem testcase reason: AGG_COL_EXCHANGE +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set useApproximateCountDistinct false +!set outputformat mysql +!use druidtest:/// +SELECT dim2, SUM(cnt), COUNT(distinct dim1) FROM druid.foo GROUP BY dim2; ++------+--------+--------+ +| dim2 | EXPR$1 | EXPR$2 | ++------+--------+--------+ +| | 1 | 1 | +| a | 2 | 2 | +| abc | 1 | 1 | +| | 2 | 2 | ++------+--------+--------+ +(4 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[SUM($2)], EXPR$2=[COUNT($0)]) + LogicalAggregate(group=[{1, 2}], EXPR$1=[SUM($4)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[SUM($2)], EXPR$2=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{1, 2}], EXPR$1=[SUM($4)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "longSum", + "name" : "a0", + "fieldName" : "cnt" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "d1", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "longSum", + "name" : "_a0", + "fieldName" : "a0" + }, { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a1" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "null", + "column" : "d0" + } + }, + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByLimitPushdownExtraction@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByLimitPushdownExtraction@NullHandling=default.iq new file mode 100644 index 000000000000..4e6bbff9d57c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByLimitPushdownExtraction@NullHandling=default.iq @@ -0,0 +1,84 @@ +# testGroupByLimitPushdownExtraction@NullHandling=default case-crc:1bfa9337 +# quidem testcase reason: IMPROVED_PLAN +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT dim4, substring(dim5, 1, 1), count(*) FROM druid.numfoo WHERE dim4 = 'a' GROUP BY 1,2 LIMIT 2; ++------+--------+--------+ +| dim4 | EXPR$1 | EXPR$2 | ++------+--------+--------+ +| a | a | 2 | +| a | b | 1 | ++------+--------+--------+ +(2 rows) + +!ok +LogicalProject(dim4=[CAST('a':VARCHAR):VARCHAR], EXPR$1=[$0], EXPR$2=[$1]) + LogicalSort(fetch=[2]) + LogicalAggregate(group=[{0}], EXPR$2=[COUNT()]) + LogicalProject(EXPR$1=[SUBSTRING($5, 1, 1)]) + LogicalFilter(condition=[=($4, 'a')]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim4=[CAST('a':VARCHAR):VARCHAR], EXPR$1=[$0], EXPR$2=[$1], druid=[logical]) + DruidSort(fetch=[2], druid=[logical]) + DruidAggregate(group=[{0}], EXPR$2=[COUNT()], druid=[logical]) + DruidProject(EXPR$1=[SUBSTRING($5, 1, 1)], druid=[logical]) + DruidFilter(condition=[=($4, 'a')]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "dimension" : { + "type" : "extraction", + "dimension" : "dim5", + "outputName" : "_d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + }, + "metric" : { + "type" : "dimension", + "ordering" : { + "type" : "lexicographic" + } + }, + "threshold" : 2, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "selector", + "dimension" : "dim4", + "value" : "a" + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "s0", + "expression" : "'a'", + "outputType" : "STRING" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByLimitPushdownExtraction@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByLimitPushdownExtraction@NullHandling=sql.iq new file mode 100644 index 000000000000..e104bbb0cf5a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByLimitPushdownExtraction@NullHandling=sql.iq @@ -0,0 +1,85 @@ +# testGroupByLimitPushdownExtraction@NullHandling=sql case-crc:1bfa9337 +# quidem testcase reason: IMPROVED_PLAN +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT dim4, substring(dim5, 1, 1), count(*) FROM druid.numfoo WHERE dim4 = 'a' GROUP BY 1,2 LIMIT 2; ++------+--------+--------+ +| dim4 | EXPR$1 | EXPR$2 | ++------+--------+--------+ +| a | a | 2 | +| a | b | 1 | ++------+--------+--------+ +(2 rows) + +!ok +LogicalProject(dim4=[CAST('a':VARCHAR):VARCHAR], EXPR$1=[$0], EXPR$2=[$1]) + LogicalSort(fetch=[2]) + LogicalAggregate(group=[{0}], EXPR$2=[COUNT()]) + LogicalProject(EXPR$1=[SUBSTRING($5, 1, 1)]) + LogicalFilter(condition=[=($4, 'a')]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(dim4=[CAST('a':VARCHAR):VARCHAR], EXPR$1=[$0], EXPR$2=[$1], druid=[logical]) + DruidSort(fetch=[2], druid=[logical]) + DruidAggregate(group=[{0}], EXPR$2=[COUNT()], druid=[logical]) + DruidProject(EXPR$1=[SUBSTRING($5, 1, 1)], druid=[logical]) + DruidFilter(condition=[=($4, 'a')]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "topN", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "dimension" : { + "type" : "extraction", + "dimension" : "dim5", + "outputName" : "_d0", + "outputType" : "STRING", + "extractionFn" : { + "type" : "substring", + "index" : 0, + "length" : 1 + } + }, + "metric" : { + "type" : "dimension", + "ordering" : { + "type" : "lexicographic" + } + }, + "threshold" : 2, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "equals", + "column" : "dim4", + "matchValueType" : "STRING", + "matchValue" : "a" + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "s0", + "expression" : "'a'", + "outputType" : "STRING" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupBySortPushDown.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupBySortPushDown.iq new file mode 100644 index 000000000000..6b4a8021d71a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupBySortPushDown.iq @@ -0,0 +1,76 @@ +# testGroupBySortPushDown case-crc:5655a1ec +# quidem testcase reason: AGG_COL_EXCHANGE +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT dim2, dim1, SUM(cnt) FROM druid.foo GROUP BY dim2, dim1 ORDER BY dim1 LIMIT 4; ++------+------+--------+ +| dim2 | dim1 | EXPR$2 | ++------+------+--------+ +| a | | 1 | +| a | 1 | 1 | +| | 10.1 | 1 | +| | 2 | 1 | ++------+------+--------+ +(4 rows) + +!ok +LogicalProject(dim2=[$1], dim1=[$0], EXPR$2=[$2]) + LogicalSort(sort0=[$0], dir0=[ASC], fetch=[4]) + LogicalAggregate(group=[{1, 2}], EXPR$2=[SUM($4)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim2=[$1], dim1=[$0], EXPR$2=[$2], druid=[logical]) + DruidSort(sort0=[$0], dir0=[ASC], fetch=[4], druid=[logical]) + DruidAggregate(group=[{1, 2}], EXPR$2=[SUM($4)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "longSum", + "name" : "a0", + "fieldName" : "cnt" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d0", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ], + "limit" : 4 + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim@NullHandling=default.iq new file mode 100644 index 000000000000..384456a0dd06 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim@NullHandling=default.iq @@ -0,0 +1,147 @@ +# testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim@NullHandling=default case-crc:634b75d5 +# quidem testcase reason: IMPROVED_PLAN +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT dim2, time_floor(gran, 'P1M') gran, sum(s) +FROM (SELECT time_floor(__time, 'P1D') AS gran, dim2, sum(m1) as s FROM druid.foo GROUP BY 1, 2 HAVING sum(m1) > 1) AS x +GROUP BY 1, 2 +ORDER BY dim2, gran desc; ++------+---------------------+--------+ +| dim2 | gran | EXPR$2 | ++------+---------------------+--------+ +| | 2001-01-01 00:00:00 | 6.0 | +| | 2000-01-01 00:00:00 | 5.0 | +| a | 2001-01-01 00:00:00 | 4.0 | +| abc | 2001-01-01 00:00:00 | 5.0 | ++------+---------------------+--------+ +(4 rows) + +!ok +LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)]) + LogicalProject(dim2=[$1], gran=[TIME_FLOOR($0, 'P1M')], s=[$2]) + LogicalFilter(condition=[>($2, 1)]) + LogicalAggregate(group=[{0, 1}], s=[SUM($2)]) + LogicalProject(gran=[TIME_FLOOR($0, 'P1D')], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[SUM($2)], druid=[logical]) + DruidProject(dim2=[$1], gran=[TIME_FLOOR($0, 'P1M')], s=[$2], druid=[logical]) + DruidFilter(condition=[>($2, 1)]) + DruidAggregate(group=[{0, 1}], s=[SUM($2)], druid=[logical]) + DruidProject(gran=[TIME_FLOOR($0, 'P1D')], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "timestamp_floor(\"__time\",'P1D',null,'UTC')", + "outputType" : "LONG" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "LONG" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "doubleSum", + "name" : "a0", + "fieldName" : "m1" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "bound", + "dimension" : "a0", + "lower" : "1", + "lowerStrict" : true, + "ordering" : { + "type" : "numeric" + } + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "timestamp_floor(\"d0\",'P1M',null,'UTC')", + "outputType" : "LONG" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "d1", + "outputName" : "_d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "v0", + "outputName" : "_d1", + "outputType" : "LONG" + } ], + "aggregations" : [ { + "type" : "doubleSum", + "name" : "_a0", + "fieldName" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "_d0", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + }, { + "dimension" : "_d1", + "direction" : "descending", + "dimensionOrder" : { + "type" : "numeric" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim@NullHandling=sql.iq new file mode 100644 index 000000000000..aafc63709ec0 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim@NullHandling=sql.iq @@ -0,0 +1,146 @@ +# testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim@NullHandling=sql case-crc:634b75d5 +# quidem testcase reason: IMPROVED_PLAN +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT dim2, time_floor(gran, 'P1M') gran, sum(s) +FROM (SELECT time_floor(__time, 'P1D') AS gran, dim2, sum(m1) as s FROM druid.foo GROUP BY 1, 2 HAVING sum(m1) > 1) AS x +GROUP BY 1, 2 +ORDER BY dim2, gran desc; ++------+---------------------+--------+ +| dim2 | gran | EXPR$2 | ++------+---------------------+--------+ +| | 2001-01-01 00:00:00 | 6.0 | +| | 2000-01-01 00:00:00 | 2.0 | +| | 2000-01-01 00:00:00 | 3.0 | +| a | 2001-01-01 00:00:00 | 4.0 | +| abc | 2001-01-01 00:00:00 | 5.0 | ++------+---------------------+--------+ +(5 rows) + +!ok +LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[SUM($2)]) + LogicalProject(dim2=[$1], gran=[TIME_FLOOR($0, 'P1M')], s=[$2]) + LogicalFilter(condition=[>($2, 1)]) + LogicalAggregate(group=[{0, 1}], s=[SUM($2)]) + LogicalProject(gran=[TIME_FLOOR($0, 'P1D')], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[SUM($2)], druid=[logical]) + DruidProject(dim2=[$1], gran=[TIME_FLOOR($0, 'P1M')], s=[$2], druid=[logical]) + DruidFilter(condition=[>($2, 1)]) + DruidAggregate(group=[{0, 1}], s=[SUM($2)], druid=[logical]) + DruidProject(gran=[TIME_FLOOR($0, 'P1D')], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "timestamp_floor(\"__time\",'P1D',null,'UTC')", + "outputType" : "LONG" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "LONG" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "doubleSum", + "name" : "a0", + "fieldName" : "m1" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "range", + "column" : "a0", + "matchValueType" : "LONG", + "lower" : 1, + "lowerOpen" : true + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "timestamp_floor(\"d0\",'P1M',null,'UTC')", + "outputType" : "LONG" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "d1", + "outputName" : "_d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "v0", + "outputName" : "_d1", + "outputType" : "LONG" + } ], + "aggregations" : [ { + "type" : "doubleSum", + "name" : "_a0", + "fieldName" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "_d0", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + }, { + "dimension" : "_d1", + "direction" : "descending", + "dimensionOrder" : { + "type" : "numeric" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByWithLiteralInSubqueryGrouping.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByWithLiteralInSubqueryGrouping.iq new file mode 100644 index 000000000000..18fbcc29a0c7 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByWithLiteralInSubqueryGrouping.iq @@ -0,0 +1,110 @@ +# testGroupByWithLiteralInSubqueryGrouping case-crc:ddf0feaf +# quidem testcase reason: IMPROVED_PLAN +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT + t1, t2 + FROM + ( SELECT + 'dummy' as t1, + CASE + WHEN + dim4 = 'b' + THEN dim4 + ELSE NULL + END AS t2 + FROM + numfoo + GROUP BY + dim4 + ) + GROUP BY + t1,t2 +; ++-------+----+ +| t1 | t2 | ++-------+----+ +| dummy | b | +| dummy | | ++-------+----+ +(2 rows) + +!ok +LogicalProject(t1=['dummy'], t2=[$0]) + LogicalAggregate(group=[{0}]) + LogicalProject(t2=[CASE(=($0, 'b'), $0, null:VARCHAR)]) + LogicalAggregate(group=[{4}]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidProject(t1=['dummy'], t2=[$0], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(t2=[CASE(=($0, 'b'), $0, null:VARCHAR)], druid=[logical]) + DruidAggregate(group=[{4}], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim4", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "case_searched((\"_d0\" == 'b'),\"_d0\",null)", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "'dummy'", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=default.iq new file mode 100644 index 000000000000..01941e6a7e9e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=default.iq @@ -0,0 +1,231 @@ +# testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=default case-crc:6c95ac54 +# quidem testcase reason: AGG_COL_EXCHANGE +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set useApproximateCountDistinct false +!set outputformat mysql +!use druidtest:/// +SELECT dim2, COUNT(*), COUNT(distinct dim1), COUNT(distinct cnt) FROM druid.foo GROUP BY dim2; ++------+--------+--------+--------+ +| dim2 | EXPR$1 | EXPR$2 | EXPR$3 | ++------+--------+--------+--------+ +| | 3 | 3 | 1 | +| a | 2 | 1 | 1 | +| abc | 1 | 1 | 1 | ++------+--------+--------+--------+ +(3 rows) + +!ok +LogicalProject(dim2=[$0], EXPR$1=[$1], EXPR$2=[$3], EXPR$3=[$5]) + LogicalJoin(condition=[IS NOT DISTINCT FROM($0, $4)], joinType=[inner]) + LogicalJoin(condition=[IS NOT DISTINCT FROM($0, $2)], joinType=[inner]) + LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}], EXPR$2=[COUNT($0)]) + LogicalAggregate(group=[{1, 2}]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}], EXPR$3=[COUNT($1)]) + LogicalAggregate(group=[{2, 4}]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim2=[$0], EXPR$1=[$1], EXPR$2=[$3], EXPR$3=[$5], druid=[logical]) + DruidJoin(condition=[IS NOT DISTINCT FROM($0, $4)], joinType=[inner]) + DruidJoin(condition=[IS NOT DISTINCT FROM($0, $2)], joinType=[inner]) + DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$2=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{1, 2}], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], EXPR$3=[COUNT($1)], druid=[logical]) + DruidAggregate(group=[{2, 4}], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "d1", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "a0" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "d0", + "value" : null + } + }, + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "notdistinctfrom(\"d0\",\"j0._d0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "cnt", + "outputName" : "d1", + "outputType" : "LONG" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "d0", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "notdistinctfrom(\"d0\",\"_j0._d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "_j0.a0", "a0", "d0", "j0.a0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=sql.iq new file mode 100644 index 000000000000..d0620db5261d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=sql.iq @@ -0,0 +1,242 @@ +# testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=sql case-crc:6c95ac54 +# quidem testcase reason: AGG_COL_EXCHANGE +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set useApproximateCountDistinct false +!set outputformat mysql +!use druidtest:/// +SELECT dim2, COUNT(*), COUNT(distinct dim1), COUNT(distinct cnt) FROM druid.foo GROUP BY dim2; ++------+--------+--------+--------+ +| dim2 | EXPR$1 | EXPR$2 | EXPR$3 | ++------+--------+--------+--------+ +| | 1 | 1 | 1 | +| a | 2 | 2 | 1 | +| abc | 1 | 1 | 1 | +| | 2 | 2 | 1 | ++------+--------+--------+--------+ +(4 rows) + +!ok +LogicalProject(dim2=[$0], EXPR$1=[$1], EXPR$2=[$3], EXPR$3=[$5]) + LogicalJoin(condition=[IS NOT DISTINCT FROM($0, $4)], joinType=[inner]) + LogicalJoin(condition=[IS NOT DISTINCT FROM($0, $2)], joinType=[inner]) + LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}], EXPR$2=[COUNT($0)]) + LogicalAggregate(group=[{1, 2}]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{0}], EXPR$3=[COUNT($1)]) + LogicalAggregate(group=[{2, 4}]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim2=[$0], EXPR$1=[$1], EXPR$2=[$3], EXPR$3=[$5], druid=[logical]) + DruidJoin(condition=[IS NOT DISTINCT FROM($0, $4)], joinType=[inner]) + DruidJoin(condition=[IS NOT DISTINCT FROM($0, $2)], joinType=[inner]) + DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{1}], EXPR$2=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{1, 2}], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{0}], EXPR$3=[COUNT($1)], druid=[logical]) + DruidAggregate(group=[{2, 4}], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "d1", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "a0" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "null", + "column" : "d0" + } + }, + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "notdistinctfrom(\"d0\",\"j0._d0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "cnt", + "outputName" : "d1", + "outputType" : "LONG" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "d0", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "a0" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "null", + "column" : "d1" + } + }, + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "notdistinctfrom(\"d0\",\"_j0._d0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "_j0.a0", "a0", "d0", "j0.a0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=default.iq new file mode 100644 index 000000000000..1cbbe18e49d8 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=default.iq @@ -0,0 +1,152 @@ +# testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=default case-crc:43cd3215 +# quidem testcase reason: SLIGHTLY_WORSE_PLAN +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set useApproximateCountDistinct false +!set useGroupingSetForExactDistinct true +!set outputformat mysql +!use druidtest:///?NumMergeBuffers=3 +SELECT +(SUM(CASE WHEN (TIMESTAMP '2000-01-04 17:00:00'<=__time AND __time= 947005200000) && (\"__time\" < 1641402000000)),\"dim1\",null)", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "a0" + }, + "filter" : { + "type" : "bound", + "dimension" : "__time", + "lower" : "947005200000", + "upper" : "1641402000000", + "upperStrict" : true, + "ordering" : { + "type" : "numeric" + } + }, + "name" : "a0" + }, { + "type" : "grouping", + "name" : "a1", + "groupings" : [ "v0" ] + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + }, + "subtotalsSpec" : [ [ "d0" ], [ ] ] + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "longMin", + "name" : "_a0", + "fieldName" : "a0" + }, + "filter" : { + "type" : "selector", + "dimension" : "a1", + "value" : "1" + }, + "name" : "_a0" + }, { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a1" + }, + "filter" : { + "type" : "and", + "fields" : [ { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "d0", + "value" : null + } + }, { + "type" : "selector", + "dimension" : "a1", + "value" : "0" + } ] + }, + "name" : "_a1" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "(CAST(\"_a0\", 'DOUBLE') / \"_a1\")", + "outputType" : "DOUBLE" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=sql.iq new file mode 100644 index 000000000000..4f6646b7f537 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=sql.iq @@ -0,0 +1,151 @@ +# testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=sql case-crc:43cd3215 +# quidem testcase reason: SLIGHTLY_WORSE_PLAN +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set useApproximateCountDistinct false +!set useGroupingSetForExactDistinct true +!set outputformat mysql +!use druidtest:///?NumMergeBuffers=3 +SELECT +(SUM(CASE WHEN (TIMESTAMP '2000-01-04 17:00:00'<=__time AND __time= 947005200000) && (\"__time\" < 1641402000000)),\"dim1\",null)", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "a0" + }, + "filter" : { + "type" : "range", + "column" : "__time", + "matchValueType" : "LONG", + "lower" : 947005200000, + "upper" : 1641402000000, + "upperOpen" : true + }, + "name" : "a0" + }, { + "type" : "grouping", + "name" : "a1", + "groupings" : [ "v0" ] + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + }, + "subtotalsSpec" : [ [ "d0" ], [ ] ] + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "longMin", + "name" : "_a0", + "fieldName" : "a0" + }, + "filter" : { + "type" : "equals", + "column" : "a1", + "matchValueType" : "LONG", + "matchValue" : 1 + }, + "name" : "_a0" + }, { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a1" + }, + "filter" : { + "type" : "and", + "fields" : [ { + "type" : "not", + "field" : { + "type" : "null", + "column" : "d0" + } + }, { + "type" : "equals", + "column" : "a1", + "matchValueType" : "LONG", + "matchValue" : 0 + } ] + }, + "name" : "_a1" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "(CAST(\"_a0\", 'DOUBLE') / \"_a1\")", + "outputType" : "DOUBLE" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testRepeatedIdenticalVirtualExpressionGrouping.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testRepeatedIdenticalVirtualExpressionGrouping.iq new file mode 100644 index 000000000000..333ea9002d38 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testRepeatedIdenticalVirtualExpressionGrouping.iq @@ -0,0 +1,71 @@ +# testRepeatedIdenticalVirtualExpressionGrouping case-crc:1a5712f6 +# quidem testcase reason: IMPROVED_PLAN +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT + CASE dim1 WHEN NULL THEN FALSE ELSE TRUE END AS col_a, + CASE dim2 WHEN NULL THEN FALSE ELSE TRUE END AS col_b +FROM foo +GROUP BY 1, 2; ++-------+-------+ +| col_a | col_b | ++-------+-------+ +| true | true | ++-------+-------+ +(1 row) + +!ok +LogicalProject(col_a=[$0], col_b=[true]) + LogicalAggregate(group=[{0}]) + LogicalProject(col_a=[true]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(col_a=[$0], col_b=[true], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(col_a=[true], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "1", + "outputType" : "LONG" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "LONG" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testSubqueryTypeMismatchWithLiterals@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testSubqueryTypeMismatchWithLiterals@NullHandling=default.iq new file mode 100644 index 000000000000..88eecd621478 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testSubqueryTypeMismatchWithLiterals@NullHandling=default.iq @@ -0,0 +1,110 @@ +# testSubqueryTypeMismatchWithLiterals@NullHandling=default case-crc:55baf28f +# quidem testcase reason: AGGREGATE_REMOVE_NOT_FIRED +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT + dim1, + SUM(CASE WHEN sum_l1 = 0 THEN 1 ELSE 0 END) AS outer_l1 +from ( + select + dim1, + SUM(l1) as sum_l1 + from numfoo + group by dim1 +) +group by 1; ++------+----------+ +| dim1 | outer_l1 | ++------+----------+ +| | 0 | +| 1 | 1 | +| 10.1 | 0 | +| 2 | 1 | +| abc | 1 | +| def | 1 | ++------+----------+ +(6 rows) + +!ok +LogicalAggregate(group=[{0}], outer_l1=[COUNT() FILTER $1]) + LogicalProject(dim1=[$0], $f2=[=($1, 0)]) + LogicalAggregate(group=[{1}], sum_l1=[SUM($11)]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{0}], outer_l1=[COUNT() FILTER $1], druid=[logical]) + DruidProject(dim1=[$0], $f2=[=($1, 0)], druid=[logical]) + DruidAggregate(group=[{1}], sum_l1=[SUM($11)], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "longSum", + "name" : "a0", + "fieldName" : "l1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "_d0", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a0" + }, + "filter" : { + "type" : "selector", + "dimension" : "a0", + "value" : "0" + }, + "name" : "_a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testSubqueryTypeMismatchWithLiterals@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testSubqueryTypeMismatchWithLiterals@NullHandling=sql.iq new file mode 100644 index 000000000000..1ccd29e4854d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testSubqueryTypeMismatchWithLiterals@NullHandling=sql.iq @@ -0,0 +1,111 @@ +# testSubqueryTypeMismatchWithLiterals@NullHandling=sql case-crc:55baf28f +# quidem testcase reason: AGGREGATE_REMOVE_NOT_FIRED +!set sqlQueryId dummy +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set debug true +!set outputformat mysql +!use druidtest:/// +SELECT + dim1, + SUM(CASE WHEN sum_l1 = 0 THEN 1 ELSE 0 END) AS outer_l1 +from ( + select + dim1, + SUM(l1) as sum_l1 + from numfoo + group by dim1 +) +group by 1; ++------+----------+ +| dim1 | outer_l1 | ++------+----------+ +| | 0 | +| 1 | 0 | +| 10.1 | 0 | +| 2 | 1 | +| abc | 0 | +| def | 0 | ++------+----------+ +(6 rows) + +!ok +LogicalAggregate(group=[{0}], outer_l1=[COUNT() FILTER $1]) + LogicalProject(dim1=[$0], $f2=[IS TRUE(=($1, 0))]) + LogicalAggregate(group=[{1}], sum_l1=[SUM($11)]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{0}], outer_l1=[COUNT() FILTER $1], druid=[logical]) + DruidProject(dim1=[$0], $f2=[IS TRUE(=($1, 0))], druid=[logical]) + DruidAggregate(group=[{1}], sum_l1=[SUM($11)], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "_d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "longSum", + "name" : "a0", + "fieldName" : "l1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "_d0", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a0" + }, + "filter" : { + "type" : "equals", + "column" : "a0", + "matchValueType" : "LONG", + "matchValue" : 0 + }, + "name" : "_a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=default.iq new file mode 100644 index 000000000000..e4c3fbb28f18 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=default.iq @@ -0,0 +1,191 @@ +# testWindowingWithScanAndSort@NullHandling=default case-crc:24efe380 +# quidem testcase reason: SLIGHTLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set enableWindowing true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +with t AS ( +SELECT + RANK() OVER (PARTITION BY m2 ORDER BY m2 ASC) + AS ranking, + COUNT(m1) as trend_score +FROM foo +GROUP BY m2,m1 LIMIT 10 +) +select ranking, trend_score from t ORDER BY trend_score; ++---------+-------------+ +| ranking | trend_score | ++---------+-------------+ +| 1 | 1 | +| 1 | 1 | +| 1 | 1 | +| 1 | 1 | +| 1 | 1 | +| 1 | 1 | ++---------+-------------+ +(6 rows) + +!ok +LogicalProject(ranking=[$2], trend_score=[$1]) + LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalSort(fetch=[10]) + LogicalWindow(window#0=[window(partition {0} order by [0] aggs [RANK()])]) + LogicalProject(m2=[$1], trend_score=[$2]) + LogicalAggregate(group=[{5, 6}], trend_score=[COUNT()]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(ranking=[$2], trend_score=[$1], druid=[logical]) + DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidSort(fetch=[10], druid=[logical]) + DruidWindow(window#0=[window(partition {0} order by [0] aggs [RANK()])]) + DruidProject(m2=[$1], trend_score=[$2], druid=[logical]) + DruidAggregate(group=[{5, 6}], trend_score=[COUNT()], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "windowOperator", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "windowOperator", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "m1", + "outputName" : "d0", + "outputType" : "FLOAT" + }, { + "type" : "default", + "dimension" : "m2", + "outputName" : "d1", + "outputType" : "DOUBLE" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "LegacySegmentSpec", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "outputSignature" : [ { + "name" : "d1", + "type" : "DOUBLE" + }, { + "name" : "a0", + "type" : "LONG" + }, { + "name" : "w0", + "type" : "LONG" + } ], + "operatorDefinition" : [ { + "type" : "naiveSort", + "columns" : [ { + "column" : "d1", + "direction" : "ASC" + } ] + }, { + "type" : "naivePartition", + "partitionColumns" : [ "d1" ] + }, { + "type" : "window", + "processor" : { + "type" : "rank", + "group" : [ "d1" ], + "outputColumn" : "w0", + "asPercent" : false + } + } ], + "leafOperators" : [ ], + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "limit" : 10, + "columns" : [ "a0", "d1", "w0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "DOUBLE", "LONG" ], + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "LegacySegmentSpec", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "outputSignature" : [ { + "name" : "w0", + "type" : "LONG" + }, { + "name" : "a0", + "type" : "LONG" + } ], + "operatorDefinition" : [ { + "type" : "naiveSort", + "columns" : [ { + "column" : "a0", + "direction" : "ASC" + } ] + }, { + "type" : "scan", + "timeRange" : null, + "filter" : null, + "offsetLimit" : null, + "projectedColumns" : [ "w0", "a0" ], + "virtualColumns" : null, + "ordering" : null + } ], + "leafOperators" : [ { + "type" : "scan", + "timeRange" : null, + "filter" : null, + "offsetLimit" : { + "offset" : 0, + "limit" : 9223372036854775807 + }, + "projectedColumns" : [ "a0", "w0" ], + "virtualColumns" : null, + "ordering" : null + } ], + "granularity" : { + "type" : "all" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=sql.iq new file mode 100644 index 000000000000..8e18b7fb640b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=sql.iq @@ -0,0 +1,202 @@ +# testWindowingWithScanAndSort@NullHandling=sql case-crc:24efe380 +# quidem testcase reason: SLIGHTLY_WORSE_PLAN +!set sqlQueryId dummy +!set defaultTimeout 300000 +!set debug true +!set enableWindowing true +!set maxScatterGatherBytes 9223372036854775807 +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set plannerStrategy DECOUPLED +!set outputformat mysql +!use druidtest:/// +with t AS ( +SELECT + RANK() OVER (PARTITION BY m2 ORDER BY m2 ASC) + AS ranking, + COUNT(m1) as trend_score +FROM foo +GROUP BY m2,m1 LIMIT 10 +) +select ranking, trend_score from t ORDER BY trend_score; ++---------+-------------+ +| ranking | trend_score | ++---------+-------------+ +| 1 | 1 | +| 1 | 1 | +| 1 | 1 | +| 1 | 1 | +| 1 | 1 | +| 1 | 1 | ++---------+-------------+ +(6 rows) + +!ok +LogicalProject(ranking=[$2], trend_score=[$1]) + LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalSort(fetch=[10]) + LogicalWindow(window#0=[window(partition {0} order by [0] aggs [RANK()])]) + LogicalProject(m2=[$1], trend_score=[$2]) + LogicalAggregate(group=[{5, 6}], trend_score=[COUNT($5)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(ranking=[$2], trend_score=[$1], druid=[logical]) + DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidSort(fetch=[10], druid=[logical]) + DruidWindow(window#0=[window(partition {0} order by [0] aggs [RANK()])]) + DruidProject(m2=[$1], trend_score=[$2], druid=[logical]) + DruidAggregate(group=[{5, 6}], trend_score=[COUNT($5)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "windowOperator", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "windowOperator", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "m1", + "outputName" : "d0", + "outputType" : "FLOAT" + }, { + "type" : "default", + "dimension" : "m2", + "outputName" : "d1", + "outputType" : "DOUBLE" + } ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "a0" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "null", + "column" : "m1" + } + }, + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "intervals" : { + "type" : "LegacySegmentSpec", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "outputSignature" : [ { + "name" : "d1", + "type" : "DOUBLE" + }, { + "name" : "a0", + "type" : "LONG" + }, { + "name" : "w0", + "type" : "LONG" + } ], + "operatorDefinition" : [ { + "type" : "naiveSort", + "columns" : [ { + "column" : "d1", + "direction" : "ASC" + } ] + }, { + "type" : "naivePartition", + "partitionColumns" : [ "d1" ] + }, { + "type" : "window", + "processor" : { + "type" : "rank", + "group" : [ "d1" ], + "outputColumn" : "w0", + "asPercent" : false + } + } ], + "leafOperators" : [ ], + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "limit" : 10, + "columns" : [ "a0", "d1", "w0" ], + "legacy" : false, + "columnTypes" : [ "LONG", "DOUBLE", "LONG" ], + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "LegacySegmentSpec", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "outputSignature" : [ { + "name" : "w0", + "type" : "LONG" + }, { + "name" : "a0", + "type" : "LONG" + } ], + "operatorDefinition" : [ { + "type" : "naiveSort", + "columns" : [ { + "column" : "a0", + "direction" : "ASC" + } ] + }, { + "type" : "scan", + "timeRange" : null, + "filter" : null, + "offsetLimit" : null, + "projectedColumns" : [ "w0", "a0" ], + "virtualColumns" : null, + "ordering" : null + } ], + "leafOperators" : [ { + "type" : "scan", + "timeRange" : null, + "filter" : null, + "offsetLimit" : { + "offset" : 0, + "limit" : 9223372036854775807 + }, + "projectedColumns" : [ "a0", "w0" ], + "virtualColumns" : null, + "ordering" : null + } ], + "granularity" : { + "type" : "all" + } +} +!nativePlan